You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2017/03/08 06:20:03 UTC

[5/8] incubator-impala git commit: IMPALA-4885: Expose Jvm thread info in web UI

IMPALA-4885: Expose Jvm thread info in web UI

This commit exposes information about JVM threads to the impalad and
catalogd web UIs. This information includes statistics about the number
of threads running in the JVM as well as per-thread stacktraces, monitors and
synchronizers. Total CPU, user CPU and blocked time is also reported per
thread.

Change-Id: Id497043ab33dcf107a562f0b1ccd5c46095d397f
Reviewed-on: http://gerrit.cloudera.org:8080/6013
Reviewed-by: Dimitris Tsirogiannis <dt...@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/317041f7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/317041f7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/317041f7

Branch: refs/heads/master
Commit: 317041f775c268d120ef99dfcb8770eed09ba52e
Parents: c4fb67c
Author: Dimitris Tsirogiannis <dt...@cloudera.com>
Authored: Tue Feb 14 22:30:45 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Wed Mar 8 01:36:41 2017 +0000

----------------------------------------------------------------------
 be/src/catalog/catalogd-main.cc                 |   2 +-
 be/src/service/impalad-main.cc                  |   2 +-
 be/src/statestore/statestored-main.cc           |   2 +-
 be/src/util/jni-util.cc                         |  12 +
 be/src/util/jni-util.h                          |   6 +
 be/src/util/thread.cc                           | 243 ++++++++++++++-----
 be/src/util/thread.h                            |   6 +-
 common/thrift/Frontend.thrift                   |  45 ++++
 .../java/org/apache/impala/common/JniUtil.java  |  56 ++++-
 www/jvm-threadz.tmpl                            |  67 +++++
 www/threadz.tmpl                                |  12 +
 www/threadz_tabs.tmpl                           |  27 +++
 12 files changed, 402 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/catalog/catalogd-main.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalogd-main.cc b/be/src/catalog/catalogd-main.cc
index 52bf283..b484c9d 100644
--- a/be/src/catalog/catalogd-main.cc
+++ b/be/src/catalog/catalogd-main.cc
@@ -71,7 +71,7 @@ int CatalogdMain(int argc, char** argv) {
 
   metrics->Init(FLAGS_enable_webserver ? webserver.get() : NULL);
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), true));
-  StartThreadInstrumentation(metrics.get(), webserver.get());
+  StartThreadInstrumentation(metrics.get(), webserver.get(), true);
 
   InitRpcEventTracing(webserver.get());
   metrics->AddProperty<string>("catalog.version", GetVersionString(true));

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/service/impalad-main.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impalad-main.cc b/be/src/service/impalad-main.cc
index 2a5f3fd..341399c 100644
--- a/be/src/service/impalad-main.cc
+++ b/be/src/service/impalad-main.cc
@@ -76,7 +76,7 @@ int ImpaladMain(int argc, char** argv) {
 
   // start backend service for the coordinator on be_port
   ExecEnv exec_env;
-  StartThreadInstrumentation(exec_env.metrics(), exec_env.webserver());
+  StartThreadInstrumentation(exec_env.metrics(), exec_env.webserver(), true);
   InitRpcEventTracing(exec_env.webserver());
 
   ThriftServer* beeswax_server = NULL;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/statestore/statestored-main.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestored-main.cc b/be/src/statestore/statestored-main.cc
index 5f1ac25..4e5680f 100644
--- a/be/src/statestore/statestored-main.cc
+++ b/be/src/statestore/statestored-main.cc
@@ -65,7 +65,7 @@ int StatestoredMain(int argc, char** argv) {
 
   metrics->Init(FLAGS_enable_webserver ? webserver.get() : NULL);
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), false));
-  StartThreadInstrumentation(metrics.get(), webserver.get());
+  StartThreadInstrumentation(metrics.get(), webserver.get(), false);
   InitRpcEventTracing(webserver.get());
   // TODO: Add a 'common metrics' method to add standard metrics to
   // both statestored and impalad

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/util/jni-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.cc b/be/src/util/jni-util.cc
index a4ca7c4..c22460e 100644
--- a/be/src/util/jni-util.cc
+++ b/be/src/util/jni-util.cc
@@ -30,6 +30,7 @@ namespace impala {
 jclass JniUtil::jni_util_cl_ = NULL;
 jclass JniUtil::internal_exc_cl_ = NULL;
 jmethodID JniUtil::get_jvm_metrics_id_ = NULL;
+jmethodID JniUtil::get_jvm_threads_id_ = NULL;
 jmethodID JniUtil::throwable_to_string_id_ = NULL;
 jmethodID JniUtil::throwable_to_stack_trace_id_ = NULL;
 
@@ -139,6 +140,12 @@ Status JniUtil::Init() {
     return Status("Failed to find JniUtil.getJvmMetrics method.");
   }
 
+  get_jvm_threads_id_ =
+      env->GetStaticMethodID(jni_util_cl_, "getJvmThreadsInfo", "([B)[B");
+  if (get_jvm_threads_id_ == NULL) {
+    if (env->ExceptionOccurred()) env->ExceptionDescribe();
+    return Status("Failed to find JniUtil.getJvmThreadsInfo method.");
+  }
 
   return Status::OK();
 }
@@ -182,6 +189,11 @@ Status JniUtil::GetJvmMetrics(const TGetJvmMetricsRequest& request,
   return JniUtil::CallJniMethod(jni_util_class(), get_jvm_metrics_id_, request, result);
 }
 
+Status JniUtil::GetJvmThreadsInfo(const TGetJvmThreadsInfoRequest& request,
+    TGetJvmThreadsInfoResponse* result) {
+  return JniUtil::CallJniMethod(jni_util_class(), get_jvm_threads_id_, request, result);
+}
+
 Status JniUtil::LoadJniMethod(JNIEnv* env, const jclass& jni_class,
     JniMethodDescriptor* descriptor) {
   (*descriptor->method_id) = env->GetMethodID(jni_class,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/util/jni-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h
index 9d125b9..4ed8abc 100644
--- a/be/src/util/jni-util.h
+++ b/be/src/util/jni-util.h
@@ -221,6 +221,11 @@ class JniUtil {
   static Status GetJvmMetrics(const TGetJvmMetricsRequest& request,
       TGetJvmMetricsResponse* result);
 
+  // Populates 'result' with information about live JVM threads. Returns
+  // Status::OK unless there is an exception.
+  static Status GetJvmThreadsInfo(const TGetJvmThreadsInfoRequest& request,
+      TGetJvmThreadsInfoResponse* result);
+
   /// Loads a method whose signature is in the supplied descriptor. Returns Status::OK
   /// and sets descriptor->method_id to a JNI method handle if successful, otherwise an
   /// error status is returned.
@@ -330,6 +335,7 @@ class JniUtil {
   static jmethodID throwable_to_string_id_;
   static jmethodID throwable_to_stack_trace_id_;
   static jmethodID get_jvm_metrics_id_;
+  static jmethodID get_jvm_threads_id_;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/util/thread.cc
----------------------------------------------------------------------
diff --git a/be/src/util/thread.cc b/be/src/util/thread.cc
index 757ba59..b59f4f9 100644
--- a/be/src/util/thread.cc
+++ b/be/src/util/thread.cc
@@ -26,6 +26,7 @@
 #include "util/coding-util.h"
 #include "util/debug-util.h"
 #include "util/error-util.h"
+#include "util/jni-util.h"
 #include "util/metrics.h"
 #include "util/webserver.h"
 #include "util/os-util.h"
@@ -38,8 +39,12 @@ using namespace rapidjson;
 
 namespace impala {
 
+static const string JVM_THREADS_WEB_PAGE = "/jvm-threadz";
+static const string JVM_THREADS_TEMPLATE = "jvm-threadz.tmpl";
 static const string THREADS_WEB_PAGE = "/threadz";
 static const string THREADS_TEMPLATE = "threadz.tmpl";
+static const string THREAD_GROUP_WEB_PAGE = "/thread-group";
+static const string THREAD_GROUP_TEMPLATE = "/thread-group.tmpl";
 
 class ThreadMgr;
 
@@ -50,13 +55,39 @@ class ThreadMgr;
 // manager after the destruction can be avoided.
 shared_ptr<ThreadMgr> thread_manager;
 
+namespace {
+
+// Example output:
+// "overview" : {
+//   "thread_count" : 30,
+//   "daemon_count" : 4,
+//   "peak_count" : 40
+// }
+// "threads": [
+//   {
+//     "summary" : "main ID:1 RUNNABLE",
+//     "cpu_time_sec" : 1.303,
+//     "user_time_sec" : 2.323,
+//     "blocked_time_ms" : -1,
+//     "blocked_count" : 20,
+//     "is_native" : false
+//   },
+//   { ... }
+// ]
+void JvmThreadsUrlCallback(const Webserver::ArgumentMap& args, Document* doc);
+
+void ThreadOverviewUrlCallback(bool include_jvm_threads,
+    const Webserver::ArgumentMap& args, Document* document);
+
+}
+
 // A singleton class that tracks all live threads, and groups them together for easy
 // auditing. Used only by Thread.
 class ThreadMgr {
  public:
   ThreadMgr() : metrics_enabled_(false) { }
 
-  Status StartInstrumentation(MetricGroup* metrics, Webserver* webserver);
+  Status StartInstrumentation(MetricGroup* metrics);
 
   // Registers a thread to the supplied category. The key is a boost::thread::id, used
   // instead of the system TID since boost::thread::id is always available, unlike
@@ -68,6 +99,46 @@ class ThreadMgr {
   // already been removed, this is a no-op.
   void RemoveThread(const thread::id& boost_id, const string& category);
 
+  // Example output:
+  // "total_threads": 144,
+  //   "thread-groups": [
+  //       {
+  //         "name": "common",
+  //             "size": 1
+  //             },
+  //       {
+  //         "name": "disk-io-mgr",
+  //             "size": 2
+  //             },
+  //       {
+  //         "name": "hdfs-worker-pool",
+  //             "size": 16
+  //             },
+  //             ... etc ...
+  //      ]
+  void GetThreadOverview(Document* document);
+
+  // Example output:
+  // "thread-group": {
+  //   "category": "disk-io-mgr",
+  //       "size": 2
+  //       },
+  //   "threads": [
+  //       {
+  //         "name": "work-loop(Disk: 0, Thread: 0)-17049",
+  //             "user_ns": 0,
+  //             "kernel_ns": 0,
+  //             "iowait_ns": 0
+  //             },
+  //       {
+  //         "name": "work-loop(Disk: 1, Thread: 0)-17050",
+  //             "user_ns": 0,
+  //             "kernel_ns": 0,
+  //             "iowait_ns": 0
+  //             }
+  //        ]
+  void ThreadGroupUrlCallback(const Webserver::ArgumentMap& args, Document* output);
+
  private:
   // Container class for any details we want to capture about a thread
   // TODO: Add start-time.
@@ -110,69 +181,16 @@ class ThreadMgr {
   // current number of running threads.
   IntGauge* total_threads_metric_;
   IntGauge* current_num_threads_metric_;
-
-  // Webpage callbacks; print all threads by category
-  // Example output:
-  // "total_threads": 144,
-  //   "thread-groups": [
-  //       {
-  //         "name": "common",
-  //             "size": 1
-  //             },
-  //       {
-  //         "name": "disk-io-mgr",
-  //             "size": 2
-  //             },
-  //       {
-  //         "name": "hdfs-worker-pool",
-  //             "size": 16
-  //             },
-  //             ... etc ...
-  //      ]
-  void ThreadGroupUrlCallback(const Webserver::ArgumentMap& args, Document* output);
-
-  // Example output:
-  // "thread-group": {
-  //   "category": "disk-io-mgr",
-  //       "size": 2
-  //       },
-  //   "threads": [
-  //       {
-  //         "name": "work-loop(Disk: 0, Thread: 0)-17049",
-  //             "user_ns": 0,
-  //             "kernel_ns": 0,
-  //             "iowait_ns": 0
-  //             },
-  //       {
-  //         "name": "work-loop(Disk: 1, Thread: 0)-17050",
-  //             "user_ns": 0,
-  //             "kernel_ns": 0,
-  //             "iowait_ns": 0
-  //             }
-  //        ]
-  void ThreadOverviewUrlCallback(const Webserver::ArgumentMap& args, Document* document);
 };
 
-Status ThreadMgr::StartInstrumentation(MetricGroup* metrics, Webserver* webserver) {
+Status ThreadMgr::StartInstrumentation(MetricGroup* metrics) {
   DCHECK(metrics != NULL);
-  DCHECK(webserver != NULL);
   lock_guard<mutex> l(lock_);
   metrics_enabled_ = true;
   total_threads_metric_ = metrics->AddGauge<int64_t>(
       "thread-manager.total-threads-created", 0L);
   current_num_threads_metric_ = metrics->AddGauge<int64_t>(
       "thread-manager.running-threads", 0L);
-
-  Webserver::UrlCallback template_callback =
-      bind<void>(mem_fn(&ThreadMgr::ThreadOverviewUrlCallback), this, _1, _2);
-  webserver->RegisterUrlCallback(THREADS_WEB_PAGE, THREADS_TEMPLATE,
-      template_callback);
-
-  Webserver::UrlCallback overview_callback =
-      bind<void>(mem_fn(&ThreadMgr::ThreadGroupUrlCallback), this, _1, _2);
-  webserver->RegisterUrlCallback("/thread-group", "thread-group.tmpl",
-      overview_callback, false);
-
   return Status::OK();
 }
 
@@ -194,8 +212,7 @@ void ThreadMgr::RemoveThread(const thread::id& boost_id, const string& category)
   if (metrics_enabled_) current_num_threads_metric_->Increment(-1L);
 }
 
-void ThreadMgr::ThreadOverviewUrlCallback(const Webserver::ArgumentMap& args,
-    Document* document) {
+void ThreadMgr::GetThreadOverview(Document* document) {
   lock_guard<mutex> l(lock_);
   if (metrics_enabled_) {
     document->AddMember("total_threads", current_num_threads_metric_->value(),
@@ -261,17 +278,8 @@ void ThreadMgr::ThreadGroupUrlCallback(const Webserver::ArgumentMap& args,
   document->AddMember("threads", lst, document->GetAllocator());
 }
 
-void InitThreading() {
-  DCHECK(thread_manager.get() == NULL);
-  thread_manager.reset(new ThreadMgr());
-}
-
-Status StartThreadInstrumentation(MetricGroup* metrics, Webserver* webserver) {
-  return thread_manager->StartInstrumentation(metrics, webserver);
-}
-
 void Thread::StartThread(const ThreadFunctor& functor) {
-  DCHECK(thread_manager.get() != NULL)
+  DCHECK(thread_manager.get() != nullptr)
       << "Thread created before InitThreading called";
   DCHECK(tid_ == UNINITIALISED_THREAD_ID) << "StartThread called twice";
 
@@ -327,4 +335,105 @@ void ThreadGroup::JoinAll() {
   for (const Thread& thread: threads_) thread.Join();
 }
 
+namespace {
+
+void RegisterUrlCallbacks(bool include_jvm_threads, Webserver* webserver) {
+  DCHECK(webserver != nullptr);
+  auto overview_callback = [include_jvm_threads]
+      (const Webserver::ArgumentMap& args, Document* doc) {
+    ThreadOverviewUrlCallback(include_jvm_threads, args, doc);
+  };
+  webserver->RegisterUrlCallback(THREADS_WEB_PAGE, THREADS_TEMPLATE, overview_callback);
+
+  auto group_callback = [] (const Webserver::ArgumentMap& args, Document* doc) {
+    thread_manager->ThreadGroupUrlCallback(args, doc);
+  };
+  webserver->RegisterUrlCallback(THREAD_GROUP_WEB_PAGE, THREAD_GROUP_TEMPLATE,
+      group_callback, false);
+
+  if (include_jvm_threads) {
+    auto jvm_threads_callback = [] (const Webserver::ArgumentMap& args, Document* doc) {
+      JvmThreadsUrlCallback(args, doc);
+    };
+    webserver->RegisterUrlCallback(JVM_THREADS_WEB_PAGE, JVM_THREADS_TEMPLATE,
+        jvm_threads_callback, false);
+  }
+}
+
+void ThreadOverviewUrlCallback(bool include_jvm_threads,
+    const Webserver::ArgumentMap& args, Document* document) {
+  thread_manager->GetThreadOverview(document);
+  if (!include_jvm_threads) return;
+
+  // Add information about the JVM threads
+  TGetJvmThreadsInfoRequest request;
+  request.get_complete_info = false;
+  TGetJvmThreadsInfoResponse response;
+  Status status = JniUtil::GetJvmThreadsInfo(request, &response);
+  if (!status.ok()) {
+    Value error(Substitute("Couldn't retrieve information about JVM threads: $0",
+        status.GetDetail()).c_str(), document->GetAllocator());
+    document->AddMember("error", error, document->GetAllocator());
+    return;
+  }
+  Value jvm_threads_val(kObjectType);
+  jvm_threads_val.AddMember("name", "jvm", document->GetAllocator());
+  jvm_threads_val.AddMember("total", response.total_thread_count,
+      document->GetAllocator());
+  jvm_threads_val.AddMember("daemon", response.daemon_thread_count,
+      document->GetAllocator());
+  document->AddMember("jvm-threads", jvm_threads_val, document->GetAllocator());
+}
+
+void JvmThreadsUrlCallback(const Webserver::ArgumentMap& args, Document* doc) {
+  DCHECK(doc != NULL);
+  TGetJvmThreadsInfoRequest request;
+  request.get_complete_info = true;
+  TGetJvmThreadsInfoResponse response;
+  Status status = JniUtil::GetJvmThreadsInfo(request, &response);
+  if (!status.ok()) {
+    Value error(Substitute("Couldn't retrieve information about JVM threads: $0",
+        status.GetDetail()).c_str(), doc->GetAllocator());
+    doc->AddMember("error", error, doc->GetAllocator());
+    return;
+  }
+  Value overview(kObjectType);
+  overview.AddMember("thread_count", response.total_thread_count, doc->GetAllocator());
+  overview.AddMember("daemon_count", response.daemon_thread_count, doc->GetAllocator());
+  overview.AddMember("peak_count", response.peak_thread_count, doc->GetAllocator());
+  doc->AddMember("overview", overview, doc->GetAllocator());
+
+  Value lst(kArrayType);
+  for (const TJvmThreadInfo& thread: response.threads) {
+    Value val(kObjectType);
+    Value summary(thread.summary.c_str(), doc->GetAllocator());
+    val.AddMember("summary", summary, doc->GetAllocator());
+    val.AddMember("cpu_time_sec", static_cast<double>(thread.cpu_time_in_ns) / 1e9,
+        doc->GetAllocator());
+    val.AddMember("user_time_sec", static_cast<double>(thread.user_time_in_ns) / 1e9,
+        doc->GetAllocator());
+    val.AddMember("blocked_time_ms", thread.blocked_time_in_ms, doc->GetAllocator());
+    val.AddMember("blocked_count", thread.blocked_count, doc->GetAllocator());
+    val.AddMember("is_native", thread.is_in_native, doc->GetAllocator());
+    lst.PushBack(val, doc->GetAllocator());
+  }
+  doc->AddMember("jvm-threads", lst, doc->GetAllocator());
+}
+
+}
+
+void InitThreading() {
+  DCHECK(thread_manager.get() == nullptr);
+  thread_manager.reset(new ThreadMgr());
+}
+
+Status StartThreadInstrumentation(MetricGroup* metrics, Webserver* webserver,
+    bool include_jvm_threads) {
+  DCHECK(metrics != nullptr);
+  DCHECK(webserver != nullptr);
+  RETURN_IF_ERROR(thread_manager->StartInstrumentation(metrics));
+  RegisterUrlCallbacks(include_jvm_threads, webserver);
+  return Status::OK();
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/be/src/util/thread.h
----------------------------------------------------------------------
diff --git a/be/src/util/thread.h b/be/src/util/thread.h
index a142a94..3a96233 100644
--- a/be/src/util/thread.h
+++ b/be/src/util/thread.h
@@ -188,8 +188,10 @@ class ThreadGroup {
 void InitThreading();
 
 /// Registers /threadz with the debug webserver, and creates thread-tracking metrics under
-/// the "thread-manager." prefix
-Status StartThreadInstrumentation(MetricGroup* metrics, Webserver* webserver);
+/// the "thread-manager." If 'include_jvm_threads' is true, shows information about
+/// live JVM threads in the web UI.
+Status StartThreadInstrumentation(MetricGroup* metrics, Webserver* webserver,
+    bool include_jvm_threads);
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/common/thrift/Frontend.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Frontend.thrift b/common/thrift/Frontend.thrift
index 8f8e3ac..4cef592 100644
--- a/common/thrift/Frontend.thrift
+++ b/common/thrift/Frontend.thrift
@@ -715,6 +715,51 @@ struct TGetJvmMetricsResponse {
   1: required list<TJvmMemoryPool> memory_pools
 }
 
+// Contains information about a JVM thread
+struct TJvmThreadInfo {
+  // Summary of a JVM thread. Includes stacktraces, locked monitors
+  // and synchronizers.
+  1: required string summary
+
+  // The total CPU time for this thread in nanoseconds
+  2: required i64 cpu_time_in_ns
+
+  // The CPU time that this thread has executed in user mode in nanoseconds
+  3: required i64 user_time_in_ns
+
+  // The number of times this thread blocked to enter or reenter a monitor
+  4: required i64 blocked_count
+
+  // Approximate accumulated elapsed time (in milliseconds) that this thread has blocked
+  // to enter or reenter a monitor
+  5: required i64 blocked_time_in_ms
+
+  // True if this thread is executing native code via the Java Native Interface (JNI)
+  6: required bool is_in_native
+}
+
+// Request to get information about JVM threads
+struct TGetJvmThreadsInfoRequest {
+  // If set, return complete info about JVM threads. Otherwise, return only
+  // the total number of live JVM threads.
+  1: required bool get_complete_info
+}
+
+struct TGetJvmThreadsInfoResponse {
+  // The current number of live threads including both daemon and non-daemon threads
+  1: required i32 total_thread_count
+
+  // The current number of live daemon threads
+  2: required i32 daemon_thread_count
+
+  // The peak live thread count since the Java virtual machine started
+  3: required i32 peak_thread_count
+
+  // Information about JVM threads. It is not included when
+  // TGetJvmThreadsInfoRequest.get_complete_info is false.
+  4: optional list<TJvmThreadInfo> threads
+}
+
 struct TGetHadoopConfigRequest {
   // The value of the <name> in the config <property>
   1: required string name

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/fe/src/main/java/org/apache/impala/common/JniUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/common/JniUtil.java b/fe/src/main/java/org/apache/impala/common/JniUtil.java
index 223c8bb..1fb00c8 100644
--- a/fe/src/main/java/org/apache/impala/common/JniUtil.java
+++ b/fe/src/main/java/org/apache/impala/common/JniUtil.java
@@ -25,7 +25,11 @@ import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
 import java.lang.management.MemoryPoolMXBean;
 import java.lang.management.MemoryUsage;
+import java.lang.management.RuntimeMXBean;
+import java.lang.management.ThreadMXBean;
+import java.lang.management.ThreadInfo;
 import java.util.ArrayList;
+import java.util.Map;
 
 import org.apache.thrift.TBase;
 import org.apache.thrift.TSerializer;
@@ -34,9 +38,14 @@ import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.protocol.TProtocolFactory;
 
+import com.google.common.base.Joiner;
+
 import org.apache.impala.thrift.TGetJvmMetricsRequest;
 import org.apache.impala.thrift.TGetJvmMetricsResponse;
+import org.apache.impala.thrift.TGetJvmThreadsInfoRequest;
+import org.apache.impala.thrift.TGetJvmThreadsInfoResponse;
 import org.apache.impala.thrift.TJvmMemoryPool;
+import org.apache.impala.thrift.TJvmThreadInfo;
 
 /**
  * Utility class with methods intended for JNI clients
@@ -187,15 +196,50 @@ public class JniUtil {
   }
 
   /**
-   * Get Java version and vendor information
+   * Get information about the live JVM threads.
+   */
+  public static byte[] getJvmThreadsInfo(byte[] argument) throws ImpalaException {
+    TGetJvmThreadsInfoRequest request = new TGetJvmThreadsInfoRequest();
+    JniUtil.deserializeThrift(protocolFactory_, request, argument);
+    TGetJvmThreadsInfoResponse response = new TGetJvmThreadsInfoResponse();
+    ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+    response.setTotal_thread_count(threadBean.getThreadCount());
+    response.setDaemon_thread_count(threadBean.getDaemonThreadCount());
+    response.setPeak_thread_count(threadBean.getPeakThreadCount());
+    if (request.get_complete_info) {
+      for (ThreadInfo threadInfo: threadBean.dumpAllThreads(true, true)) {
+        TJvmThreadInfo tThreadInfo = new TJvmThreadInfo();
+        long id = threadInfo.getThreadId();
+        tThreadInfo.setSummary(threadInfo.toString());
+        tThreadInfo.setCpu_time_in_ns(threadBean.getThreadCpuTime(id));
+        tThreadInfo.setUser_time_in_ns(threadBean.getThreadUserTime(id));
+        tThreadInfo.setBlocked_count(threadInfo.getBlockedCount());
+        tThreadInfo.setBlocked_time_in_ms(threadInfo.getBlockedTime());
+        tThreadInfo.setIs_in_native(threadInfo.isInNative());
+        response.addToThreads(tThreadInfo);
+      }
+    }
+
+    TSerializer serializer = new TSerializer(protocolFactory_);
+    try {
+      return serializer.serialize(response);
+    } catch (TException e) {
+      throw new InternalException(e.getMessage());
+    }
+  }
+
+  /**
+   * Get Java version, input arguments and system properties.
    */
   public static String getJavaVersion() {
+    RuntimeMXBean runtime = ManagementFactory.getRuntimeMXBean();
     StringBuilder sb = new StringBuilder();
-    sb.append("Java Version Info: ");
-    sb.append(System.getProperty("java.runtime.name"));
-    sb.append(" (");
-    sb.append(System.getProperty("java.runtime.version"));
-    sb.append(")");
+    sb.append("Java Input arguments:\n");
+    sb.append(Joiner.on(" ").join(runtime.getInputArguments()));
+    sb.append("\nJava System properties:\n");
+    for (Map.Entry<String, String> entry: runtime.getSystemProperties().entrySet()) {
+      sb.append(entry.getKey() + ":" + entry.getValue() + "\n");
+    }
     return sb.toString();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/www/jvm-threadz.tmpl
----------------------------------------------------------------------
diff --git a/www/jvm-threadz.tmpl b/www/jvm-threadz.tmpl
new file mode 100644
index 0000000..91500d9
--- /dev/null
+++ b/www/jvm-threadz.tmpl
@@ -0,0 +1,67 @@
+<!--
+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.
+-->
+{{> www/common-header.tmpl }}
+
+{{> www/threadz_tabs.tmpl }}
+
+<h3>JVM Threads</h3>
+
+{{#overview}}
+<h3>Total: {{thread_count}}, Daemon: {{daemon_count}}, Peak: {{peak_count}}</h3>
+{{/overview}}
+
+<table id="java-threads-tbl" class='table table-hover table-bordered'>
+  <thead>
+    <tr>
+      <th>Summary</th>
+      <th>CPU time (s)</th>
+      <th>User time (s)</th>
+      <th>Blocked time (ms)</th>
+      <th>Blocked times</th>
+      <th>Native</th>
+    </tr>
+  </thead>
+  <tbody>
+    {{#jvm-threads}}
+    <tr>
+      <td>{{summary}}</td>
+      <td>{{cpu_time_sec}}</td>
+      <td>{{user_time_sec}}</td>
+      <td>{{blocked_time_ms}}</td>
+      <td>{{blocked_count}}</td>
+      <td>{{is_native}}</td>
+    </tr>
+    {{/jvm-threads}}
+  </tbody>
+</table>
+
+<script>
+    $(document).ready(function() {
+        $('#java-threads-tbl').DataTable({
+            "order": [[ 1, "asc" ]],
+            "pageLength": 100
+        });
+    });
+</script>
+
+<script>
+  document.getElementById("jvm-tab").className = "active";
+</script>
+
+{{> www/common-footer.tmpl }}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/www/threadz.tmpl
----------------------------------------------------------------------
diff --git a/www/threadz.tmpl b/www/threadz.tmpl
index 00b02be..9a9e3c1 100644
--- a/www/threadz.tmpl
+++ b/www/threadz.tmpl
@@ -18,6 +18,8 @@ under the License.
 -->
 {{> www/common-header.tmpl }}
 
+{{> www/threadz_tabs.tmpl }}
+
 <h2>Thread Groups</h2>
 
 {{#total-threads}}
@@ -32,4 +34,14 @@ under the License.
 </a>
 {{/thread-groups}}
 
+{{#jvm-threads}}
+<a href='/jvm-threadz'>
+  <h3>{{name}} : (Total: {{total}}, Daemon: {{daemon}})</h3>
+</a>
+{{/jvm-threads}}
+
+<script>
+document.getElementById("overview-tab").className = "active";
+</script>
+
 {{> www/common-footer.tmpl }}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/317041f7/www/threadz_tabs.tmpl
----------------------------------------------------------------------
diff --git a/www/threadz_tabs.tmpl b/www/threadz_tabs.tmpl
new file mode 100644
index 0000000..0dbb6e8
--- /dev/null
+++ b/www/threadz_tabs.tmpl
@@ -0,0 +1,27 @@
+<!--
+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.
+-->
+
+<h2>Threads</h2>
+
+<ul class="nav nav-tabs">
+  <li id="overview-tab" role="presentation"><a href="/threadz">Overview</a></li>
+  {{?jvm-threads}}
+  <li id="jvm-tab" role="presentation"><a href="/jvm-threadz">JVM</a></li>
+  {{/jvm-threads}}
+</ul>