You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by mj...@apache.org on 2017/07/26 19:32:47 UTC

incubator-impala git commit: IMPALA-5511: Add process start time to debug web page

Repository: incubator-impala
Updated Branches:
  refs/heads/master c5a9b43db -> b088878c8


IMPALA-5511: Add process start time to debug web page

Read the start date and time of the impalad, catalogd and statestored processes
for the Debug Web UI. Uses the stat command on the /proc/<pid> directory and
format the date with the date command to local time format.

Change-Id: I05ae2f80835b1b0e4bc3b38731778ba0871338a4
Reviewed-on: http://gerrit.cloudera.org:8080/7363
Reviewed-by: Matthew Jacobs <mj...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/incubator-impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-impala/commit/b088878c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/b088878c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/b088878c

Branch: refs/heads/master
Commit: b088878c8c7dab7783bc73fb9514e83095292c20
Parents: c5a9b43
Author: Gabor Kaszab <ga...@cloudera.com>
Authored: Wed Jul 5 17:05:17 2017 +0200
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Jul 26 19:02:07 2017 +0000

----------------------------------------------------------------------
 be/src/catalog/catalogd-main.cc       |  3 ++
 be/src/service/impala-server.cc       |  2 --
 be/src/service/impalad-main.cc        |  3 ++
 be/src/statestore/statestored-main.cc |  3 ++
 be/src/util/CMakeLists.txt            |  1 +
 be/src/util/common-metrics.cc         | 33 +++++++++++++++++++
 be/src/util/common-metrics.h          | 36 +++++++++++++++++++++
 be/src/util/default-path-handlers.cc  | 51 ++++++++++++++++++++++++++++--
 be/src/util/default-path-handlers.h   |  8 ++++-
 be/src/util/impalad-metrics.cc        |  5 ---
 be/src/util/impalad-metrics.h         |  4 ---
 be/src/util/webserver-test.cc         |  2 ++
 be/src/util/webserver.cc              | 30 ------------------
 be/src/util/webserver.h               |  3 --
 common/thrift/metrics.json            | 10 +++---
 www/root.tmpl                         |  3 ++
 16 files changed, 146 insertions(+), 51 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/catalog/catalogd-main.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalogd-main.cc b/be/src/catalog/catalogd-main.cc
index 75b7caa..102b66b 100644
--- a/be/src/catalog/catalogd-main.cc
+++ b/be/src/catalog/catalogd-main.cc
@@ -32,6 +32,7 @@
 #include "util/debug-util.h"
 #include "util/jni-util.h"
 #include "util/metrics.h"
+#include "util/common-metrics.h"
 #include "util/network-util.h"
 #include "util/memory-metrics.h"
 #include "util/webserver.h"
@@ -76,6 +77,8 @@ int CatalogdMain(int argc, char** argv) {
   InitRpcEventTracing(webserver.get());
   metrics->AddProperty<string>("catalog.version", GetVersionString(true));
 
+  CommonMetrics::InitCommonMetrics(metrics.get());
+
   CatalogServer catalog_server(metrics.get());
   ABORT_IF_ERROR(catalog_server.Start());
   catalog_server.RegisterWebpages(webserver.get());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index c3d22fe..4870091 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -337,8 +337,6 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
   // Initialize impalad metrics
   ImpaladMetrics::CreateMetrics(
       exec_env->metrics()->GetOrCreateChildGroup("impala-server"));
-  ImpaladMetrics::IMPALA_SERVER_START_TIME->set_value(
-      TimestampValue::LocalTime().ToString());
 
   ABORT_IF_ERROR(ExternalDataSourceExecutor::InitJNI(exec_env->metrics()));
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/service/impalad-main.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index 70cecba..1f0dd81 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -33,6 +33,7 @@
 #include "common/status.h"
 #include "runtime/coordinator.h"
 #include "runtime/exec-env.h"
+#include "util/common-metrics.h"
 #include "util/jni-util.h"
 #include "util/network-util.h"
 #include "rpc/thrift-util.h"
@@ -79,6 +80,8 @@ int ImpaladMain(int argc, char** argv) {
   StartThreadInstrumentation(exec_env.metrics(), exec_env.webserver(), true);
   InitRpcEventTracing(exec_env.webserver());
 
+  CommonMetrics::InitCommonMetrics(exec_env.metrics());
+
   ThriftServer* beeswax_server = NULL;
   ThriftServer* hs2_server = NULL;
   ThriftServer* be_server = NULL;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/statestore/statestored-main.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestored-main.cc b/be/src/statestore/statestored-main.cc
index 0886c70..53286f5 100644
--- a/be/src/statestore/statestored-main.cc
+++ b/be/src/statestore/statestored-main.cc
@@ -28,6 +28,7 @@
 #include "rpc/rpc-trace.h"
 #include "runtime/mem-tracker.h"
 #include "statestore/statestore.h"
+#include "util/common-metrics.h"
 #include "util/debug-util.h"
 #include "util/metrics.h"
 #include "util/memory-metrics.h"
@@ -71,6 +72,8 @@ int StatestoredMain(int argc, char** argv) {
   // both statestored and impalad
   metrics->AddProperty<string>("statestore.version", GetVersionString(true));
 
+  CommonMetrics::InitCommonMetrics(metrics.get());
+
   Statestore statestore(metrics.get());
   statestore.RegisterWebpages(webserver.get());
   boost::shared_ptr<TProcessor> processor(

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/util/CMakeLists.txt b/be/src/util/CMakeLists.txt
index 8039b31..c38be86 100644
--- a/be/src/util/CMakeLists.txt
+++ b/be/src/util/CMakeLists.txt
@@ -34,6 +34,7 @@ add_library(Util
   bloom-filter.cc
   coding-util.cc
   codec.cc
+  common-metrics.cc
   compress.cc
   cpu-info.cc
   decimal-util.cc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/common-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/common-metrics.cc b/be/src/util/common-metrics.cc
new file mode 100644
index 0000000..ed724bb
--- /dev/null
+++ b/be/src/util/common-metrics.cc
@@ -0,0 +1,33 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/common-metrics.h"
+#include "runtime/timestamp-value.h"
+
+namespace impala {
+
+StringProperty* CommonMetrics::PROCESS_START_TIME = nullptr;
+string CommonMetrics::PROCESS_START_TIME_METRIC_NAME = "process-start-time";
+
+void CommonMetrics::InitCommonMetrics(MetricGroup* metric_group) {
+  PROCESS_START_TIME = metric_group->AddProperty<string>(
+    PROCESS_START_TIME_METRIC_NAME, "");
+  // TODO: IMPALA-5599: Clean up non-TIMESTAMP usages of TimestampValue
+  PROCESS_START_TIME->set_value(TimestampValue::LocalTime().ToString());
+}
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/common-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/common-metrics.h b/be/src/util/common-metrics.h
new file mode 100644
index 0000000..05f0091
--- /dev/null
+++ b/be/src/util/common-metrics.h
@@ -0,0 +1,36 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "util/metrics.h"
+
+namespace impala {
+
+/// This class stores the metrics that are common for the Impalad
+/// Statestored and Catalogd processes.
+/// Also responsible for registering and initializing these metrics.
+class CommonMetrics {
+public:
+  static StringProperty* PROCESS_START_TIME;
+
+  /// Registers and initializes the commnon metrics
+  static void InitCommonMetrics(MetricGroup* metric_group);
+
+private:
+  static string PROCESS_START_TIME_METRIC_NAME;
+};
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/default-path-handlers.cc
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.cc b/be/src/util/default-path-handlers.cc
index ef17a1f..508ea45 100644
--- a/be/src/util/default-path-handlers.cc
+++ b/be/src/util/default-path-handlers.cc
@@ -27,9 +27,15 @@
 
 #include "common/logging.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/exec-env.h"
+#include "service/impala-server.h"
+#include "util/common-metrics.h"
 #include "util/debug-util.h"
 #include "util/pprof-path-handlers.h"
-#include "util/webserver.h"
+#include "util/mem-info.h"
+#include "util/cpu-info.h"
+#include "util/disk-info.h"
+#include "util/process-state-info.h"
 
 #include "common/names.h"
 
@@ -174,8 +180,41 @@ void MemUsageHandler(MemTracker* mem_tracker, MetricGroup* metric_group,
   }
 }
 
+namespace impala {
+
+void RootHandler(const Webserver::ArgumentMap& args, Document* document) {
+  Value version(GetVersionString().c_str(), document->GetAllocator());
+  document->AddMember("version", version, document->GetAllocator());
+  Value cpu_info(CpuInfo::DebugString().c_str(), document->GetAllocator());
+  document->AddMember("cpu_info", cpu_info, document->GetAllocator());
+  Value mem_info(MemInfo::DebugString().c_str(), document->GetAllocator());
+  document->AddMember("mem_info", mem_info, document->GetAllocator());
+  Value disk_info(DiskInfo::DebugString().c_str(), document->GetAllocator());
+  document->AddMember("disk_info", disk_info, document->GetAllocator());
+  Value os_info(OsInfo::DebugString().c_str(), document->GetAllocator());
+  document->AddMember("os_info", os_info, document->GetAllocator());
+  Value process_state_info(ProcessStateInfo().DebugString().c_str(),
+    document->GetAllocator());
+  document->AddMember("process_state_info", process_state_info,
+    document->GetAllocator());
+
+  if (CommonMetrics::PROCESS_START_TIME != nullptr) {
+    Value process_start_time(CommonMetrics::PROCESS_START_TIME->value().c_str(),
+      document->GetAllocator());
+    document->AddMember("process_start_time", process_start_time,
+      document->GetAllocator());
+  }
 
-void impala::AddDefaultUrlCallbacks(
+  ExecEnv* env = ExecEnv::GetInstance();
+  if (env == nullptr || env->impala_server() == nullptr) return;
+  document->AddMember("impala_server_mode", true, document->GetAllocator());
+  document->AddMember("is_coordinator", env->impala_server()->IsCoordinator(),
+      document->GetAllocator());
+  document->AddMember("is_executor", env->impala_server()->IsExecutor(),
+      document->GetAllocator());
+}
+
+void AddDefaultUrlCallbacks(
     Webserver* webserver, MemTracker* process_mem_tracker, MetricGroup* metric_group) {
   webserver->RegisterUrlCallback("/logs", "logs.tmpl", LogsHandler);
   webserver->RegisterUrlCallback("/varz", "flags.tmpl", FlagsHandler);
@@ -193,4 +232,12 @@ void impala::AddDefaultUrlCallbacks(
     AddPprofUrlCallbacks(webserver);
   }
 #endif
+
+  auto root_handler =
+    [](const Webserver::ArgumentMap& args, Document* doc) {
+      RootHandler(args, doc);
+    };
+  webserver->RegisterUrlCallback("/", "root.tmpl", root_handler);
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/default-path-handlers.h
----------------------------------------------------------------------
diff --git a/be/src/util/default-path-handlers.h b/be/src/util/default-path-handlers.h
index aca8b83..b059f87 100644
--- a/be/src/util/default-path-handlers.h
+++ b/be/src/util/default-path-handlers.h
@@ -20,16 +20,22 @@
 
 #include <stdio.h>
 
+#include "util/webserver.h"
+#include "rapidjson/document.h"
+
 namespace impala {
 
 class MemTracker;
-class Webserver;
 class MetricGroup;
 
 /// Adds a set of default path handlers to the webserver to display
 /// logs and configuration flags
 void AddDefaultUrlCallbacks(Webserver* webserver, MemTracker* process_mem_tracker = NULL,
     MetricGroup* metric_group = NULL);
+
+/// Registered to handle "/"
+/// Populates document with various system-wide information.
+void RootHandler(const Webserver::ArgumentMap& args, rapidjson::Document* document);
 }
 
 #endif // IMPALA_UTIL_DEFAULT_PATH_HANDLERS_H

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/impalad-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.cc b/be/src/util/impalad-metrics.cc
index 7155a8d..6d214cd 100644
--- a/be/src/util/impalad-metrics.cc
+++ b/be/src/util/impalad-metrics.cc
@@ -26,8 +26,6 @@ namespace impala {
 
 // Naming convention: Components should be separated by '.' and words should
 // be separated by '-'.
-const char* ImpaladMetricKeys::IMPALA_SERVER_START_TIME =
-    "impala-server.start-time";
 const char* ImpaladMetricKeys::IMPALA_SERVER_VERSION =
     "impala-server.version";
 const char* ImpaladMetricKeys::IMPALA_SERVER_READY =
@@ -146,7 +144,6 @@ IntGauge* ImpaladMetrics::RESULTSET_CACHE_TOTAL_BYTES = NULL;
 // Properties
 BooleanProperty* ImpaladMetrics::CATALOG_READY = NULL;
 BooleanProperty* ImpaladMetrics::IMPALA_SERVER_READY = NULL;
-StringProperty* ImpaladMetrics::IMPALA_SERVER_START_TIME = NULL;
 StringProperty* ImpaladMetrics::IMPALA_SERVER_VERSION = NULL;
 
 // Histograms
@@ -159,8 +156,6 @@ ImpaladMetrics::IO_MGR_CACHED_FILE_HANDLES_HIT_RATIO = NULL;
 
 void ImpaladMetrics::CreateMetrics(MetricGroup* m) {
   // Initialize impalad metrics
-  IMPALA_SERVER_START_TIME = m->AddProperty<string>(
-      ImpaladMetricKeys::IMPALA_SERVER_START_TIME, "");
   IMPALA_SERVER_VERSION = m->AddProperty<string>(
       ImpaladMetricKeys::IMPALA_SERVER_VERSION, GetVersionString(true));
   IMPALA_SERVER_READY = m->AddProperty<bool>(

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/impalad-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.h b/be/src/util/impalad-metrics.h
index 2dafc6a..9d1cfc2 100644
--- a/be/src/util/impalad-metrics.h
+++ b/be/src/util/impalad-metrics.h
@@ -29,9 +29,6 @@ class HistogramMetric;
 /// Contains the keys (strings) for impala metrics.
 class ImpaladMetricKeys {
  public:
-  /// Local time that the server started
-  static const char* IMPALA_SERVER_START_TIME;
-
   /// Full version string of the Impala server
   static const char* IMPALA_SERVER_VERSION;
 
@@ -196,7 +193,6 @@ class ImpaladMetrics {
   // Properties
   static BooleanProperty* CATALOG_READY;
   static BooleanProperty* IMPALA_SERVER_READY;
-  static StringProperty* IMPALA_SERVER_START_TIME;
   static StringProperty* IMPALA_SERVER_VERSION;
   // Histograms
   static HistogramMetric* QUERY_DURATIONS;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/webserver-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index d5a1b6d..7298ee8 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -23,6 +23,7 @@
 
 #include "testutil/gtest-util.h"
 #include "util/webserver.h"
+#include "util/default-path-handlers.h"
 #include "common/init.h"
 
 DECLARE_int32(webserver_port);
@@ -94,6 +95,7 @@ Status HttpGet(const string& host, const int32_t& port, const string& url_path,
 TEST(Webserver, SmokeTest) {
   Webserver webserver(FLAGS_webserver_port);
   ASSERT_OK(webserver.Start());
+  AddDefaultUrlCallbacks(&webserver);
 
   stringstream contents;
   ASSERT_OK(HttpGet("localhost", FLAGS_webserver_port, "/", &contents));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/webserver.cc
----------------------------------------------------------------------
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index 92b6061..ef96181 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -176,31 +176,6 @@ Webserver::~Webserver() {
   Stop();
 }
 
-void Webserver::RootHandler(const ArgumentMap& args, Document* document) {
-  Value version(GetVersionString().c_str(), document->GetAllocator());
-  document->AddMember("version", version, document->GetAllocator());
-  Value cpu_info(CpuInfo::DebugString().c_str(), document->GetAllocator());
-  document->AddMember("cpu_info", cpu_info, document->GetAllocator());
-  Value mem_info(MemInfo::DebugString().c_str(), document->GetAllocator());
-  document->AddMember("mem_info", mem_info, document->GetAllocator());
-  Value disk_info(DiskInfo::DebugString().c_str(), document->GetAllocator());
-  document->AddMember("disk_info", disk_info, document->GetAllocator());
-  Value os_info(OsInfo::DebugString().c_str(), document->GetAllocator());
-  document->AddMember("os_info", os_info, document->GetAllocator());
-  Value process_state_info(ProcessStateInfo().DebugString().c_str(),
-    document->GetAllocator());
-  document->AddMember("process_state_info", process_state_info,
-    document->GetAllocator());
-
-  ExecEnv* env = ExecEnv::GetInstance();
-  if (env == nullptr || env->impala_server() == nullptr) return;
-  document->AddMember("impala_server_mode", true, document->GetAllocator());
-  document->AddMember("is_coordinator", env->impala_server()->IsCoordinator(),
-      document->GetAllocator());
-  document->AddMember("is_executor", env->impala_server()->IsExecutor(),
-      document->GetAllocator());
-}
-
 void Webserver::ErrorHandler(const ArgumentMap& args, Document* document) {
   ArgumentMap::const_iterator it = args.find(ERROR_KEY);
   if (it == args.end()) return;
@@ -340,11 +315,6 @@ Status Webserver::Start() {
     return Status(error_msg.str());
   }
 
-  UrlCallback default_callback =
-      bind<void>(mem_fn(&Webserver::RootHandler), this, _1, _2);
-
-  RegisterUrlCallback("/", "root.tmpl", default_callback, false);
-
   LOG(INFO) << "Webserver started";
   return Status::OK();
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/be/src/util/webserver.h
----------------------------------------------------------------------
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index 861291b..1fc7bb2 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -150,9 +150,6 @@ class Webserver {
   void RenderUrlWithTemplate(const ArgumentMap& arguments, const UrlHandler& url_handler,
       std::stringstream* output, ContentType* content_type);
 
-  /// Registered to handle "/", populates document with various system-wide information.
-  void RootHandler(const ArgumentMap& args, rapidjson::Document* document);
-
   /// Called when an error is encountered, e.g. when a handler for a URI cannot be found.
   void ErrorHandler(const ArgumentMap& args, rapidjson::Document* document);
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/common/thrift/metrics.json
----------------------------------------------------------------------
diff --git a/common/thrift/metrics.json b/common/thrift/metrics.json
index 4d9bbf2..4f86363 100644
--- a/common/thrift/metrics.json
+++ b/common/thrift/metrics.json
@@ -570,14 +570,16 @@
     "key": "impala-server.hedged-read-ops-win"
   },
   {
-    "description": "The local start time of the Impala Server.",
+    "description": "The local start time of the process",
     "contexts": [
-      "IMPALAD"
+      "IMPALAD",
+      "CATALOGSERVER",
+      "STATESTORE"
     ],
-    "label": "Impala Server Start Time",
+    "label": "Process Start Time",
     "units": "NONE",
     "kind": "PROPERTY",
-    "key": "impala-server.start-time"
+    "key": "process-start-time"
   },
   {
     "description": "The full version string of the Impala Server.",

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/b088878c/www/root.tmpl
----------------------------------------------------------------------
diff --git a/www/root.tmpl b/www/root.tmpl
index d9c6d2a..40d1cf3 100644
--- a/www/root.tmpl
+++ b/www/root.tmpl
@@ -26,6 +26,9 @@ under the License.
   <h2>Vers<span id="v">i</span>on</h2>
   <pre id="version_pre">{{version}}</pre>
 
+  <h2>Process Start Time</h2>
+  <pre>{{process_start_time}}</pre>
+
   <h2>Hardware Info</h2>
   <pre>{{cpu_info}} {{mem_info}} {{disk_info}}</pre>