You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2020/12/12 05:17:25 UTC

[kudu] branch master updated: [server] add 'uptime' metric for server

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new b003609  [server] add 'uptime' metric for server
b003609 is described below

commit b003609acc189e5443fbe4869ce7ab539b0cb0bc
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Dec 10 18:54:21 2020 -0800

    [server] add 'uptime' metric for server
    
    This patch adds a new 'uptime' metric for a Kudu server.  The metric's
    value is reported as the duration of the time interval passed from the
    start of the server.  The interval duration is reported in microseconds
    as many other time-related metrics.
    
    Initially I thought to reuse already existing field that stores server's
    start time as wall clock time in seconds, but such approach would not
    produce a reliable result given that the wall clock time can be set any
    time by a super user, hence a wall-clock-based delta could not reliably
    provide the actual uptime of a server.
    
    I also did a minor refactoring on BaseServer::Init() and
    BaseServer::Start() methods, moving some steps from Init() to Start()
    which are belonging more to Start() rather than Init().
    
    The motivation for this patch is the realization that sometimes it's not
    easy to interpret metric values reported for a server without knowing
    its uptime.
    
    I didn't add any tests for the newly introduced metric, but I manually
    verified that both kudu-master and kudu-tserver processes report uptime
    among their metrics at the "/metrics" HTTP endpoint.
    
    Change-Id: Idb362a71c625c05a2185b95ec24d3f371dcb4155
    Reviewed-on: http://gerrit.cloudera.org:8080/16858
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Kudu Jenkins
---
 src/kudu/common/wire_protocol.proto             |  2 +-
 src/kudu/integration-tests/registration-test.cc |  9 +++---
 src/kudu/master/master.cc                       |  2 +-
 src/kudu/server/server_base.cc                  | 43 ++++++++++++++++---------
 src/kudu/server/server_base.h                   | 27 +++++++++++++---
 src/kudu/tserver/heartbeater.cc                 |  2 +-
 6 files changed, 58 insertions(+), 27 deletions(-)

diff --git a/src/kudu/common/wire_protocol.proto b/src/kudu/common/wire_protocol.proto
index f65ddea..cce7a87 100644
--- a/src/kudu/common/wire_protocol.proto
+++ b/src/kudu/common/wire_protocol.proto
@@ -94,7 +94,7 @@ message ServerRegistrationPB {
   // 'http_addresses' field.
   optional bool https_enabled = 4;
 
-  // Seconds since the epoch.
+  // The wall clock time when the server started as seconds since the Epoch.
   optional int64 start_time = 5;
 
   // The path of a UNIX domain socket where the server is listening.
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index bf29446..fc87982 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -279,16 +279,17 @@ TEST_F(RegistrationTest, TestMasterSoftwareVersion) {
   }
 }
 
-TEST_F(RegistrationTest, TestServerStartTime) {
+TEST_F(RegistrationTest, TestServerStartWallTime) {
   ServerRegistrationPB reg;
   cluster_->mini_master()->master()->GetMasterRegistration(&reg);
   ASSERT_LE(setup_time_, reg.start_time());
   ASSERT_LE(reg.start_time(), WallTime_Now());
 
   for (int i = 0; i < cluster_->num_tablet_servers(); ++i) {
-    auto start_time = cluster_->mini_tablet_server(i)->server()->start_time();
-    ASSERT_LE(setup_time_, start_time);
-    ASSERT_LE(start_time, WallTime_Now());
+    auto start_walltime =
+        cluster_->mini_tablet_server(i)->server()->start_walltime();
+    ASSERT_LE(setup_time_, start_walltime);
+    ASSERT_LE(start_walltime, WallTime_Now());
   }
 }
 
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index f44fbd0..7446b77 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -389,7 +389,7 @@ Status Master::InitMasterRegistration() {
     reg.set_https_enabled(web_server()->IsSecure());
   }
   reg.set_software_version(VersionInfo::GetVersionInfo());
-  reg.set_start_time(start_time_);
+  reg.set_start_time(start_walltime_);
 
   registration_.Swap(&reg);
   registration_initialized_.store(true);
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 645d46f..7e7a546 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -220,6 +220,11 @@ DECLARE_string(keytab_file);
 DECLARE_string(principal);
 
 METRIC_DECLARE_gauge_size(merged_entities_count_of_server);
+METRIC_DEFINE_gauge_int64(server, uptime,
+                          "Server Uptime",
+                          kudu::MetricUnit::kMicroseconds,
+                          "Time interval since the server has started",
+                          kudu::MetricLevel::kInfo);
 
 using kudu::security::RpcAuthentication;
 using kudu::security::RpcEncryption;
@@ -392,6 +397,7 @@ int64_t GetFileCacheCapacity(Env* env) {
 ServerBase::ServerBase(string name, const ServerBaseOptions& options,
                        const string& metric_namespace)
     : name_(std::move(name)),
+      start_time_(MonoTime::Min()),
       minidump_handler_(new MinidumpExceptionHandler()),
       mem_tracker_(CreateMemTrackerForServer()),
       metric_registry_(new MetricRegistry()),
@@ -504,7 +510,6 @@ Status ServerBase::Init() {
 
   // Create the Messenger.
   rpc::MessengerBuilder builder(name_);
-
   builder.set_num_reactors(FLAGS_num_reactor_threads)
          .set_min_negotiation_threads(FLAGS_min_negotiation_threads)
          .set_max_negotiation_threads(FLAGS_max_negotiation_threads)
@@ -547,21 +552,10 @@ Status ServerBase::Init() {
                           "for RPC server");
   }
 
-  RETURN_NOT_OK(rpc_server_->Bind());
-
-  RETURN_NOT_OK_PREPEND(StartMetricsLogging(), "Could not enable metrics logging");
-
-  result_tracker_->StartGCThread();
-  RETURN_NOT_OK(StartExcessLogFileDeleterThread());
-#ifdef TCMALLOC_ENABLED
-  RETURN_NOT_OK(StartTcmallocMemoryGcThread());
-#endif
-
-  return Status::OK();
+  return rpc_server_->Bind();
 }
 
 Status ServerBase::InitAcls() {
-
   string service_user;
   boost::optional<string> keytab_user = security::GetLoggedInUsernameFromKeytab();
   if (keytab_user) {
@@ -789,6 +783,27 @@ Status ServerBase::Start() {
 
   RETURN_NOT_OK(RegisterService(
       unique_ptr<rpc::ServiceIf>(new GenericServiceImpl(this))));
+
+  // Webserver shows the wall clock time when server was started and exposes
+  // the server's uptime along with all other metrics, and the metrics logger
+  // accesses the uptime metric as well, so it's necessary to have corresponding
+  // information ready at this point.
+  start_time_ = MonoTime::Now();
+  start_walltime_ = static_cast<int64_t>(WallTime_Now());
+
+  METRIC_uptime.InstantiateFunctionGauge(
+      metric_entity_,
+      [this]() {return (MonoTime::Now() - this->start_time()).ToMicroseconds();})->
+          AutoDetachToLastValue(&metric_detacher_);
+
+  RETURN_NOT_OK_PREPEND(StartMetricsLogging(), "Could not enable metrics logging");
+
+  result_tracker_->StartGCThread();
+  RETURN_NOT_OK(StartExcessLogFileDeleterThread());
+#ifdef TCMALLOC_ENABLED
+  RETURN_NOT_OK(StartTcmallocMemoryGcThread());
+#endif
+
   RETURN_NOT_OK(rpc_server_->Start());
 
   if (web_server_) {
@@ -805,8 +820,6 @@ Status ServerBase::Start() {
                           "Failed to dump server info to " + options_.dump_info_path);
   }
 
-  start_time_ = WallTime_Now();
-
   return Status::OK();
 }
 
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index feceb7f..1fd2082 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -20,12 +20,16 @@
 #include <memory>
 #include <string>
 
+#include <glog/logging.h>
+
 #include "kudu/gutil/macros.h"
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/security/simple_acl.h"
 #include "kudu/server/server_base_options.h"
 #include "kudu/util/countdown_latch.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
 #include "kudu/util/status.h"
 
 namespace kudu {
@@ -34,8 +38,6 @@ class DnsResolver;
 class FileCache;
 class FsManager;
 class MemTracker;
-class MetricEntity;
-class MetricRegistry;
 class MinidumpExceptionHandler;
 class NodeInstancePB;
 class RpcServer;
@@ -110,7 +112,12 @@ class ServerBase {
   // Return a PB describing the status of the server (version info, bound ports, etc)
   Status GetStatusPB(ServerStatusPB* status) const;
 
-  int64_t start_time() const {
+  int64_t start_walltime() const {
+    return start_walltime_;
+  }
+
+  const MonoTime& start_time() const {
+    DCHECK(start_time_ > MonoTime::Min());
     return start_time_;
   }
 
@@ -176,8 +183,14 @@ class ServerBase {
   void LogUnauthorizedAccess(rpc::RpcContext* rpc) const;
 
   const std::string name_;
-  // Seconds since the epoch.
-  int64_t start_time_;
+
+  // Start wall clock time: wall clock as seconds since the Epoch when
+  // the Start() method was called.
+  int64_t start_walltime_;
+
+  // Start time: a snapshot of the monotonic clock when the Start() method
+  // was called.
+  MonoTime start_time_;
 
   std::unique_ptr<MinidumpExceptionHandler> minidump_handler_;
   std::shared_ptr<MemTracker> mem_tracker_;
@@ -247,6 +260,10 @@ class ServerBase {
 
   std::unique_ptr<ScopedGLogMetrics> glog_metrics_;
 
+  // NOTE: it's important that this is the first member to be destructed. This
+  // ensures we do not attempt to collect metrics while calling the destructor.
+  FunctionGaugeDetacher metric_detacher_;
+
   DISALLOW_COPY_AND_ASSIGN(ServerBase);
 };
 
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index e3bf926..b38bd27 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -355,7 +355,7 @@ Status Heartbeater::Thread::SetupRegistration(ServerRegistrationPB* reg) {
     reg->set_https_enabled(server_->web_server()->IsSecure());
   }
   reg->set_software_version(VersionInfo::GetVersionInfo());
-  reg->set_start_time(server_->start_time());
+  reg->set_start_time(server_->start_walltime());
 
   return Status::OK();
 }