18794: Merge branch 'main'
authorTom Clegg <tom@curii.com>
Mon, 9 May 2022 13:49:36 +0000 (09:49 -0400)
committerTom Clegg <tom@curii.com>
Mon, 9 May 2022 13:49:36 +0000 (09:49 -0400)
Arvados-DCO-1.1-Signed-off-by: Tom Clegg <tom@curii.com>

26 files changed:
apps/workbench/app/controllers/management_controller.rb [moved from apps/workbench/app/controllers/healthcheck_controller.rb with 54% similarity]
apps/workbench/config/arvados_config.rb
apps/workbench/config/initializers/reload_config.rb [new file with mode: 0644]
apps/workbench/config/routes.rb
apps/workbench/test/controllers/healthcheck_controller_test.rb [deleted file]
apps/workbench/test/controllers/management_controller_test.rb [new file with mode: 0644]
cmd/arvados-server/cmd.go
doc/_includes/_restart_api.liquid
doc/admin/maintenance-and-upgrading.html.textile.liquid
doc/install/install-api-server.html.textile.liquid
lib/config/load.go
lib/config/load_test.go
lib/controller/rails_restart_test.go [new file with mode: 0644]
lib/service/cmd.go
sdk/go/arvados/config.go
sdk/go/health/aggregator.go
sdk/go/health/aggregator_test.go
sdk/python/tests/run_test_server.py
services/api/app/controllers/arvados/v1/healthcheck_controller.rb [deleted file]
services/api/app/controllers/arvados/v1/management_controller.rb [new file with mode: 0644]
services/api/config/arvados_config.rb
services/api/config/initializers/reload_config.rb [new file with mode: 0644]
services/api/config/routes.rb
services/api/test/functional/arvados/v1/healthcheck_controller_test.rb [deleted file]
services/api/test/functional/arvados/v1/management_controller_test.rb [new file with mode: 0644]
services/api/test/integration/errors_test.rb

similarity index 54%
rename from apps/workbench/app/controllers/healthcheck_controller.rb
rename to apps/workbench/app/controllers/management_controller.rb
index 691bef8ee578bbd3beeb616906ac44a4df253631..4c8b52f6615f44b5c2765747eaa93f00a58aa3ec 100644 (file)
@@ -2,7 +2,9 @@
 #
 # SPDX-License-Identifier: AGPL-3.0
 
-class HealthcheckController < ApplicationController
+require 'app_version'
+
+class ManagementController < ApplicationController
   skip_around_action :thread_clear
   skip_around_action :set_thread_api_token
   skip_around_action :require_thread_api_token
@@ -28,7 +30,21 @@ class HealthcheckController < ApplicationController
     end
   end
 
-  def ping
+  def metrics
+    render content_type: 'text/plain', plain: <<~EOF
+# HELP arvados_config_load_timestamp_seconds Time when config file was loaded.
+# TYPE arvados_config_load_timestamp_seconds gauge
+arvados_config_load_timestamp_seconds{sha256="#{Rails.configuration.SourceSHA256}"} #{Rails.configuration.LoadTimestamp.to_f}
+# HELP arvados_config_source_timestamp_seconds Timestamp of config file when it was loaded.
+# TYPE arvados_config_source_timestamp_seconds gauge
+arvados_config_source_timestamp_seconds{sha256="#{Rails.configuration.SourceSHA256}"} #{Rails.configuration.SourceTimestamp.to_f}
+# HELP arvados_version_running Indicated version is running.
+# TYPE arvados_version_running gauge
+arvados_version_running{version="#{AppVersion.package_version}"} 1
+EOF
+  end
+
+  def health
     resp = {"health" => "OK"}
     render json: resp
   end
index 7cc46d2983490896c36e96c9081bf3e74013efa2..86b4a47539971e5aa9502676367227595d4179af 100644 (file)
@@ -21,6 +21,7 @@ require 'open3'
 
 # Load the defaults, used by config:migrate and fallback loading
 # legacy application.yml
+load_time = Time.now.utc
 defaultYAML, stderr, status = Open3.capture3("arvados-server", "config-dump", "-config=-", "-skip-legacy", stdin_data: "Clusters: {xxxxx: {}}")
 if !status.success?
   puts stderr
@@ -30,6 +31,8 @@ confs = YAML.load(defaultYAML, deserialize_symbols: false)
 clusterID, clusterConfig = confs["Clusters"].first
 $arvados_config_defaults = clusterConfig
 $arvados_config_defaults["ClusterID"] = clusterID
+$arvados_config_defaults["SourceTimestamp"] = Time.rfc3339(confs["SourceTimestamp"])
+$arvados_config_defaults["SourceSHA256"] = confs["SourceSHA256"]
 
 if ENV["ARVADOS_CONFIG"] == "none"
   # Don't load config. This magic value is set by packaging scripts so
@@ -45,6 +48,8 @@ else
       clusterID, clusterConfig = confs["Clusters"].first
       $arvados_config_global = clusterConfig
       $arvados_config_global["ClusterID"] = clusterID
+      $arvados_config_global["SourceTimestamp"] = Time.rfc3339(confs["SourceTimestamp"])
+      $arvados_config_global["SourceSHA256"] = confs["SourceSHA256"]
     else
       # config-dump failed, assume we will be loading from legacy
       # application.yml, initialize with defaults.
@@ -55,6 +60,7 @@ end
 
 # Now make a copy
 $arvados_config = $arvados_config_global.deep_dup
+$arvados_config["LoadTimestamp"] = load_time
 
 # Declare all our configuration items.
 arvcfg = ConfigLoader.new
diff --git a/apps/workbench/config/initializers/reload_config.rb b/apps/workbench/config/initializers/reload_config.rb
new file mode 100644 (file)
index 0000000..58949c5
--- /dev/null
@@ -0,0 +1,70 @@
+# Copyright (C) The Arvados Authors. All rights reserved.
+#
+# SPDX-License-Identifier: AGPL-3.0
+
+# Please don't edit this version. Instead, edit
+# services/api/config/initializers/reload_config.rb and update this
+# copy. Or find a more reasonable way to share the code.
+
+if !File.owned?(Rails.root.join('tmp'))
+  Rails.logger.debug("reload_config: not owner of #{Rails.root}/tmp, skipping")
+elsif ENV["ARVADOS_CONFIG"] == "none"
+  Rails.logger.debug("reload_config: no config in use, skipping")
+else
+  Thread.new do
+    lockfile = Rails.root.join('tmp', 'reload_config.lock')
+    File.open(lockfile, File::WRONLY|File::CREAT, 0600) do |f|
+      # Note we don't use LOCK_NB here. If we did, each time passenger
+      # kills the lock-holder process, we would be left with nobody
+      # checking for updates until passenger starts a new worker,
+      # which could be a long time.
+      Rails.logger.debug("reload_config: waiting for lock on #{lockfile}")
+      f.flock(File::LOCK_EX)
+
+      t_lastload = Rails.configuration.SourceTimestamp
+      hash_lastload = Rails.configuration.SourceSHA256
+      conffile = ENV['ARVADOS_CONFIG'] || "/etc/arvados/config.yml"
+      Rails.logger.info("reload_config: polling for updated mtime on #{conffile} with threshold #{t_lastload}")
+      while true
+        sleep 1
+        t = File.mtime(conffile)
+        # If the file is newer than 5s, re-read it even if the
+        # timestamp matches the previously loaded file. This enables
+        # us to detect changes even if the filesystem's timestamp
+        # precision cannot represent multiple updates per second.
+        if t.to_f != t_lastload.to_f || Time.now.to_f - t.to_f < 5
+          Open3.popen2("arvados-server", "config-dump", "-skip-legacy") do |stdin, stdout, status_thread|
+            confs = YAML.load(stdout, deserialize_symbols: false)
+            hash = confs["SourceSHA256"]
+          rescue => e
+            Rails.logger.info("reload_config: config file updated but could not be loaded: #{e}")
+            t_lastload = t
+            continue
+          end
+          if hash == hash_lastload
+            # If we reloaded a new or updated file, but the content is
+            # identical, keep polling instead of restarting.
+            t_lastload = t
+            continue
+          end
+
+          restartfile = Rails.root.join('tmp', 'restart.txt')
+          touchtime = Time.now
+          Rails.logger.info("reload_config: mtime on #{conffile} changed to #{t}, touching #{restartfile} to #{touchtime}")
+          begin
+            File.utime(touchtime, touchtime, restartfile)
+          rescue
+            # remove + re-create works even if the existing file is
+            # owned by root, provided the tempdir is writable.
+            File.unlink(restartfile) rescue nil
+            File.open(restartfile, 'w') {}
+          end
+          # Even if passenger doesn't notice that we hit restart.txt
+          # and kill our process, there's no point waiting around to
+          # hit it again.
+          break
+        end
+      end
+    end
+  end
+end
index ffc09ac933acf8d88fa1b07cac460c144a805a45..0bf8dffc67be2602760b68a55f2d2bd1580ac303 100644 (file)
@@ -130,7 +130,8 @@ Rails.application.routes.draw do
 
   root :to => 'projects#index'
 
-  match '/_health/ping', to: 'healthcheck#ping', via: [:get]
+  match '/_health/:check', to: 'management#health', via: [:get]
+  match '/metrics', to: 'management#metrics', via: [:get]
 
   get '/tests/mithril', to: 'tests#mithril'
 
diff --git a/apps/workbench/test/controllers/healthcheck_controller_test.rb b/apps/workbench/test/controllers/healthcheck_controller_test.rb
deleted file mode 100644 (file)
index c3a0dde..0000000
+++ /dev/null
@@ -1,37 +0,0 @@
-# Copyright (C) The Arvados Authors. All rights reserved.
-#
-# SPDX-License-Identifier: AGPL-3.0
-
-require 'test_helper'
-
-class HealthcheckControllerTest < ActionController::TestCase
-  reset_api_fixtures :after_each_test, false
-  reset_api_fixtures :after_suite, false
-
-  [
-    [false, nil, 404, 'disabled'],
-    [true, nil, 401, 'authorization required'],
-    [true, 'badformatwithnoBearer', 403, 'authorization error'],
-    [true, 'Bearer wrongtoken', 403, 'authorization error'],
-    [true, 'Bearer configuredmanagementtoken', 200, '{"health":"OK"}'],
-  ].each do |enabled, header, error_code, error_msg|
-    test "ping when #{if enabled then 'enabled' else 'disabled' end} with header '#{header}'" do
-      if enabled
-        Rails.configuration.ManagementToken = 'configuredmanagementtoken'
-      else
-        Rails.configuration.ManagementToken = ""
-      end
-
-      @request.headers['Authorization'] = header
-      get :ping
-      assert_response error_code
-
-      resp = JSON.parse(@response.body)
-      if error_code == 200
-        assert_equal(JSON.load('{"health":"OK"}'), resp)
-      else
-        assert_equal(resp['errors'], error_msg)
-      end
-    end
-  end
-end
diff --git a/apps/workbench/test/controllers/management_controller_test.rb b/apps/workbench/test/controllers/management_controller_test.rb
new file mode 100644 (file)
index 0000000..80dc944
--- /dev/null
@@ -0,0 +1,76 @@
+# Copyright (C) The Arvados Authors. All rights reserved.
+#
+# SPDX-License-Identifier: AGPL-3.0
+
+require 'test_helper'
+
+class ManagementControllerTest < ActionController::TestCase
+  reset_api_fixtures :after_each_test, false
+  reset_api_fixtures :after_suite, false
+
+  [
+    [false, nil, 404, 'disabled'],
+    [true, nil, 401, 'authorization required'],
+    [true, 'badformatwithnoBearer', 403, 'authorization error'],
+    [true, 'Bearer wrongtoken', 403, 'authorization error'],
+    [true, 'Bearer configuredmanagementtoken', 200, '{"health":"OK"}'],
+  ].each do |enabled, header, error_code, error_msg|
+    test "health check ping when #{if enabled then 'enabled' else 'disabled' end} with header '#{header}'" do
+      if enabled
+        Rails.configuration.ManagementToken = 'configuredmanagementtoken'
+      else
+        Rails.configuration.ManagementToken = ""
+      end
+
+      @request.headers['Authorization'] = header
+      get(:health, params: {check: 'ping'})
+      assert_response error_code
+
+      resp = JSON.parse(@response.body)
+      if error_code == 200
+        assert_equal(JSON.load('{"health":"OK"}'), resp)
+      else
+        assert_equal(resp['errors'], error_msg)
+      end
+    end
+  end
+
+  test "metrics" do
+    mtime = File.mtime(ENV["ARVADOS_CONFIG"])
+    hash = Digest::SHA256.hexdigest(File.read(ENV["ARVADOS_CONFIG"]))
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    @request.headers['Authorization'] = "Bearer configuredmanagementtoken"
+    get :metrics
+    assert_response :success
+    assert_equal 'text/plain', @response.content_type
+
+    assert_match /\narvados_config_source_timestamp_seconds{sha256="#{hash}"} #{Regexp.escape mtime.utc.to_f.to_s}\n/, @response.body
+
+    # Expect mtime < loadtime < now
+    m = @response.body.match(/\narvados_config_load_timestamp_seconds{sha256="#{hash}"} (.*?)\n/)
+    assert_operator m[1].to_f, :>, mtime.utc.to_f
+    assert_operator m[1].to_f, :<, Time.now.utc.to_f
+
+    assert_match /\narvados_version_running{version="#{Regexp.escape AppVersion.package_version}"} 1\n/, @response.body
+  end
+
+  test "metrics disabled" do
+    Rails.configuration.ManagementToken = ""
+    @request.headers['Authorization'] = "Bearer configuredmanagementtoken"
+    get :metrics
+    assert_response 404
+  end
+
+  test "metrics bad token" do
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    @request.headers['Authorization'] = "Bearer asdf"
+    get :metrics
+    assert_response 403
+  end
+
+  test "metrics unauthorized" do
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    get :metrics
+    assert_response 401
+  end
+end
index c9a91c872160cb74c10d098c4ae2f0f9c16ce667..ae1e3fbeee2d1e2d4310bb3f221b1f14bf77b17b 100644 (file)
@@ -21,6 +21,7 @@ import (
        "git.arvados.org/arvados.git/lib/install"
        "git.arvados.org/arvados.git/lib/lsf"
        "git.arvados.org/arvados.git/lib/recovercollection"
+       "git.arvados.org/arvados.git/sdk/go/health"
        "git.arvados.org/arvados.git/services/githttpd"
        keepbalance "git.arvados.org/arvados.git/services/keep-balance"
        keepweb "git.arvados.org/arvados.git/services/keep-web"
@@ -36,6 +37,7 @@ var (
                "--version": cmd.Version,
 
                "boot":               boot.Command,
+               "check":              health.CheckCommand,
                "cloudtest":          cloudtest.Command,
                "config-check":       config.CheckCommand,
                "config-defaults":    config.DumpDefaultsCommand,
index c3e0330b8715adc2256fab1c53145393a3d2336a..d6c5c3433e4050ac32385f2a5c0efaed7e017d7d 100644 (file)
@@ -1,8 +1,15 @@
+{% comment %}
+Copyright (C) The Arvados Authors. All rights reserved.
+
+SPDX-License-Identifier: CC-BY-SA-3.0
+{% endcomment %}
+
 h2(#restart-api). Restart the API server and controller
 
 *Make sure the cluster config file is up to date on the API server host* then restart the API server and controller processes to ensure the configuration changes are visible to the whole cluster.
 
 <notextile>
 <pre><code># <span class="userinput">systemctl restart nginx arvados-controller</span>
+# <span class="userinput">arvados-server check</span>
 </code></pre>
 </notextile>
index 0d517fe915c943e205215a1fd38eff9d27b23915..3cc80a35609f46c9909bdb2c0ce9785ba80ac8fa 100644 (file)
@@ -50,6 +50,8 @@ h3(#restart). Restart the services affected by the change
 
 If you know which Arvados service uses the specific configuration that was modified, restart those services. When in doubt, restart all Arvados system services.
 
+To check for services that have not restarted since the configuration file was updated, run the @arvados-server check@ command on each system node.
+
 h2(#upgrading). Upgrading Arvados
 
 Upgrading Arvados typically involves the following steps:
@@ -61,4 +63,6 @@ Upgrading Arvados typically involves the following steps:
 # rebuild and deploy the "compute node image":{{site.baseurl}}/install/crunch2-cloud/install-compute-node.html (cloud only)
 # Install new packages using @apt-get upgrade@ or @yum upgrade@.
 # Wait for package installation scripts as they perform any necessary data migrations.
+# Run @arvados-server config-check@ to detect configuration errors or deprecated entries.
 # Verify that the Arvados services were restarted as part of the package upgrades.
+# Run @arvados-server check@ to detect services that did not restart properly.
index 7d0353c9e71283617e0bb6c5e53c1de89204663b..6c3eabba4f063e942f7dffba99bfa296dc258a58 100644 (file)
@@ -211,8 +211,7 @@ Confirm working Rails API server:
 
 Confirm that you can use the system root token to act as the system root user:
 
-<notextile><pre><code>
-$ curl -H "Authorization: Bearer $system_root_token" https://<span class="userinput">ClusterID.example.com</span>/arvados/v1/users/current
+<notextile><pre><code>$ curl -H "Authorization: Bearer $system_root_token" https://<span class="userinput">ClusterID.example.com</span>/arvados/v1/users/current
 </code></pre></notextile>
 
 h3. Troubleshooting
index 6099215edc2f4d4fb5c014d2ee6b48aa327d730e..8f8ab2bf27312adb76b6597ef1f51f8e21ecb338 100644 (file)
@@ -6,6 +6,7 @@ package config
 
 import (
        "bytes"
+       "crypto/sha256"
        _ "embed"
        "encoding/json"
        "errors"
@@ -17,10 +18,12 @@ import (
        "regexp"
        "strconv"
        "strings"
+       "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "github.com/ghodss/yaml"
        "github.com/imdario/mergo"
+       "github.com/prometheus/client_golang/prometheus"
        "github.com/sirupsen/logrus"
 )
 
@@ -46,6 +49,12 @@ type Loader struct {
        KeepBalancePath         string
 
        configdata []byte
+       // UTC time for configdata: either the modtime of the file we
+       // read configdata from, or the time when we read configdata
+       // from a pipe.
+       sourceTimestamp time.Time
+       // UTC time when configdata was read.
+       loadTimestamp time.Time
 }
 
 // NewLoader returns a new Loader with Stdin and Logger set to the
@@ -166,25 +175,36 @@ func (ldr *Loader) MungeLegacyConfigArgs(lgr logrus.FieldLogger, args []string,
        return munged
 }
 
-func (ldr *Loader) loadBytes(path string) ([]byte, error) {
+func (ldr *Loader) loadBytes(path string) (buf []byte, sourceTime, loadTime time.Time, err error) {
+       loadTime = time.Now().UTC()
        if path == "-" {
-               return ioutil.ReadAll(ldr.Stdin)
+               buf, err = ioutil.ReadAll(ldr.Stdin)
+               sourceTime = loadTime
+               return
        }
        f, err := os.Open(path)
        if err != nil {
-               return nil, err
+               return
        }
        defer f.Close()
-       return ioutil.ReadAll(f)
+       fi, err := f.Stat()
+       if err != nil {
+               return
+       }
+       sourceTime = fi.ModTime().UTC()
+       buf, err = ioutil.ReadAll(f)
+       return
 }
 
 func (ldr *Loader) Load() (*arvados.Config, error) {
        if ldr.configdata == nil {
-               buf, err := ldr.loadBytes(ldr.Path)
+               buf, sourceTime, loadTime, err := ldr.loadBytes(ldr.Path)
                if err != nil {
                        return nil, err
                }
                ldr.configdata = buf
+               ldr.sourceTimestamp = sourceTime
+               ldr.loadTimestamp = loadTime
        }
 
        // FIXME: We should reject YAML if the same key is used twice
@@ -330,6 +350,8 @@ func (ldr *Loader) Load() (*arvados.Config, error) {
                        }
                }
        }
+       cfg.SourceTimestamp = ldr.sourceTimestamp
+       cfg.SourceSHA256 = fmt.Sprintf("%x", sha256.Sum256(ldr.configdata))
        return &cfg, nil
 }
 
@@ -555,3 +577,30 @@ func (ldr *Loader) autofillPreemptible(label string, cc *arvados.Cluster) {
        }
 
 }
+
+// RegisterMetrics registers metrics showing the timestamp and content
+// hash of the currently loaded config.
+//
+// Must not be called more than once for a given registry. Must not be
+// called before Load(). Metrics are not updated by subsequent calls
+// to Load().
+func (ldr *Loader) RegisterMetrics(reg *prometheus.Registry) {
+       hash := fmt.Sprintf("%x", sha256.Sum256(ldr.configdata))
+       vec := prometheus.NewGaugeVec(prometheus.GaugeOpts{
+               Namespace: "arvados",
+               Subsystem: "config",
+               Name:      "source_timestamp_seconds",
+               Help:      "Timestamp of config file when it was loaded.",
+       }, []string{"sha256"})
+       vec.WithLabelValues(hash).Set(float64(ldr.sourceTimestamp.UnixNano()) / 1e9)
+       reg.MustRegister(vec)
+
+       vec = prometheus.NewGaugeVec(prometheus.GaugeOpts{
+               Namespace: "arvados",
+               Subsystem: "config",
+               Name:      "load_timestamp_seconds",
+               Help:      "Time when config file was loaded.",
+       }, []string{"sha256"})
+       vec.WithLabelValues(hash).Set(float64(ldr.loadTimestamp.UnixNano()) / 1e9)
+       reg.MustRegister(vec)
+}
index 2d87b906c9b399f49f243d170984a31d712f6ed9..abf321705662f19e8f1bdb006cee71f7d87da659 100644 (file)
@@ -12,13 +12,18 @@ import (
        "os"
        "os/exec"
        "reflect"
+       "regexp"
        "strings"
        "testing"
+       "time"
 
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/ctxlog"
        "github.com/ghodss/yaml"
+       "github.com/prometheus/client_golang/prometheus"
+       "github.com/prometheus/common/expfmt"
        "github.com/sirupsen/logrus"
+       "golang.org/x/sys/unix"
        check "gopkg.in/check.v1"
 )
 
@@ -315,8 +320,16 @@ Clusters:
        c.Assert(err, check.IsNil)
        yaml, err := yaml.Marshal(cfg)
        c.Assert(err, check.IsNil)
+       // Well, *nearly* no warnings. SourceTimestamp and
+       // SourceSHA256 are included in a config-dump, but not
+       // expected in a real config file.
+       yaml = regexp.MustCompile(`(^|\n)(Source(Timestamp|SHA256): .*?\n)+`).ReplaceAll(yaml, []byte("$1"))
        cfgDumped, err := testLoader(c, string(yaml), &logbuf).Load()
        c.Assert(err, check.IsNil)
+       // SourceTimestamp and SourceSHA256 aren't expected to be
+       // preserved through dump+load
+       cfgDumped.SourceTimestamp = cfg.SourceTimestamp
+       cfgDumped.SourceSHA256 = cfg.SourceSHA256
        c.Check(cfg, check.DeepEquals, cfgDumped)
        c.Check(logbuf.String(), check.Equals, "")
 }
@@ -499,6 +512,12 @@ func checkEquivalentLoaders(c *check.C, gotldr, expectedldr *Loader) {
        c.Assert(err, check.IsNil)
        expected, err := expectedldr.Load()
        c.Assert(err, check.IsNil)
+       // The inputs generally aren't even files, so SourceTimestamp
+       // can't be expected to match.
+       got.SourceTimestamp = expected.SourceTimestamp
+       // Obviously the content isn't identical -- otherwise we
+       // wouldn't need to check that it's equivalent.
+       got.SourceSHA256 = expected.SourceSHA256
        checkEqualYAML(c, got, expected)
 }
 
@@ -758,3 +777,46 @@ Clusters:
        c.Check(logbuf.String(), check.Not(check.Matches), `(?ms).*Type2\.preemptible.*`)
        c.Check(logbuf.String(), check.Not(check.Matches), `(?ms).*(z1111|z2222)[^\n]*InstanceTypes.*`)
 }
+
+func (s *LoadSuite) TestSourceTimestamp(c *check.C) {
+       conftime, err := time.Parse(time.RFC3339, "2022-03-04T05:06:07-08:00")
+       c.Assert(err, check.IsNil)
+       confdata := `Clusters: {zzzzz: {}}`
+       conffile := c.MkDir() + "/config.yml"
+       ioutil.WriteFile(conffile, []byte(confdata), 0777)
+       tv := unix.NsecToTimeval(conftime.UnixNano())
+       unix.Lutimes(conffile, []unix.Timeval{tv, tv})
+       for _, trial := range []struct {
+               configarg  string
+               expectTime time.Time
+       }{
+               {"-", time.Now()},
+               {conffile, conftime},
+       } {
+               c.Logf("trial: %+v", trial)
+               ldr := NewLoader(strings.NewReader(confdata), ctxlog.TestLogger(c))
+               ldr.Path = trial.configarg
+               cfg, err := ldr.Load()
+               c.Assert(err, check.IsNil)
+               c.Check(cfg.SourceTimestamp, check.Equals, cfg.SourceTimestamp.UTC())
+               c.Check(cfg.SourceTimestamp, check.Equals, ldr.sourceTimestamp)
+               c.Check(int(cfg.SourceTimestamp.Sub(trial.expectTime).Seconds()), check.Equals, 0)
+               c.Check(int(ldr.loadTimestamp.Sub(time.Now()).Seconds()), check.Equals, 0)
+
+               var buf bytes.Buffer
+               reg := prometheus.NewRegistry()
+               ldr.RegisterMetrics(reg)
+               enc := expfmt.NewEncoder(&buf, expfmt.FmtText)
+               got, _ := reg.Gather()
+               for _, mf := range got {
+                       enc.Encode(mf)
+               }
+               c.Check(buf.String(), check.Matches, `# HELP .*
+# TYPE .*
+arvados_config_load_timestamp_seconds{sha256="83aea5d82eb1d53372cd65c936c60acc1c6ef946e61977bbca7cfea709d201a8"} \Q`+fmt.Sprintf("%g", float64(ldr.loadTimestamp.UnixNano())/1e9)+`\E
+# HELP .*
+# TYPE .*
+arvados_config_source_timestamp_seconds{sha256="83aea5d82eb1d53372cd65c936c60acc1c6ef946e61977bbca7cfea709d201a8"} \Q`+fmt.Sprintf("%g", float64(cfg.SourceTimestamp.UnixNano())/1e9)+`\E
+`)
+       }
+}
diff --git a/lib/controller/rails_restart_test.go b/lib/controller/rails_restart_test.go
new file mode 100644 (file)
index 0000000..bb307d3
--- /dev/null
@@ -0,0 +1,89 @@
+// Copyright (C) The Arvados Authors. All rights reserved.
+//
+// SPDX-License-Identifier: AGPL-3.0
+
+package controller
+
+import (
+       "bytes"
+       "crypto/sha256"
+       "crypto/tls"
+       "fmt"
+       "io/ioutil"
+       "net/http"
+       "net/url"
+       "os"
+       "strings"
+       "time"
+
+       "git.arvados.org/arvados.git/lib/config"
+       "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       check "gopkg.in/check.v1"
+)
+
+var _ = check.Suite(&railsRestartSuite{})
+
+type railsRestartSuite struct{}
+
+// This tests RailsAPI, not controller -- but tests RailsAPI's
+// integration with passenger, so it needs to run against the
+// run-tests.sh environment where RailsAPI runs under passenger, not
+// in the Rails test environment.
+func (s *railsRestartSuite) TestConfigReload(c *check.C) {
+       hc := http.Client{Transport: &http.Transport{
+               TLSClientConfig: &tls.Config{InsecureSkipVerify: true},
+       }}
+
+       confdata, err := os.ReadFile(os.Getenv("ARVADOS_CONFIG"))
+       c.Assert(err, check.IsNil)
+       oldhash := fmt.Sprintf("%x", sha256.Sum256(confdata))
+       c.Logf("oldhash %s", oldhash)
+
+       ldr := config.NewLoader(&bytes.Buffer{}, ctxlog.TestLogger(c))
+       cfg, err := ldr.Load()
+       c.Assert(err, check.IsNil)
+       cc, err := cfg.GetCluster("")
+       c.Assert(err, check.IsNil)
+       var metricsURL string
+       for u := range cc.Services.RailsAPI.InternalURLs {
+               u := url.URL(u)
+               mu, err := u.Parse("/metrics")
+               c.Assert(err, check.IsNil)
+               metricsURL = mu.String()
+       }
+
+       req, err := http.NewRequest(http.MethodGet, metricsURL, nil)
+       c.Assert(err, check.IsNil)
+       req.Header.Set("Authorization", "Bearer "+arvadostest.ManagementToken)
+
+       resp, err := hc.Do(req)
+       c.Assert(err, check.IsNil)
+       c.Check(resp.StatusCode, check.Equals, http.StatusOK)
+       body, err := ioutil.ReadAll(resp.Body)
+       c.Assert(err, check.IsNil)
+       c.Check(string(body), check.Matches, `(?ms).*`+oldhash+`.*`)
+
+       f, err := os.OpenFile(os.Getenv("ARVADOS_CONFIG"), os.O_WRONLY|os.O_APPEND, 0)
+       c.Assert(err, check.IsNil)
+       _, err = f.Write([]byte{'\n'})
+       c.Assert(err, check.IsNil)
+       err = f.Close()
+       c.Assert(err, check.IsNil)
+       newhash := fmt.Sprintf("%x", sha256.Sum256(append(confdata, '\n')))
+       c.Logf("newhash %s", newhash)
+
+       // Wait for RailsAPI's 1 Hz reload_config thread to poll and
+       // hit restart.txt
+       for deadline := time.Now().Add(10 * time.Second); time.Now().Before(deadline); time.Sleep(time.Second) {
+               resp, err = hc.Do(req)
+               c.Assert(err, check.IsNil)
+               c.Check(resp.StatusCode, check.Equals, http.StatusOK)
+               body, err = ioutil.ReadAll(resp.Body)
+               c.Assert(err, check.IsNil)
+               if strings.Contains(string(body), newhash) {
+                       break
+               }
+       }
+       c.Check(string(body), check.Matches, `(?ms).*`+newhash+`.*`)
+}
index 43357998d8b6c79cbb065f82e4ce92f71c080d54..679cbede13bc8cf141a34ec40373a458c5195451 100644 (file)
@@ -128,6 +128,17 @@ func (c *command) RunCommand(prog string, args []string, stdin io.Reader, stdout
        ctx = context.WithValue(ctx, contextKeyURL{}, listenURL)
 
        reg := prometheus.NewRegistry()
+       loader.RegisterMetrics(reg)
+
+       // arvados_version_running{version="1.2.3~4"} 1.0
+       mVersion := prometheus.NewGaugeVec(prometheus.GaugeOpts{
+               Namespace: "arvados",
+               Name:      "version_running",
+               Help:      "Indicated version is running.",
+       }, []string{"version"})
+       mVersion.WithLabelValues(cmd.Version.String()).Set(1)
+       reg.MustRegister(mVersion)
+
        handler := c.newHandler(ctx, cluster, cluster.SystemRootToken, reg)
        if err = handler.CheckHealth(); err != nil {
                return 1
index 1295350a4dd066d8960566cf854ebb69117d6839..f0adcda5f1c0ba2710ea0bd62b643453f46acfca 100644 (file)
@@ -10,6 +10,7 @@ import (
        "fmt"
        "net/url"
        "os"
+       "time"
 
        "git.arvados.org/arvados.git/sdk/go/config"
 )
@@ -24,6 +25,8 @@ var DefaultConfigFile = func() string {
 type Config struct {
        Clusters         map[string]Cluster
        AutoReloadConfig bool
+       SourceTimestamp  time.Time
+       SourceSHA256     string
 }
 
 // GetConfig returns the current system config, loading it from
index 23d7e8d431b7ed1160f3fcf8796e5b755d1dd078..5e010d88bc1bd32159ebfaf928392b948357cfd4 100644 (file)
@@ -5,22 +5,36 @@
 package health
 
 import (
+       "bufio"
+       "bytes"
        "context"
        "crypto/tls"
        "encoding/json"
+       "errors"
+       "flag"
        "fmt"
+       "io"
+       "net"
        "net/http"
        "net/url"
+       "regexp"
+       "strconv"
+       "strings"
        "sync"
        "time"
 
+       "git.arvados.org/arvados.git/lib/cmd"
+       "git.arvados.org/arvados.git/lib/config"
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/auth"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "github.com/ghodss/yaml"
+       "github.com/sirupsen/logrus"
 )
 
 const defaultTimeout = arvados.Duration(2 * time.Second)
 
-// Aggregator implements http.Handler. It handles "GET /_health/all"
+// Aggregator implements service.Handler. It handles "GET /_health/all"
 // by checking the health of all configured services on the cluster
 // and responding 200 if everything is healthy.
 type Aggregator struct {
@@ -99,6 +113,8 @@ type ClusterHealthResponse struct {
        // "service S is needed, but isn't configured to run
        // anywhere."
        Services map[arvados.ServiceName]ServiceHealth `json:"services"`
+
+       Errors []string `json:"errors"`
 }
 
 type CheckResult struct {
@@ -108,10 +124,16 @@ type CheckResult struct {
        HTTPStatusText string                 `json:",omitempty"`
        Response       map[string]interface{} `json:"response"`
        ResponseTime   json.Number            `json:"responseTime"`
+       Metrics        Metrics                `json:"-"`
+}
+
+type Metrics struct {
+       ConfigSourceTimestamp time.Time
+       ConfigSourceSHA256    string
 }
 
 type ServiceHealth struct {
-       Health string `json:"health"`
+       Health string `json:"health"` // "OK", "ERROR", or "SKIP"
        N      int    `json:"n"`
 }
 
@@ -129,7 +151,7 @@ func (agg *Aggregator) ClusterHealth() ClusterHealthResponse {
                // Ensure svc is listed in resp.Services.
                mtx.Lock()
                if _, ok := resp.Services[svcName]; !ok {
-                       resp.Services[svcName] = ServiceHealth{Health: "ERROR"}
+                       resp.Services[svcName] = ServiceHealth{Health: "MISSING"}
                }
                mtx.Unlock()
 
@@ -153,18 +175,30 @@ func (agg *Aggregator) ClusterHealth() ClusterHealthResponse {
                                        }
                                } else {
                                        result = agg.ping(pingURL)
+                                       if result.Health != "SKIP" {
+                                               m, err := agg.metrics(pingURL)
+                                               if err != nil && result.Error == "" {
+                                                       result.Error = "metrics: " + err.Error()
+                                               }
+                                               result.Metrics = m
+                                       }
                                }
 
                                mtx.Lock()
                                defer mtx.Unlock()
                                resp.Checks[fmt.Sprintf("%s+%s", svcName, pingURL)] = result
-                               if result.Health == "OK" {
+                               if result.Health == "OK" || result.Health == "SKIP" {
                                        h := resp.Services[svcName]
                                        h.N++
-                                       h.Health = "OK"
+                                       if result.Health == "OK" || h.N == 1 {
+                                               // "" => "SKIP" or "OK"
+                                               // "SKIP" => "OK"
+                                               h.Health = result.Health
+                                       }
                                        resp.Services[svcName] = h
                                } else {
                                        resp.Health = "ERROR"
+                                       resp.Errors = append(resp.Errors, fmt.Sprintf("%s: %s: %s", svcName, result.Health, result.Error))
                                }
                        }(svcName, addr)
                }
@@ -173,12 +207,44 @@ func (agg *Aggregator) ClusterHealth() ClusterHealthResponse {
 
        // Report ERROR if a needed service didn't fail any checks
        // merely because it isn't configured to run anywhere.
-       for _, sh := range resp.Services {
-               if sh.Health != "OK" {
-                       resp.Health = "ERROR"
-                       break
+       for svcName, sh := range resp.Services {
+               switch svcName {
+               case arvados.ServiceNameDispatchCloud,
+                       arvados.ServiceNameDispatchLSF:
+                       // ok to not run any given dispatcher
+               case arvados.ServiceNameHealth,
+                       arvados.ServiceNameWorkbench1,
+                       arvados.ServiceNameWorkbench2:
+                       // typically doesn't have InternalURLs in config
+               default:
+                       if sh.Health != "OK" && sh.Health != "SKIP" {
+                               resp.Health = "ERROR"
+                               resp.Errors = append(resp.Errors, fmt.Sprintf("%s: %s: no InternalURLs configured", svcName, sh.Health))
+                               continue
+                       }
+               }
+       }
+
+       var newest Metrics
+       for _, result := range resp.Checks {
+               if result.Metrics.ConfigSourceTimestamp.After(newest.ConfigSourceTimestamp) {
+                       newest = result.Metrics
                }
        }
+       var mismatches []string
+       for target, result := range resp.Checks {
+               if hash := result.Metrics.ConfigSourceSHA256; hash != "" && hash != newest.ConfigSourceSHA256 {
+                       mismatches = append(mismatches, target)
+               }
+       }
+       for _, target := range mismatches {
+               msg := fmt.Sprintf("outdated config: %s: config file (sha256 %s) does not match latest version with timestamp %s",
+                       strings.TrimSuffix(target, "/_health/ping"),
+                       resp.Checks[target].Metrics.ConfigSourceSHA256,
+                       newest.ConfigSourceTimestamp.Format(time.RFC3339))
+               resp.Errors = append(resp.Errors, msg)
+               resp.Health = "ERROR"
+       }
        return resp
 }
 
@@ -194,7 +260,9 @@ func (agg *Aggregator) ping(target *url.URL) (result CheckResult) {
        }()
        result.Health = "ERROR"
 
-       req, err := http.NewRequest("GET", target.String(), nil)
+       ctx, cancel := context.WithTimeout(context.Background(), time.Duration(agg.timeout))
+       defer cancel()
+       req, err := http.NewRequestWithContext(ctx, "GET", target.String(), nil)
        if err != nil {
                result.Error = err.Error()
                return
@@ -204,10 +272,17 @@ func (agg *Aggregator) ping(target *url.URL) (result CheckResult) {
        // Avoid workbench1's redirect-http-to-https feature
        req.Header.Set("X-Forwarded-Proto", "https")
 
-       ctx, cancel := context.WithTimeout(req.Context(), time.Duration(agg.timeout))
-       defer cancel()
-       req = req.WithContext(ctx)
        resp, err := agg.httpClient.Do(req)
+       if urlerr, ok := err.(*url.Error); ok {
+               if neterr, ok := urlerr.Err.(*net.OpError); ok && isLocalHost(target.Hostname()) {
+                       result = CheckResult{
+                               Health: "SKIP",
+                               Error:  neterr.Error(),
+                       }
+                       err = nil
+                       return
+               }
+       }
        if err != nil {
                result.Error = err.Error()
                return
@@ -232,6 +307,56 @@ func (agg *Aggregator) ping(target *url.URL) (result CheckResult) {
        return
 }
 
+var reMetric = regexp.MustCompile(`([a-z_]+){sha256="([0-9a-f]+)"} (\d[\d\.e\+]+)`)
+
+func (agg *Aggregator) metrics(pingURL *url.URL) (result Metrics, err error) {
+       metricsURL, err := pingURL.Parse("/metrics")
+       if err != nil {
+               return
+       }
+       ctx, cancel := context.WithTimeout(context.Background(), time.Duration(agg.timeout))
+       defer cancel()
+       req, err := http.NewRequestWithContext(ctx, "GET", metricsURL.String(), nil)
+       if err != nil {
+               return
+       }
+       req.Header.Set("Authorization", "Bearer "+agg.Cluster.ManagementToken)
+
+       // Avoid workbench1's redirect-http-to-https feature
+       req.Header.Set("X-Forwarded-Proto", "https")
+
+       resp, err := agg.httpClient.Do(req)
+       if err != nil {
+               return
+       } else if resp.StatusCode != http.StatusOK {
+               err = fmt.Errorf("%s: HTTP %d %s", metricsURL.String(), resp.StatusCode, resp.Status)
+               return
+       }
+
+       scanner := bufio.NewScanner(resp.Body)
+       for scanner.Scan() {
+               m := reMetric.FindSubmatch(scanner.Bytes())
+               if len(m) != 4 || string(m[1]) != "arvados_config_source_timestamp_seconds" {
+                       continue
+               }
+               result.ConfigSourceSHA256 = string(m[2])
+               unixtime, _ := strconv.ParseFloat(string(m[3]), 64)
+               result.ConfigSourceTimestamp = time.UnixMicro(int64(unixtime * 1e6))
+       }
+       if err = scanner.Err(); err != nil {
+               err = fmt.Errorf("error parsing response from %s: %w", metricsURL.String(), err)
+               return
+       }
+       return
+}
+
+// Test whether host is an easily recognizable loopback address:
+// 0.0.0.0, 127.x.x.x, ::1, or localhost.
+func isLocalHost(host string) bool {
+       ip := net.ParseIP(host)
+       return ip.IsLoopback() || bytes.Equal(ip.To4(), []byte{0, 0, 0, 0}) || strings.EqualFold(host, "localhost")
+}
+
 func (agg *Aggregator) checkAuth(req *http.Request) bool {
        creds := auth.CredentialsFromRequest(req)
        for _, token := range creds.Tokens {
@@ -241,3 +366,72 @@ func (agg *Aggregator) checkAuth(req *http.Request) bool {
        }
        return false
 }
+
+var errSilent = errors.New("")
+
+var CheckCommand cmd.Handler = checkCommand{}
+
+type checkCommand struct{}
+
+func (ccmd checkCommand) RunCommand(prog string, args []string, stdin io.Reader, stdout, stderr io.Writer) int {
+       logger := ctxlog.New(stderr, "json", "info")
+       ctx := ctxlog.Context(context.Background(), logger)
+       err := ccmd.run(ctx, prog, args, stdin, stdout, stderr)
+       if err != nil {
+               if err != errSilent {
+                       fmt.Fprintln(stdout, err.Error())
+               }
+               return 1
+       }
+       return 0
+}
+
+func (ccmd checkCommand) run(ctx context.Context, prog string, args []string, stdin io.Reader, stdout, stderr io.Writer) error {
+       flags := flag.NewFlagSet("", flag.ContinueOnError)
+       flags.SetOutput(stderr)
+       loader := config.NewLoader(stdin, ctxlog.New(stderr, "text", "info"))
+       loader.SetupFlags(flags)
+       versionFlag := flags.Bool("version", false, "Write version information to stdout and exit 0")
+       timeout := flags.Duration("timeout", defaultTimeout.Duration(), "Maximum time to wait for health responses")
+       outputYAML := flags.Bool("yaml", false, "Output full health report in YAML format (default mode shows errors as plain text, is silent on success)")
+       if ok, _ := cmd.ParseFlags(flags, prog, args, "", stderr); !ok {
+               // cmd.ParseFlags already reported the error
+               return errSilent
+       } else if *versionFlag {
+               cmd.Version.RunCommand(prog, args, stdin, stdout, stderr)
+               return nil
+       }
+       cfg, err := loader.Load()
+       if err != nil {
+               return err
+       }
+       cluster, err := cfg.GetCluster("")
+       if err != nil {
+               return err
+       }
+       logger := ctxlog.New(stderr, cluster.SystemLogs.Format, cluster.SystemLogs.LogLevel).WithFields(logrus.Fields{
+               "ClusterID": cluster.ClusterID,
+       })
+       ctx = ctxlog.Context(ctx, logger)
+       agg := Aggregator{Cluster: cluster, timeout: arvados.Duration(*timeout)}
+       resp := agg.ClusterHealth()
+       if *outputYAML {
+               y, err := yaml.Marshal(resp)
+               if err != nil {
+                       return err
+               }
+               stdout.Write(y)
+               if resp.Health != "OK" {
+                       return errSilent
+               }
+               return nil
+       }
+       if resp.Health != "OK" {
+               for _, msg := range resp.Errors {
+                       fmt.Fprintln(stdout, msg)
+               }
+               fmt.Fprintln(stderr, "health check failed")
+               return errSilent
+       }
+       return nil
+}
index f8507ef4f5b53aa568652f3253d670548248b54d..f8f7ff9f1b7e189b83423d017be2a48fd763cf28 100644 (file)
@@ -5,14 +5,22 @@
 package health
 
 import (
+       "bytes"
+       "crypto/sha256"
        "encoding/json"
+       "fmt"
+       "io/ioutil"
        "net/http"
        "net/http/httptest"
+       "regexp"
        "strings"
        "time"
 
+       "git.arvados.org/arvados.git/lib/config"
        "git.arvados.org/arvados.git/sdk/go/arvados"
        "git.arvados.org/arvados.git/sdk/go/arvadostest"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
+       "github.com/ghodss/yaml"
        "gopkg.in/check.v1"
 )
 
@@ -30,9 +38,17 @@ func (s *AggregatorSuite) TestInterface(c *check.C) {
 }
 
 func (s *AggregatorSuite) SetUpTest(c *check.C) {
-       s.handler = &Aggregator{Cluster: &arvados.Cluster{
-               ManagementToken: arvadostest.ManagementToken,
-       }}
+       ldr := config.NewLoader(bytes.NewBufferString(`Clusters: {zzzzz: {}}`), ctxlog.TestLogger(c))
+       ldr.Path = "-"
+       cfg, err := ldr.Load()
+       c.Assert(err, check.IsNil)
+       cluster, err := cfg.GetCluster("")
+       c.Assert(err, check.IsNil)
+       cluster.ManagementToken = arvadostest.ManagementToken
+       cluster.SystemRootToken = arvadostest.SystemRootToken
+       cluster.Collections.BlobSigningKey = arvadostest.BlobSigningKey
+       cluster.Volumes["z"] = arvados.Volume{StorageClasses: map[string]bool{"default": true}}
+       s.handler = &Aggregator{Cluster: cluster}
        s.req = httptest.NewRequest("GET", "/_health/all", nil)
        s.req.Header.Set("Authorization", "Bearer "+arvadostest.ManagementToken)
        s.resp = httptest.NewRecorder()
@@ -107,6 +123,103 @@ func (s *AggregatorSuite) TestHealthyAndUnhealthy(c *check.C) {
        c.Logf("%#v", ep)
 }
 
+// If an InternalURL host is 0.0.0.0, localhost, 127/8, or ::1 and
+// nothing is listening there, don't fail the health check -- instead,
+// assume the relevant component just isn't installed/enabled on this
+// node, but does work when contacted through ExternalURL.
+func (s *AggregatorSuite) TestUnreachableLoopbackPort(c *check.C) {
+       srvH, listenH := s.stubServer(&healthyHandler{})
+       defer srvH.Close()
+       s.setAllServiceURLs(listenH)
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.Keepproxy, "http://localhost:9/")
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.Workbench1, "http://0.0.0.0:9/")
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.Keepbalance, "http://127.0.0.127:9/")
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.WebDAV, "http://[::1]:9/")
+       s.handler.ServeHTTP(s.resp, s.req)
+       s.checkOK(c)
+
+       // If a non-loopback address is unreachable, that's still a
+       // fail.
+       s.resp = httptest.NewRecorder()
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.WebDAV, "http://172.31.255.254:9/")
+       s.handler.ServeHTTP(s.resp, s.req)
+       s.checkUnhealthy(c)
+}
+
+func (s *AggregatorSuite) TestIsLocalHost(c *check.C) {
+       c.Check(isLocalHost("Localhost"), check.Equals, true)
+       c.Check(isLocalHost("localhost"), check.Equals, true)
+       c.Check(isLocalHost("127.0.0.1"), check.Equals, true)
+       c.Check(isLocalHost("127.0.0.127"), check.Equals, true)
+       c.Check(isLocalHost("127.1.2.7"), check.Equals, true)
+       c.Check(isLocalHost("0.0.0.0"), check.Equals, true)
+       c.Check(isLocalHost("::1"), check.Equals, true)
+       c.Check(isLocalHost("1.2.3.4"), check.Equals, false)
+       c.Check(isLocalHost("1::1"), check.Equals, false)
+       c.Check(isLocalHost("example.com"), check.Equals, false)
+       c.Check(isLocalHost("127.0.0"), check.Equals, false)
+       c.Check(isLocalHost(""), check.Equals, false)
+}
+
+func (s *AggregatorSuite) TestConfigMismatch(c *check.C) {
+       // time1/hash1: current config
+       time1 := time.Now().Add(time.Second - time.Minute - time.Hour)
+       hash1 := fmt.Sprintf("%x", sha256.Sum256([]byte(`Clusters: {zzzzz: {SystemRootToken: xyzzy}}`)))
+       // time2/hash2: old config
+       time2 := time1.Add(-time.Hour)
+       hash2 := fmt.Sprintf("%x", sha256.Sum256([]byte(`Clusters: {zzzzz: {SystemRootToken: old-token}}`)))
+
+       // srv1: current file
+       handler1 := healthyHandler{configHash: hash1, configTime: time1}
+       srv1, listen1 := s.stubServer(&handler1)
+       defer srv1.Close()
+       // srv2: old file, current content
+       handler2 := healthyHandler{configHash: hash1, configTime: time2}
+       srv2, listen2 := s.stubServer(&handler2)
+       defer srv2.Close()
+       // srv3: old file, old content
+       handler3 := healthyHandler{configHash: hash2, configTime: time2}
+       srv3, listen3 := s.stubServer(&handler3)
+       defer srv3.Close()
+       // srv4: no metrics handler
+       handler4 := healthyHandler{}
+       srv4, listen4 := s.stubServer(&handler4)
+       defer srv4.Close()
+
+       s.setAllServiceURLs(listen1)
+
+       // listen2 => old timestamp, same content => no problem
+       s.resp = httptest.NewRecorder()
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.DispatchCloud,
+               "http://localhost"+listen2+"/")
+       s.handler.ServeHTTP(s.resp, s.req)
+       resp := s.checkOK(c)
+
+       // listen4 => no metrics on some services => no problem
+       s.resp = httptest.NewRecorder()
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.WebDAV,
+               "http://localhost"+listen4+"/")
+       s.handler.ServeHTTP(s.resp, s.req)
+       resp = s.checkOK(c)
+
+       // listen3 => old timestamp, old content => report discrepancy
+       s.resp = httptest.NewRecorder()
+       arvadostest.SetServiceURL(&s.handler.Cluster.Services.Keepstore,
+               "http://localhost"+listen1+"/",
+               "http://localhost"+listen3+"/")
+       s.handler.ServeHTTP(s.resp, s.req)
+       resp = s.checkUnhealthy(c)
+       if c.Check(len(resp.Errors) > 0, check.Equals, true) {
+               c.Check(resp.Errors[0], check.Matches, `outdated config: \Qkeepstore+http://localhost`+listen3+`\E: config file \(sha256 .*\) does not match latest version with timestamp .*`)
+       }
+
+       // no services report config time (migrating to current version) => no problem
+       s.resp = httptest.NewRecorder()
+       s.setAllServiceURLs(listen4)
+       s.handler.ServeHTTP(s.resp, s.req)
+       s.checkOK(c)
+}
+
 func (s *AggregatorSuite) TestPingTimeout(c *check.C) {
        s.handler.timeout = arvados.Duration(100 * time.Millisecond)
        srv, listen := s.stubServer(&slowHandler{})
@@ -122,6 +235,32 @@ func (s *AggregatorSuite) TestPingTimeout(c *check.C) {
        c.Check(rt > 0.005, check.Equals, true)
 }
 
+func (s *AggregatorSuite) TestCheckCommand(c *check.C) {
+       srv, listen := s.stubServer(&healthyHandler{})
+       defer srv.Close()
+       s.setAllServiceURLs(listen)
+       tmpdir := c.MkDir()
+       confdata, err := yaml.Marshal(arvados.Config{Clusters: map[string]arvados.Cluster{s.handler.Cluster.ClusterID: *s.handler.Cluster}})
+       c.Assert(err, check.IsNil)
+       confdata = regexp.MustCompile(`Source(Timestamp|SHA256): [^\n]+\n`).ReplaceAll(confdata, []byte{})
+       err = ioutil.WriteFile(tmpdir+"/config.yml", confdata, 0777)
+       c.Assert(err, check.IsNil)
+
+       var stdout, stderr bytes.Buffer
+
+       exitcode := CheckCommand.RunCommand("check", []string{"-config=" + tmpdir + "/config.yml"}, &bytes.Buffer{}, &stdout, &stderr)
+       c.Check(exitcode, check.Equals, 0)
+       c.Check(stderr.String(), check.Equals, "")
+       c.Check(stdout.String(), check.Equals, "")
+
+       stdout.Reset()
+       stderr.Reset()
+       exitcode = CheckCommand.RunCommand("check", []string{"-config=" + tmpdir + "/config.yml", "-yaml"}, &bytes.Buffer{}, &stdout, &stderr)
+       c.Check(exitcode, check.Equals, 0)
+       c.Check(stderr.String(), check.Equals, "")
+       c.Check(stdout.String(), check.Matches, `(?ms).*(\n|^)health: OK\n.*`)
+}
+
 func (s *AggregatorSuite) checkError(c *check.C) {
        c.Check(s.resp.Code, check.Not(check.Equals), http.StatusOK)
        var resp ClusterHealthResponse
@@ -179,11 +318,37 @@ func (*unhealthyHandler) ServeHTTP(resp http.ResponseWriter, req *http.Request)
        }
 }
 
-type healthyHandler struct{}
+type healthyHandler struct {
+       configHash string
+       configTime time.Time
+}
 
-func (*healthyHandler) ServeHTTP(resp http.ResponseWriter, req *http.Request) {
+func (h *healthyHandler) ServeHTTP(resp http.ResponseWriter, req *http.Request) {
+       authOK := req.Header.Get("Authorization") == "Bearer "+arvadostest.ManagementToken
        if req.URL.Path == "/_health/ping" {
+               if !authOK {
+                       http.Error(resp, "unauthorized", http.StatusUnauthorized)
+                       return
+               }
                resp.Write([]byte(`{"health":"OK"}`))
+       } else if req.URL.Path == "/metrics" {
+               if !authOK {
+                       http.Error(resp, "unauthorized", http.StatusUnauthorized)
+                       return
+               }
+               t := h.configTime
+               if t.IsZero() {
+                       t = time.Now()
+               }
+               fmt.Fprintf(resp, `# HELP arvados_config_load_timestamp_seconds Time when config file was loaded.
+# TYPE arvados_config_load_timestamp_seconds gauge
+arvados_config_load_timestamp_seconds{sha256="%s"} %g
+# HELP arvados_config_source_timestamp_seconds Timestamp of config file when it was loaded.
+# TYPE arvados_config_source_timestamp_seconds gauge
+arvados_config_source_timestamp_seconds{sha256="%s"} %g
+`,
+                       h.configHash, float64(time.Now().UnixNano())/1e9,
+                       h.configHash, float64(t.UnixNano())/1e9)
        } else {
                http.Error(resp, "not found", http.StatusNotFound)
        }
index 6f591b74a5a20f51b1d87e798c07eebf83606f3f..2c01b35aeac79b1642b18c7af7d166ef2cffdc3c 100644 (file)
@@ -331,6 +331,19 @@ def run(leave_running_atexit=False):
         os.makedirs(gitdir)
     subprocess.check_output(['tar', '-xC', gitdir, '-f', gittarball])
 
+    # Customizing the passenger config template is the only documented
+    # way to override the default passenger_stat_throttle_rate (10 s).
+    # In the testing environment, we want restart.txt to take effect
+    # immediately.
+    resdir = subprocess.check_output(['bundle', 'exec', 'passenger-config', 'about', 'resourcesdir']).decode().rstrip()
+    with open(resdir + '/templates/standalone/config.erb') as f:
+        template = f.read()
+    newtemplate = re.sub('http {', 'http {\n        passenger_stat_throttle_rate 0;', template)
+    if newtemplate == template:
+        raise "template edit failed"
+    with open('tmp/passenger-nginx.conf.erb', 'w') as f:
+        f.write(newtemplate)
+
     port = internal_port_from_config("RailsAPI")
     env = os.environ.copy()
     env['RAILS_ENV'] = 'test'
@@ -344,6 +357,10 @@ def run(leave_running_atexit=False):
     railsapi = subprocess.Popen(
         ['bundle', 'exec',
          'passenger', 'start', '-p{}'.format(port),
+         '--nginx-config-template', 'tmp/passenger-nginx.conf.erb',
+        '--no-friendly-error-pages',
+        '--disable-anonymous-telemetry',
+        '--disable-security-update-check',
          '--pid-file', pid_file,
          '--log-file', '/dev/stdout',
          '--ssl',
diff --git a/services/api/app/controllers/arvados/v1/healthcheck_controller.rb b/services/api/app/controllers/arvados/v1/healthcheck_controller.rb
deleted file mode 100644 (file)
index c562082..0000000
+++ /dev/null
@@ -1,36 +0,0 @@
-# Copyright (C) The Arvados Authors. All rights reserved.
-#
-# SPDX-License-Identifier: AGPL-3.0
-
-class Arvados::V1::HealthcheckController < ApplicationController
-  skip_before_action :catch_redirect_hint
-  skip_before_action :find_objects_for_index
-  skip_before_action :find_object_by_uuid
-  skip_before_action :load_filters_param
-  skip_before_action :load_limit_offset_order_params
-  skip_before_action :load_select_param
-  skip_before_action :load_read_auths
-  skip_before_action :load_where_param
-  skip_before_action :render_404_if_no_object
-  skip_before_action :require_auth_scope
-
-  before_action :check_auth_header
-
-  def check_auth_header
-    mgmt_token = Rails.configuration.ManagementToken
-    auth_header = request.headers['Authorization']
-
-    if mgmt_token == ""
-      send_json ({"errors" => "disabled"}), status: 404
-    elsif !auth_header
-      send_json ({"errors" => "authorization required"}), status: 401
-    elsif auth_header != 'Bearer '+mgmt_token
-      send_json ({"errors" => "authorization error"}), status: 403
-    end
-  end
-
-  def ping
-    resp = {"health" => "OK"}
-    send_json resp
-  end
-end
diff --git a/services/api/app/controllers/arvados/v1/management_controller.rb b/services/api/app/controllers/arvados/v1/management_controller.rb
new file mode 100644 (file)
index 0000000..8f37960
--- /dev/null
@@ -0,0 +1,55 @@
+# Copyright (C) The Arvados Authors. All rights reserved.
+#
+# SPDX-License-Identifier: AGPL-3.0
+
+class Arvados::V1::ManagementController < ApplicationController
+  skip_before_action :catch_redirect_hint
+  skip_before_action :find_objects_for_index
+  skip_before_action :find_object_by_uuid
+  skip_before_action :load_filters_param
+  skip_before_action :load_limit_offset_order_params
+  skip_before_action :load_select_param
+  skip_before_action :load_read_auths
+  skip_before_action :load_where_param
+  skip_before_action :render_404_if_no_object
+  skip_before_action :require_auth_scope
+
+  before_action :check_auth_header
+
+  def check_auth_header
+    mgmt_token = Rails.configuration.ManagementToken
+    auth_header = request.headers['Authorization']
+
+    if mgmt_token == ""
+      send_json ({"errors" => "disabled"}), status: 404
+    elsif !auth_header
+      send_json ({"errors" => "authorization required"}), status: 401
+    elsif auth_header != 'Bearer '+mgmt_token
+      send_json ({"errors" => "authorization error"}), status: 403
+    end
+  end
+
+  def metrics
+    render content_type: 'text/plain', plain: <<~EOF
+# HELP arvados_config_load_timestamp_seconds Time when config file was loaded.
+# TYPE arvados_config_load_timestamp_seconds gauge
+arvados_config_load_timestamp_seconds{sha256="#{Rails.configuration.SourceSHA256}"} #{Rails.configuration.LoadTimestamp.to_f}
+# HELP arvados_config_source_timestamp_seconds Timestamp of config file when it was loaded.
+# TYPE arvados_config_source_timestamp_seconds gauge
+arvados_config_source_timestamp_seconds{sha256="#{Rails.configuration.SourceSHA256}"} #{Rails.configuration.SourceTimestamp.to_f}
+# HELP arvados_version_running Indicated version is running.
+# TYPE arvados_version_running gauge
+arvados_version_running{version="#{AppVersion.package_version}"} 1
+EOF
+  end
+
+  def health
+    case params[:check]
+    when 'ping'
+      resp = {"health" => "OK"}
+      send_json resp
+    else
+      send_json ({"errors" => "not found"}), status: 404
+    end
+  end
+end
index 8a96c432a8df89873cccfaca7832dd9120ca9741..c0f7ee174fb65f8ef34d8502cc78d26e104f50ef 100644 (file)
@@ -30,6 +30,7 @@ end
 
 # Load the defaults, used by config:migrate and fallback loading
 # legacy application.yml
+load_time = Time.now.utc
 defaultYAML, stderr, status = Open3.capture3("arvados-server", "config-dump", "-config=-", "-skip-legacy", stdin_data: "Clusters: {xxxxx: {}}")
 if !status.success?
   puts stderr
@@ -39,6 +40,8 @@ confs = YAML.load(defaultYAML, deserialize_symbols: false)
 clusterID, clusterConfig = confs["Clusters"].first
 $arvados_config_defaults = clusterConfig
 $arvados_config_defaults["ClusterID"] = clusterID
+$arvados_config_defaults["SourceTimestamp"] = Time.rfc3339(confs["SourceTimestamp"])
+$arvados_config_defaults["SourceSHA256"] = confs["SourceSHA256"]
 
 if ENV["ARVADOS_CONFIG"] == "none"
   # Don't load config. This magic value is set by packaging scripts so
@@ -54,6 +57,8 @@ else
       clusterID, clusterConfig = confs["Clusters"].first
       $arvados_config_global = clusterConfig
       $arvados_config_global["ClusterID"] = clusterID
+      $arvados_config_global["SourceTimestamp"] = Time.rfc3339(confs["SourceTimestamp"])
+      $arvados_config_global["SourceSHA256"] = confs["SourceSHA256"]
     else
       # config-dump failed, assume we will be loading from legacy
       # application.yml, initialize with defaults.
@@ -64,6 +69,7 @@ end
 
 # Now make a copy
 $arvados_config = $arvados_config_global.deep_dup
+$arvados_config["LoadTimestamp"] = load_time
 
 def arrayToHash cfg, k, v
   val = {}
diff --git a/services/api/config/initializers/reload_config.rb b/services/api/config/initializers/reload_config.rb
new file mode 100644 (file)
index 0000000..1582855
--- /dev/null
@@ -0,0 +1,66 @@
+# Copyright (C) The Arvados Authors. All rights reserved.
+#
+# SPDX-License-Identifier: AGPL-3.0
+
+if !File.owned?(Rails.root.join('tmp'))
+  Rails.logger.debug("reload_config: not owner of #{Rails.root}/tmp, skipping")
+elsif ENV["ARVADOS_CONFIG"] == "none"
+  Rails.logger.debug("reload_config: no config in use, skipping")
+else
+  Thread.new do
+    lockfile = Rails.root.join('tmp', 'reload_config.lock')
+    File.open(lockfile, File::WRONLY|File::CREAT, 0600) do |f|
+      # Note we don't use LOCK_NB here. If we did, each time passenger
+      # kills the lock-holder process, we would be left with nobody
+      # checking for updates until passenger starts a new worker,
+      # which could be a long time.
+      Rails.logger.debug("reload_config: waiting for lock on #{lockfile}")
+      f.flock(File::LOCK_EX)
+
+      t_lastload = Rails.configuration.SourceTimestamp
+      hash_lastload = Rails.configuration.SourceSHA256
+      conffile = ENV['ARVADOS_CONFIG'] || "/etc/arvados/config.yml"
+      Rails.logger.info("reload_config: polling for updated mtime on #{conffile} with threshold #{t_lastload}")
+      while true
+        sleep 1
+        t = File.mtime(conffile)
+        # If the file is newer than 5s, re-read it even if the
+        # timestamp matches the previously loaded file. This enables
+        # us to detect changes even if the filesystem's timestamp
+        # precision cannot represent multiple updates per second.
+        if t.to_f != t_lastload.to_f || Time.now.to_f - t.to_f < 5
+          Open3.popen2("arvados-server", "config-dump", "-skip-legacy") do |stdin, stdout, status_thread|
+            confs = YAML.load(stdout, deserialize_symbols: false)
+            hash = confs["SourceSHA256"]
+          rescue => e
+            Rails.logger.info("reload_config: config file updated but could not be loaded: #{e}")
+            t_lastload = t
+            continue
+          end
+          if hash == hash_lastload
+            # If we reloaded a new or updated file, but the content is
+            # identical, keep polling instead of restarting.
+            t_lastload = t
+            continue
+          end
+
+          restartfile = Rails.root.join('tmp', 'restart.txt')
+          touchtime = Time.now
+          Rails.logger.info("reload_config: mtime on #{conffile} changed to #{t}, touching #{restartfile} to #{touchtime}")
+          begin
+            File.utime(touchtime, touchtime, restartfile)
+          rescue
+            # remove + re-create works even if the existing file is
+            # owned by root, provided the tempdir is writable.
+            File.unlink(restartfile) rescue nil
+            File.open(restartfile, 'w') {}
+          end
+          # Even if passenger doesn't notice that we hit restart.txt
+          # and kill our process, there's no point waiting around to
+          # hit it again.
+          break
+        end
+      end
+    end
+  end
+end
index 98f5788d6505d3525115f3b3a8e5622b8a059937..9c7bfc3a7ac8ad1f8cd1ef54dd0dfc2c01f1065b 100644 (file)
@@ -112,7 +112,8 @@ Rails.application.routes.draw do
 
   match '/static/login_failure', to: 'static#login_failure', as: :login_failure, via: [:get, :post]
 
-  match '/_health/ping', to: 'arvados/v1/healthcheck#ping', via: [:get]
+  match '/_health/:check', to: 'arvados/v1/management#health', via: [:get]
+  match '/metrics', to: 'arvados/v1/management#metrics', via: [:get]
 
   # Send unroutable requests to an arbitrary controller
   # (ends up at ApplicationController#render_not_found)
diff --git a/services/api/test/functional/arvados/v1/healthcheck_controller_test.rb b/services/api/test/functional/arvados/v1/healthcheck_controller_test.rb
deleted file mode 100644 (file)
index 76fdb04..0000000
+++ /dev/null
@@ -1,34 +0,0 @@
-# Copyright (C) The Arvados Authors. All rights reserved.
-#
-# SPDX-License-Identifier: AGPL-3.0
-
-require 'test_helper'
-
-class Arvados::V1::HealthcheckControllerTest < ActionController::TestCase
-  [
-    [false, nil, 404, 'disabled'],
-    [true, nil, 401, 'authorization required'],
-    [true, 'badformatwithnoBearer', 403, 'authorization error'],
-    [true, 'Bearer wrongtoken', 403, 'authorization error'],
-    [true, 'Bearer configuredmanagementtoken', 200, '{"health":"OK"}'],
-  ].each do |enabled, header, error_code, error_msg|
-    test "ping when #{if enabled then 'enabled' else 'disabled' end} with header '#{header}'" do
-      if enabled
-        Rails.configuration.ManagementToken = 'configuredmanagementtoken'
-      else
-        Rails.configuration.ManagementToken = ""
-      end
-
-      @request.headers['Authorization'] = header
-      get :ping
-      assert_response error_code
-
-      resp = JSON.parse(@response.body)
-      if error_code == 200
-        assert_equal(JSON.load('{"health":"OK"}'), resp)
-      else
-        assert_equal(error_msg, resp['errors'])
-      end
-    end
-  end
-end
diff --git a/services/api/test/functional/arvados/v1/management_controller_test.rb b/services/api/test/functional/arvados/v1/management_controller_test.rb
new file mode 100644 (file)
index 0000000..6d27bcc
--- /dev/null
@@ -0,0 +1,73 @@
+# Copyright (C) The Arvados Authors. All rights reserved.
+#
+# SPDX-License-Identifier: AGPL-3.0
+
+require 'test_helper'
+
+class Arvados::V1::ManagementControllerTest < ActionController::TestCase
+  [
+    [false, nil, 404, 'disabled'],
+    [true, nil, 401, 'authorization required'],
+    [true, 'badformatwithnoBearer', 403, 'authorization error'],
+    [true, 'Bearer wrongtoken', 403, 'authorization error'],
+    [true, 'Bearer configuredmanagementtoken', 200, '{"health":"OK"}'],
+  ].each do |enabled, header, error_code, error_msg|
+    test "_health/ping when #{if enabled then 'enabled' else 'disabled' end} with header '#{header}'" do
+      if enabled
+        Rails.configuration.ManagementToken = 'configuredmanagementtoken'
+      else
+        Rails.configuration.ManagementToken = ""
+      end
+
+      @request.headers['Authorization'] = header
+      get :health, params: {check: 'ping'}
+      assert_response error_code
+
+      resp = JSON.parse(@response.body)
+      if error_code == 200
+        assert_equal(JSON.load('{"health":"OK"}'), resp)
+      else
+        assert_equal(error_msg, resp['errors'])
+      end
+    end
+  end
+
+  test "metrics" do
+    mtime = File.mtime(ENV["ARVADOS_CONFIG"])
+    hash = Digest::SHA256.hexdigest(File.read(ENV["ARVADOS_CONFIG"]))
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    @request.headers['Authorization'] = "Bearer configuredmanagementtoken"
+    get :metrics
+    assert_response :success
+    assert_equal 'text/plain', @response.content_type
+
+    assert_match /\narvados_config_source_timestamp_seconds{sha256="#{hash}"} #{Regexp.escape mtime.utc.to_f.to_s}\n/, @response.body
+
+    # Expect mtime < loadtime < now
+    m = @response.body.match(/\narvados_config_load_timestamp_seconds{sha256="#{hash}"} (.*?)\n/)
+    assert_operator m[1].to_f, :>, mtime.utc.to_f
+    assert_operator m[1].to_f, :<, Time.now.utc.to_f
+
+    assert_match /\narvados_version_running{version="#{Regexp.escape AppVersion.package_version}"} 1\n/, @response.body
+  end
+
+  test "metrics disabled" do
+    Rails.configuration.ManagementToken = ""
+    @request.headers['Authorization'] = "Bearer configuredmanagementtoken"
+    get :metrics
+    assert_response 404
+  end
+
+  test "metrics bad token" do
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    @request.headers['Authorization'] = "Bearer asdf"
+    get :metrics
+    assert_response 403
+  end
+
+  test "metrics unauthorized" do
+    Rails.configuration.ManagementToken = "configuredmanagementtoken"
+    get :metrics
+    assert_response 401
+  end
+end
index a2a1545cee93d7ffcdd5a63073881abc960caa90..8e9929117b2dd32e4e10777f1f7f7bb249ba6649 100644 (file)
@@ -24,7 +24,7 @@ class ErrorsTest < ActionDispatch::IntegrationTest
       # Generally, new routes should appear under /arvados/v1/. If
       # they appear elsewhere, that might have been caused by default
       # rails generator behavior that we don't want.
-      assert_match(/^\/(|\*a|arvados\/v1\/.*|auth\/.*|login|logout|database\/reset|discovery\/.*|static\/.*|sys\/trash_sweep|themes\/.*|assets|_health\/.*)(\(\.:format\))?$/,
+      assert_match(/^\/(|\*a|arvados\/v1\/.*|auth\/.*|login|logout|database\/reset|discovery\/.*|static\/.*|sys\/trash_sweep|themes\/.*|assets|_health\/.*|metrics)(\(\.:format\))?$/,
                    route.path.spec.to_s,
                    "Unexpected new route: #{route.path.spec}")
     end