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

[4/5] incubator-impala git commit: IMPALA-4822: Implement dynamic log level changes

IMPALA-4822: Implement dynamic log level changes

Very often we have to change the logging levels
of Impalads and Catalog server for debugging purposes.
Currently, there is no way to do that without a restart
of the daemons, which is not a viable option in production
deployments.

This patch addresses this supportability gap by exposing
the ability to set dynamic logging levels using a simple
web endpoint on Impalad/Catalog/Statestore web pages.

This includes setting VLOG levels (equivalent to --v flag)
as well as setting log4j levels on the Frontend and the
Catalog JVMs.

Change-Id: I588418e9bcb0b66d33138baf96207a5a35bfbd63
Reviewed-on: http://gerrit.cloudera.org:8080/5792
Reviewed-by: Bharath Vissapragada <bh...@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/5d306eff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-impala/tree/5d306eff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-impala/diff/5d306eff

Branch: refs/heads/master
Commit: 5d306eff7fa5a0ef8dbccfe6e5135eaeea25ba1a
Parents: ba2613a
Author: Bharath Vissapragada <bh...@cloudera.com>
Authored: Sun Jan 22 19:17:38 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue Mar 7 07:53:14 2017 +0000

----------------------------------------------------------------------
 be/src/catalog/catalog-server.cc                |   4 +
 be/src/service/impala-http-handler.cc           |   3 +
 be/src/statestore/statestore.cc                 |   3 +
 be/src/util/jni-util.cc                         |   8 +
 be/src/util/jni-util.h                          |  16 +-
 be/src/util/logging-support.cc                  | 184 ++++++++++++++++++-
 be/src/util/logging-support.h                   |  12 +-
 common/thrift/Logging.thrift                    |  13 ++
 .../apache/impala/service/BackendConfig.java    |   3 +
 .../org/apache/impala/util/GlogAppender.java    |  47 ++++-
 tests/webserver/test_web_pages.py               |  90 +++++++++
 www/log_level.tmpl                              |  97 ++++++++++
 12 files changed, 469 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/catalog/catalog-server.cc
----------------------------------------------------------------------
diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index 099ffde..08015b6 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -23,6 +23,8 @@
 #include "catalog/catalog-util.h"
 #include "statestore/statestore-subscriber.h"
 #include "util/debug-util.h"
+#include "util/logging-support.h"
+#include "util/webserver.h"
 #include "gen-cpp/CatalogInternalService_types.h"
 #include "gen-cpp/CatalogObjects_types.h"
 #include "gen-cpp/CatalogService_types.h"
@@ -202,6 +204,8 @@ void CatalogServer::RegisterWebpages(Webserver* webserver) {
       bind<void>(mem_fn(&CatalogServer::CatalogObjectsUrlCallback), this, _1, _2);
   webserver->RegisterUrlCallback(CATALOG_OBJECT_WEB_PAGE, CATALOG_OBJECT_TEMPLATE,
       catalog_objects_callback, false);
+
+  RegisterLogLevelCallbacks(webserver, true);
 }
 
 void CatalogServer::UpdateCatalogTopicCallback(

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/service/impala-http-handler.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 7483dd3..ff3c7bc 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -31,6 +31,7 @@
 #include "service/query-exec-state.h"
 #include "thrift/protocol/TDebugProtocol.h"
 #include "util/coding-util.h"
+#include "util/logging-support.h"
 #include "util/redactor.h"
 #include "util/summary-util.h"
 #include "util/time.h"
@@ -123,6 +124,8 @@ void ImpalaHttpHandler::RegisterHandlers(Webserver* webserver) {
   webserver->RegisterUrlCallback("/query_stmt", "query_stmt.tmpl",
       [this](const auto& args, auto* doc) {
         this->QuerySummaryHandler(false, false, args, doc); }, false);
+
+  RegisterLogLevelCallbacks(webserver, true);
 }
 
 void ImpalaHttpHandler::HadoopVarzHandler(const Webserver::ArgumentMap& args,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/statestore/statestore.cc
----------------------------------------------------------------------
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index e63835f..1b1034e 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -27,6 +27,7 @@
 #include "statestore/failure-detector.h"
 #include "rpc/thrift-util.h"
 #include "util/debug-util.h"
+#include "util/logging-support.h"
 #include "util/time.h"
 #include "util/uid-util.h"
 #include "util/webserver.h"
@@ -262,6 +263,8 @@ void Statestore::RegisterWebpages(Webserver* webserver) {
       bind<void>(&Statestore::SubscribersHandler, this, _1, _2);
   webserver->RegisterUrlCallback("/subscribers", "statestore_subscribers.tmpl",
       subscribers_callback);
+
+  RegisterLogLevelCallbacks(webserver, false);
 }
 
 void Statestore::TopicsHandler(const Webserver::ArgumentMap& args,

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/util/jni-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.cc b/be/src/util/jni-util.cc
index 3d9f6a4..a4ca7c4 100644
--- a/be/src/util/jni-util.cc
+++ b/be/src/util/jni-util.cc
@@ -190,4 +190,12 @@ Status JniUtil::LoadJniMethod(JNIEnv* env, const jclass& jni_class,
   return Status::OK();
 }
 
+Status JniUtil::LoadStaticJniMethod(JNIEnv* env, const jclass& jni_class,
+    JniMethodDescriptor* descriptor) {
+  (*descriptor->method_id) = env->GetStaticMethodID(jni_class,
+      descriptor->name.c_str(), descriptor->signature.c_str());
+  RETURN_ERROR_IF_EXC(env);
+  return Status::OK();
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/util/jni-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/jni-util.h b/be/src/util/jni-util.h
index 18ce315..9d125b9 100644
--- a/be/src/util/jni-util.h
+++ b/be/src/util/jni-util.h
@@ -227,8 +227,20 @@ class JniUtil {
   static Status LoadJniMethod(JNIEnv* jni_env, const jclass& jni_class,
       JniMethodDescriptor* descriptor);
 
-  /// Utility methods to avoid repeating lots of the JNI call boilerplate. It seems these
-  /// must be defined in the header to compile properly.
+  /// Same as LoadJniMethod(...), except that this loads a static method.
+  static Status LoadStaticJniMethod(JNIEnv* jni_env, const jclass& jni_class,
+      JniMethodDescriptor* descriptor);
+
+  /// Utility methods to avoid repeating lots of the JNI call boilerplate.
+  static Status CallJniMethod(const jobject& obj, const jmethodID& method) {
+    JNIEnv* jni_env = getJNIEnv();
+    JniLocalFrame jni_frame;
+    RETURN_IF_ERROR(jni_frame.push(jni_env));
+    jni_env->CallObjectMethod(obj, method);
+    RETURN_ERROR_IF_EXC(jni_env);
+    return Status::OK();
+  }
+
   template <typename T>
   static Status CallJniMethod(const jobject& obj, const jmethodID& method, const T& arg) {
     JNIEnv* jni_env = getJNIEnv();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/util/logging-support.cc
----------------------------------------------------------------------
diff --git a/be/src/util/logging-support.cc b/be/src/util/logging-support.cc
index 6958e43..fd6b8af 100644
--- a/be/src/util/logging-support.cc
+++ b/be/src/util/logging-support.cc
@@ -19,12 +19,18 @@
 
 #include <glob.h>
 #include <sys/stat.h>
+#include <rapidjson/document.h>
+#include <gflags/gflags.h>
 
 #include "common/logging.h"
+#include "rpc/jni-thrift-util.h"
+#include "util/webserver.h"
 
 #include "common/names.h"
 
 using namespace impala;
+using namespace rapidjson;
+using namespace std;
 
 DEFINE_int32(non_impala_java_vlog, 0, "(Advanced) The log level (equivalent to --v) for "
     "non-Impala Java classes (0: INFO, 1 and 2: DEBUG, 3: TRACE)");
@@ -36,12 +42,6 @@ JNIEXPORT void JNICALL
 Java_org_apache_impala_util_NativeLogger_Log(
     JNIEnv* env, jclass caller_class, int severity, jstring msg, jstring file,
     int line_number) {
-
-  // Mimic the behaviour of VLOG(N) by ignoring verbose log messages when appropriate.
-  if (severity == TLogLevel::VLOG && !VLOG_IS_ON(1)) return;
-  if (severity == TLogLevel::VLOG_2 && !VLOG_IS_ON(2)) return;
-  if (severity == TLogLevel::VLOG_3 && !VLOG_IS_ON(3)) return;
-
   // Unused required argument to GetStringUTFChars
   jboolean dummy;
   const char* filename = env->GetStringUTFChars(file, &dummy);
@@ -74,6 +74,160 @@ Java_org_apache_impala_util_NativeLogger_Log(
   env->ReleaseStringUTFChars(file, filename);
 }
 
+namespace {
+// Defaults to startup flag --v. FLAGS_v can be overriden at runtime for
+// debugging, so we save the original value here in case we need to restore
+// the defaults. Set in GetThriftBackendGflags().
+int FLAGS_v_original_value;
+
+static jclass log4j_logger_class_;
+// Jni method descriptors corresponding to getLogLevel() and setLogLevel() operations.
+static jmethodID get_log_level_method; // GlogAppender.getLogLevel()
+static jmethodID set_log_level_method; // GlogAppender.setLogLevel()
+static jmethodID reset_log_levels_method; // GlogAppender.resetLogLevels()
+
+// Helper method to set a message into a member in the document
+void AddDocumentMember(const string& message, const char* member,
+    Document* document) {
+  Value output(message.c_str(), document->GetAllocator());
+  document->AddMember(member, output, document->GetAllocator());
+}
+
+template<class F>
+Webserver::UrlCallback MakeCallback(const F& fnc, bool display_log4j_handlers) {
+  return [fnc, display_log4j_handlers](const auto& args, auto* doc) {
+    // Display log4j log level handlers only when display_log4j_handlers is true.
+    if (display_log4j_handlers) AddDocumentMember("true", "include_log4j_handlers", doc);
+    (*fnc)(args, doc);
+  };
+}
+
+void InitDynamicLoggingSupport() {
+  JNIEnv* env = getJNIEnv();
+  log4j_logger_class_ = env->FindClass("org/apache/impala/util/GlogAppender");
+  EXIT_IF_EXC(env);
+  JniMethodDescriptor get_log_level_method_desc =
+      {"getLogLevel", "([B)Ljava/lang/String;", &get_log_level_method};
+  JniMethodDescriptor set_log_level_method_desc =
+      {"setLogLevel", "([B)Ljava/lang/String;", &set_log_level_method};
+  JniMethodDescriptor reset_log_level_method_desc =
+      {"resetLogLevels", "()V", &reset_log_levels_method};
+  ABORT_IF_ERROR(JniUtil::LoadStaticJniMethod(
+      env, log4j_logger_class_, &get_log_level_method_desc));
+  ABORT_IF_ERROR(JniUtil::LoadStaticJniMethod(
+      env, log4j_logger_class_, &set_log_level_method_desc));
+  ABORT_IF_ERROR(JniUtil::LoadStaticJniMethod(
+      env, log4j_logger_class_, &reset_log_level_method_desc));
+
+  FLAGS_v_original_value = FLAGS_v;
+  // Register a validator function for FLAGS_v to make sure it is in the [0-3]
+  // range. This is called everytime we try to override FLAGS_v using
+  // SetCommandLineOption().
+  google::RegisterFlagValidator(&FLAGS_v,
+      [](const char* flagname, int value) { return value >= 0 && value <= 3; });
+}
+
+// Helper method to get the log level of given Java class. It is a JNI wrapper around
+// GlogAppender.getLogLevel().
+Status GetJavaLogLevel(const TGetJavaLogLevelParams& params, string* result) {
+  RETURN_IF_ERROR(
+      JniUtil::CallJniMethod(log4j_logger_class_, get_log_level_method, params, result));
+  return Status::OK();
+}
+
+Status ResetJavaLogLevels() {
+  RETURN_IF_ERROR(
+      JniUtil::CallJniMethod(log4j_logger_class_, reset_log_levels_method));
+  return Status::OK();
+}
+
+// Callback handler for /get_java_loglevel.
+void GetJavaLogLevelCallback(const Webserver::ArgumentMap& args, Document* document) {
+  Webserver::ArgumentMap::const_iterator log_getclass = args.find("class");
+  if (log_getclass == args.end() || log_getclass->second.empty()) {
+    AddDocumentMember("Invalid input class name", "error", document);
+    return;
+  }
+  string result;
+  TGetJavaLogLevelParams params;
+  params.__set_class_name(log_getclass->second);
+  Status status = GetJavaLogLevel(params, &result);
+  if (!status.ok()) {
+    AddDocumentMember(status.GetDetail(), "error", document);
+    return;
+  }
+  if (result.empty()) {
+    AddDocumentMember("Invalid input class name", "error", document);
+    return;
+  }
+  AddDocumentMember(result, "get_java_loglevel_result", document);
+}
+
+// Callback handler for /set_java_loglevel.
+void SetJavaLogLevelCallback(const Webserver::ArgumentMap& args, Document* document) {
+  Webserver::ArgumentMap::const_iterator classname = args.find("class");
+  Webserver::ArgumentMap::const_iterator level = args.find("level");
+  if (classname == args.end() || classname->second.empty() ||
+      level == args.end() || level->second.empty()) {
+    AddDocumentMember("Invalid input parameters. Either class name or log level "
+        "is empty.", "error", document);
+    return;
+  }
+  TSetJavaLogLevelParams params;
+  string result;
+  params.__set_class_name(classname->second);
+  params.__set_log_level(level->second);
+  Status status =
+      JniUtil::CallJniMethod(log4j_logger_class_, set_log_level_method, params, &result);
+  if (!status.ok()) {
+    AddDocumentMember(status.GetDetail(), "error", document);
+    return;
+  }
+  if (result.empty()) {
+    AddDocumentMember("Invalid input parameters. Either class name or log level "
+        "is empty.", "error", document);
+    return;
+  }
+  AddDocumentMember(result, "set_java_loglevel_result", document);
+}
+
+// Callback handler for /reset_java_loglevel.
+void ResetJavaLogLevelCallback(const Webserver::ArgumentMap& args, Document* document) {
+  Status status = ResetJavaLogLevels();
+  if (!status.ok()) {
+    AddDocumentMember(status.GetDetail(), "error", document);
+    return;
+  }
+  AddDocumentMember("Java log levels reset.", "reset_java_loglevel_result", document);
+}
+
+// Callback handler for /set_glog_level
+void SetGlogLevelCallback(const Webserver::ArgumentMap& args, Document* document) {
+  Webserver::ArgumentMap::const_iterator glog_level = args.find("glog");
+  if (glog_level == args.end() || glog_level->second.empty()) {
+    AddDocumentMember("Bad glog level input. Valid inputs are integers in the "
+        "range [0-3].", "error", document);
+    return;
+  }
+  string new_log_level = google::SetCommandLineOption("v", glog_level->second.data());
+  if (new_log_level.empty()) {
+    AddDocumentMember("Bad glog level input. Valid inputs are integers in the "
+        "range [0-3].", "error", document);
+    return;
+  }
+  AddDocumentMember(new_log_level, "set_glog_level_result", document);
+  AddDocumentMember(to_string(FLAGS_v_original_value), "default_glog_level", document);
+}
+
+// Callback handler for /reset_glog_level
+void ResetGlogLevelCallback(const Webserver::ArgumentMap& args, Document* document) {
+  string new_log_level = google::SetCommandLineOption("v",
+      to_string(FLAGS_v_original_value).data());
+  AddDocumentMember(new_log_level, "reset_glog_level_result", document);
+}
+
+}
+
 namespace impala {
 
 void InitJvmLoggingSupport() {
@@ -85,6 +239,7 @@ void InitJvmLoggingSupport() {
   nm.fnPtr = reinterpret_cast<void*>(::Java_org_apache_impala_util_NativeLogger_Log);
   env->RegisterNatives(native_backend_cl, &nm, 1);
   EXIT_IF_EXC(env);
+  InitDynamicLoggingSupport();
 }
 
 TLogLevel::type FlagToTLogLevel(int flag) {
@@ -145,4 +300,21 @@ void LoggingSupport::DeleteOldLogs(const string& path_pattern, int max_log_files
   }
 }
 
+void RegisterLogLevelCallbacks(Webserver* webserver, bool register_log4j_handlers) {
+  webserver->RegisterUrlCallback("/log_level", "log_level.tmpl",
+      MakeCallback([](const Webserver::ArgumentMap& args, Document* document){},
+      register_log4j_handlers));
+  webserver->RegisterUrlCallback("/set_glog_level", "log_level.tmpl",
+      MakeCallback(&SetGlogLevelCallback, register_log4j_handlers), false);
+  webserver->RegisterUrlCallback("/reset_glog_level", "log_level.tmpl",
+      MakeCallback(&ResetGlogLevelCallback, register_log4j_handlers), false);
+  if (!register_log4j_handlers) return;
+  webserver->RegisterUrlCallback("/get_java_loglevel", "log_level.tmpl",
+      MakeCallback(&GetJavaLogLevelCallback, register_log4j_handlers), false);
+  webserver->RegisterUrlCallback("/set_java_loglevel", "log_level.tmpl",
+      MakeCallback(&SetJavaLogLevelCallback, register_log4j_handlers), false);
+  webserver->RegisterUrlCallback("/reset_java_loglevel", "log_level.tmpl",
+      MakeCallback(&ResetJavaLogLevelCallback, register_log4j_handlers), false);
+}
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/be/src/util/logging-support.h
----------------------------------------------------------------------
diff --git a/be/src/util/logging-support.h b/be/src/util/logging-support.h
index 4710f0c..87579af 100644
--- a/be/src/util/logging-support.h
+++ b/be/src/util/logging-support.h
@@ -23,14 +23,22 @@
 
 namespace impala {
 
-/// InitLoggingSupport registers the native logging functions with JNI. This allows
-/// the Java log4j log messages to be forwarded to Glog.
+class Webserver;
+
+/// Registers the required native logging functions with JNI. This allows the Java log4j
+/// log messages to be forwarded to Glog. Also loads the JNI helper methods to dynamically
+/// change these Java log levels.
 void InitJvmLoggingSupport();
 
 /// Helper function to convert a command line logging flag value (input as an int) to the
 /// matching TLogLevel enum value.
 TLogLevel::type FlagToTLogLevel(int flag);
 
+/// Registers the call back methods for handling dynamic log level changes. Since every
+/// daemon need not include an embedded jvm, dynamic log4j configuration is supported only
+/// when register_log4j_handlers is true.
+void RegisterLogLevelCallbacks(Webserver* webserver, bool register_log4j_handlers);
+
 class LoggingSupport {
  public:
   /// Helper function for log rotation that deletes all files matching the path pattern

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/common/thrift/Logging.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/Logging.thrift b/common/thrift/Logging.thrift
index cd3db98..13e8923 100644
--- a/common/thrift/Logging.thrift
+++ b/common/thrift/Logging.thrift
@@ -30,3 +30,16 @@ enum TLogLevel {
   ERROR,
   FATAL
 }
+
+// Helper structs for GetJavaLogLevel(), SetJavaLogLevel() methods.
+// These are used as input params to get/set the logging level of a
+// particular Java class at runtime using GlogAppender.getLogLevel()
+// and GlogAppender.setLogLevel() methods.
+struct TGetJavaLogLevelParams {
+  1: required string class_name
+}
+
+struct TSetJavaLogLevelParams {
+  1: required string class_name
+  2: required string log_level
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/fe/src/main/java/org/apache/impala/service/BackendConfig.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/BackendConfig.java b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
index 47c987e..5a4a440 100644
--- a/fe/src/main/java/org/apache/impala/service/BackendConfig.java
+++ b/fe/src/main/java/org/apache/impala/service/BackendConfig.java
@@ -55,6 +55,9 @@ public class BackendConfig {
   }
   public int getKuduClientTimeoutMs() { return backendCfg_.kudu_operation_timeout_ms; }
 
+  public int getImpalaLogLevel() { return backendCfg_.impala_log_lvl; }
+  public int getNonImpalaJavaVlogLevel() { return backendCfg_.non_impala_java_vlog; }
+
   // Inits the auth_to_local configuration in the static KerberosName class.
   private static void initAuthToLocal() {
     // If auth_to_local is enabled, we read the configuration hadoop.security.auth_to_local

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/fe/src/main/java/org/apache/impala/util/GlogAppender.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/GlogAppender.java b/fe/src/main/java/org/apache/impala/util/GlogAppender.java
index edefd0f..2eb33f3 100644
--- a/fe/src/main/java/org/apache/impala/util/GlogAppender.java
+++ b/fe/src/main/java/org/apache/impala/util/GlogAppender.java
@@ -22,19 +22,30 @@ import java.util.Properties;
 import org.apache.log4j.AppenderSkeleton;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
+import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
 import org.apache.log4j.spi.LoggingEvent;
 
 import org.apache.impala.common.InternalException;
+import org.apache.impala.common.ImpalaException;
+import org.apache.impala.common.JniUtil;
+import org.apache.impala.service.BackendConfig;
+import org.apache.impala.thrift.TGetJavaLogLevelParams;
+import org.apache.impala.thrift.TSetJavaLogLevelParams;
 import org.apache.impala.thrift.TLogLevel;
+import org.apache.thrift.protocol.TBinaryProtocol;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 
 /**
  * log4j appender which calls into C++ code to log messages at their correct severities
  * via glog.
  */
 public class GlogAppender extends AppenderSkeleton {
+  private final static TBinaryProtocol.Factory protocolFactory_ =
+      new TBinaryProtocol.Factory();
+
   // GLOG takes care of formatting, so we don't require a layout
   public boolean requiresLayout() { return false; }
 
@@ -123,7 +134,41 @@ public class GlogAppender extends AppenderSkeleton {
     properties.setProperty("log4j.logger.org.apache.impala",
         log4jLevelForTLogLevel(impalaLogLevel));
     PropertyConfigurator.configure(properties);
-    Logger.getLogger(GlogAppender.class).info(String.format("Logging initialized. " +
+    Logger.getLogger(GlogAppender.class).info(String.format("Logging (re)initialized. " +
         "Impala: %s, All other: %s", impalaLogLevel, otherLogLevel));
   }
+
+  /**
+   * Get the log4j log level corresponding to a serialized TGetJavaLogLevelParams.
+   */
+  public static String getLogLevel(byte[] serializedParams) throws ImpalaException {
+    TGetJavaLogLevelParams thriftParams = new TGetJavaLogLevelParams();
+    JniUtil.deserializeThrift(protocolFactory_, thriftParams, serializedParams);
+    String className = thriftParams.getClass_name();
+    if (Strings.isNullOrEmpty(className)) return null;
+    return Logger.getLogger(className).getEffectiveLevel().toString();
+  }
+
+  /**
+   * Sets the logging level of a class as per serialized TSetJavaLogLevelParams.
+   */
+  public static String setLogLevel(byte[] serializedParams) throws ImpalaException {
+    TSetJavaLogLevelParams thriftParams = new TSetJavaLogLevelParams();
+    JniUtil.deserializeThrift(protocolFactory_, thriftParams, serializedParams);
+    String className = thriftParams.getClass_name();
+    String logLevel = thriftParams.getLog_level();
+    if (Strings.isNullOrEmpty(className) || Strings.isNullOrEmpty(logLevel)) return null;
+    // Level.toLevel() returns DEBUG for an incorrect logLevel input.
+    Logger.getLogger(className).setLevel(Level.toLevel(logLevel));
+    return Logger.getLogger(className).getEffectiveLevel().toString();
+  }
+
+  /**
+   * Re-initializes the Java log4j logging levels.
+   */
+  public static void resetLogLevels() throws ImpalaException {
+    LogManager.resetConfiguration();
+    Install(TLogLevel.values()[BackendConfig.INSTANCE.getImpalaLogLevel()],
+        TLogLevel.values()[BackendConfig.INSTANCE.getNonImpalaJavaVlogLevel()]);
+  }
 };

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/tests/webserver/test_web_pages.py
----------------------------------------------------------------------
diff --git a/tests/webserver/test_web_pages.py b/tests/webserver/test_web_pages.py
index 0f993df..4604e22 100644
--- a/tests/webserver/test_web_pages.py
+++ b/tests/webserver/test_web_pages.py
@@ -20,6 +20,19 @@ from tests.common.impala_test_suite import ImpalaTestSuite
 import requests
 
 class TestWebPage(ImpalaTestSuite):
+
+  GET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/get_java_loglevel"
+  SET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/set_java_loglevel"
+  RESET_JAVA_LOGLEVEL_URL = "http://localhost:{0}/reset_java_loglevel"
+  SET_GLOG_LOGLEVEL_URL = "http://localhost:{0}/set_glog_level"
+  RESET_GLOG_LOGLEVEL_URL = "http://localhost:{0}/reset_glog_level"
+  # log4j changes do not apply to the statestore since it doesn't
+  # have an embedded JVM. So we make two sets of ports to test the
+  # log level endpoints, one without the statestore port and the
+  # one with it.
+  TEST_PORTS_WITHOUT_SS = ["25000", "25020"]
+  TEST_PORTS_WITH_SS = ["25000", "25010", "25020"]
+
   def test_memz(self):
     """test /memz at impalad / statestored / catalogd"""
 
@@ -39,4 +52,81 @@ class TestWebPage(ImpalaTestSuite):
     result = impalad.service.read_debug_webpage("query_profile_encoded?query_id=123")
     assert result.startswith("Could not obtain runtime profile: Query id")
 
+  def get_and_check_status(self, url, string_to_search = "", without_ss = True):
+    """Helper method that polls a given url and asserts the return code is ok and
+    the response contains the input string. 'without_ss', when true, excludes the
+    statestore endpoint of the url. Should be applied only for log4j logging changes."""
+    ports_to_test = self.TEST_PORTS_WITHOUT_SS if without_ss else self.TEST_PORTS_WITH_SS
+    for port in ports_to_test:
+      input_url = url.format(port)
+      response = requests.get(input_url)
+      assert response.status_code == requests.codes.ok\
+          and string_to_search in response.text, "Offending url: " + input_url
+
+  def test_log_level(self):
+    """Test that the /log_level page outputs are as expected and work well on basic and
+    malformed inputs. This however does not test that the log level changes are actually
+    in effect."""
+    # Check that the log_level end points are accessible.
+    self.get_and_check_status(self.GET_JAVA_LOGLEVEL_URL)
+    self.get_and_check_status(self.SET_JAVA_LOGLEVEL_URL)
+    self.get_and_check_status(self.RESET_JAVA_LOGLEVEL_URL)
+    self.get_and_check_status(self.SET_GLOG_LOGLEVEL_URL, without_ss=False)
+    self.get_and_check_status(self.RESET_GLOG_LOGLEVEL_URL, without_ss=False)
+    # Try getting log level of a class.
+    get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsTable")
+    self.get_and_check_status(get_loglevel_url, "DEBUG")
+
+    # Set the log level of a class to TRACE and confirm the setting is in place
+    set_loglevel_url = (self.SET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsTable&level=trace")
+    self.get_and_check_status(set_loglevel_url, "Effective log level: TRACE")
+
+    get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsTable")
+    self.get_and_check_status(get_loglevel_url, "TRACE")
+    # Check the log level of a different class and confirm it is still DEBUG
+    get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsPartition")
+    self.get_and_check_status(get_loglevel_url, "DEBUG")
+
+    # Reset Java logging levels and check the logging level of the class again
+    self.get_and_check_status(self.RESET_JAVA_LOGLEVEL_URL, "Java log levels reset.")
+    get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsTable")
+    self.get_and_check_status(get_loglevel_url, "DEBUG")
+
+    # Set a new glog level and make sure the setting has been applied.
+    set_glog_url = (self.SET_GLOG_LOGLEVEL_URL + "?glog=3")
+    self.get_and_check_status(set_glog_url, "v set to 3", False)
+
+    # Try resetting the glog logging defaults again.
+    self.get_and_check_status( self.RESET_GLOG_LOGLEVEL_URL, "v set to ", False)
+
+    # Try to get the log level of an empty class input
+    get_loglevel_url = (self.GET_JAVA_LOGLEVEL_URL + "?class=")
+    self.get_and_check_status(get_loglevel_url, without_ss=True)
+
+    # Same as above, for set log level request
+    set_loglevel_url = (self.SET_JAVA_LOGLEVEL_URL + "?class=")
+    self.get_and_check_status(get_loglevel_url, without_ss=True)
+
+    # Empty input for setting a glog level request
+    set_glog_url = (self.SET_GLOG_LOGLEVEL_URL + "?glog=")
+    self.get_and_check_status(set_glog_url, without_ss=False)
+
+    # Try setting a non-existent log level on a valid class. In such cases,
+    # log4j automatically sets it as DEBUG. This is the behavior of
+    # Level.toLevel() method.
+    set_loglevel_url = (self.SET_JAVA_LOGLEVEL_URL + "?class" +
+        "=org.apache.impala.catalog.HdfsTable&level=foo&")
+    self.get_and_check_status(set_loglevel_url, "Effective log level: DEBUG")
+
+    # Try setting an invalid glog level.
+    set_glog_url = self.SET_GLOG_LOGLEVEL_URL + "?glog=foo"
+    self.get_and_check_status(set_glog_url, "Bad glog level input", False)
 
+    # Try a non-existent endpoint on log_level URL.
+    bad_loglevel_url = self.SET_GLOG_LOGLEVEL_URL + "?badurl=foo"
+    self.get_and_check_status(bad_loglevel_url, without_ss=False)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/5d306eff/www/log_level.tmpl
----------------------------------------------------------------------
diff --git a/www/log_level.tmpl b/www/log_level.tmpl
new file mode 100644
index 0000000..b95da82
--- /dev/null
+++ b/www/log_level.tmpl
@@ -0,0 +1,97 @@
+<!--
+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}}
+<style type="text/css">
+  .log-level{
+  width: 50%;
+  line-height: 15px
+  }
+</style>
+
+{{?include_log4j_handlers}}
+<h2>Change Java log level (log4j)</h2>
+<div class="log-level">
+<form action="get_java_loglevel">
+  <div class="form-group">
+    <input type="text" class="form-control" name="class" placeholder="e.g. org.apache.impala.analysis.Analyzer">
+  </div>
+  <button type="submit" class="btn btn-primary btn-sm">Get Java Log Level</button>
+  <label>{{get_java_loglevel_result}}</label>
+</form>
+<br>
+<form action="set_java_loglevel">
+  <div class="form-group" name="level">
+    <input type="text" class="form-control" name="class" placeholder="e.g. org.apache.impala.analysis.Analyzer">
+    <br>
+    <div class="col-xs-20">
+      <label>Log level:</label>
+      <select name="level" class="selectpicker" data-style="btn-primary btn-sm">
+        <option value="all">ALL</option>
+        <option value="debug">DEBUG</option>
+        <option value="error">ERROR</option>
+        <option value="fatal">FATAL</option>
+        <option value="info">INFO</option>
+        <option value="off">OFF</option>
+        <option value="trace">TRACE</option>
+        <option value="warn">WARN</option>
+      </select>
+      <button type="submit" class="btn btn-primary btn-sm">Set Java Log Level</button>
+      {{?set_java_loglevel_result}}
+      <strong> Effective log level: {{set_java_loglevel_result}}</strong>
+      {{/set_java_loglevel_result}}
+    </div>
+  </div>
+</form>
+<form action="reset_java_loglevel">
+  <div class="col-xs-20">
+    <button type="submit" class="btn btn-primary btn-sm">Reset Java Log Levels</button>
+    <strong>{{reset_java_loglevel_result}}</strong>
+  </div>
+</form>
+{{/include_log4j_handlers}}
+
+<h2>Change backend logging level (glog)</h2>
+<form action="set_glog_level">
+  <div class="form-group" name="level">
+    <div class="col-xs-20">
+      <label>Log level:</label>
+      <select name="glog" class="selectpicker" data-style="btn-primary btn-sm">
+        <option value="0">0</option>
+        <option value="1">1</option>
+        <option value="2">2</option>
+        <option value="3">3</option>
+      </select>
+      <button type="submit" class="btn btn-primary btn-sm">Set Glog Level</button>
+      {{?set_glog_level_result}}
+      <strong>&nbsp &nbsp Glog level for --{{set_glog_level_result}}</strong>
+      <strong>&nbsp &nbsp Default glog level: {{default_glog_level}}</strong>
+      {{/set_glog_level_result}}
+    </div>
+  </div>
+</form>
+<form action="reset_glog_level">
+  <div class="col-xs-20">
+    <button type="submit" class="btn btn-primary btn-sm">Reset Glog Levels</button>
+    {{?reset_glog_level_result}}
+    <strong> &nbsp &nbsp Glog logging level for --{{reset_glog_level_result}}</strong>
+    {{/reset_glog_level_result}}
+  </div>
+</form>
+{{>www/common-footer.tmpl}}