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

[impala] 01/02: IMPALA-10010: Add option to configure metrics webserver

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

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

commit 34c2be0c3847103a6a7f0a7d66de8add285abc66
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Mon Jul 27 11:25:42 2020 -0700

    IMPALA-10010: Add option to configure metrics webserver
    
    Currently, when security is turned on for the webui, eg. with
    --webserver_require_ldap, authentication is applied to all webui
    endpoints.
    
    However, there are some endpoints that expose low-sensitivity info and
    which are scraped by other systems that it may be difficult to get
    credentials to in order to be able to authenticate.
    
    This patch adds a flag, --metrics_webserver_port, which if specified
    turns on an unsecured webserver that exposes only the /metrics,
    /jsonmetrics, /metrics_prometheus for all Impala daemons. Impalads
    also have the /healthz endpoint exposed.
    
    Testing:
    - Added a test that turns on the metrics server and verifies its
      reachable and unsecured.
    
    Change-Id: Ibcf297d798a1a5c9cd59d4d82706d2d945e10d3d
    Reviewed-on: http://gerrit.cloudera.org:8080/16270
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/catalog/catalogd-main.cc                    | 12 ++++-
 be/src/runtime/exec-env.cc                         | 22 +++++++++-
 be/src/runtime/exec-env.h                          |  2 +
 be/src/service/impala-http-handler.cc              |  4 +-
 be/src/service/impala-http-handler.h               |  5 ++-
 be/src/service/impala-server.cc                    |  4 ++
 be/src/statestore/statestored-main.cc              | 13 +++++-
 be/src/util/metrics.cc                             |  2 +-
 be/src/util/metrics.h                              |  2 +-
 be/src/util/webserver.cc                           | 51 ++++++++++++++--------
 be/src/util/webserver.h                            | 10 ++++-
 .../impala/customcluster/CustomClusterRunner.java  | 18 +++++---
 .../impala/customcluster/LdapWebserverTest.java    | 43 ++++++++++++++++--
 .../test/java/org/apache/impala/util/Metrics.java  | 16 +++++--
 14 files changed, 161 insertions(+), 43 deletions(-)

diff --git a/be/src/catalog/catalogd-main.cc b/be/src/catalog/catalogd-main.cc
index 8479326..d3bc474 100644
--- a/be/src/catalog/catalogd-main.cc
+++ b/be/src/catalog/catalogd-main.cc
@@ -49,6 +49,8 @@ DECLARE_string(ssl_private_key);
 DECLARE_string(ssl_private_key_password_cmd);
 DECLARE_string(ssl_cipher_list);
 DECLARE_string(ssl_minimum_version);
+DECLARE_string(metrics_webserver_interface);
+DECLARE_int32(metrics_webserver_port);
 
 #include "common/names.h"
 
@@ -67,12 +69,20 @@ int CatalogdMain(int argc, char** argv) {
 
   if (FLAGS_enable_webserver) {
     AddDefaultUrlCallbacks(webserver.get(), metrics.get());
+    ABORT_IF_ERROR(metrics->RegisterHttpHandlers(webserver.get()));
     ABORT_IF_ERROR(webserver->Start());
   } else {
     LOG(INFO) << "Not starting webserver";
   }
 
-  ABORT_IF_ERROR(metrics->Init(FLAGS_enable_webserver ? webserver.get() : nullptr));
+  scoped_ptr<Webserver> metrics_webserver;
+  if (FLAGS_metrics_webserver_port > 0) {
+    metrics_webserver.reset(new Webserver(FLAGS_metrics_webserver_interface,
+        FLAGS_metrics_webserver_port, metrics.get(), Webserver::AuthMode::NONE));
+    ABORT_IF_ERROR(metrics->RegisterHttpHandlers(metrics_webserver.get()));
+    ABORT_IF_ERROR(metrics_webserver->Start());
+  }
+
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), true, nullptr, nullptr));
   ABORT_IF_ERROR(StartMemoryMaintenanceThread());
   ABORT_IF_ERROR(StartThreadInstrumentation(metrics.get(), webserver.get(), true));
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 347f2d7..063622c 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -153,6 +153,14 @@ DEFINE_int32(catalog_client_rpc_retry_interval_ms, 3000, "(Advanced) The time to
     "before retrying when the catalog RPC client fails to connect to catalogd or when "
     "RPCs to the catalogd fail.");
 
+DEFINE_int32(metrics_webserver_port, 0,
+    "If non-zero, the port to run the metrics webserver on, which exposes the /metrics, "
+    "/jsonmetrics, /metrics_prometheus, and /healthz endpoints without authentication "
+    "enabled.");
+
+DEFINE_string(metrics_webserver_interface, "",
+    "Interface to start metrics webserver on. If blank, webserver binds to 0.0.0.0");
+
 const static string DEFAULT_FS = "fs.defaultFS";
 
 // The multiplier for how many queries a dedicated coordinator can run compared to an
@@ -284,6 +292,12 @@ ExecEnv::ExecEnv(int backend_port, int krpc_port,
   admission_controller_.reset(
       new AdmissionController(cluster_membership_mgr_.get(), statestore_subscriber_.get(),
           request_pool_service_.get(), metrics_.get(), configured_backend_address_));
+
+  if (FLAGS_metrics_webserver_port > 0) {
+    metrics_webserver_.reset(new Webserver(FLAGS_metrics_webserver_interface,
+        FLAGS_metrics_webserver_port, metrics_.get(), Webserver::AuthMode::NONE));
+  }
+
   exec_env_ = this;
 }
 
@@ -364,7 +378,13 @@ Status ExecEnv::Init() {
 
   InitSystemStateInfo();
 
-  RETURN_IF_ERROR(metrics_->Init(enable_webserver_ ? webserver_.get() : nullptr));
+  if (enable_webserver_) {
+    RETURN_IF_ERROR(metrics_->RegisterHttpHandlers(webserver_.get()));
+  }
+  if (FLAGS_metrics_webserver_port > 0) {
+    RETURN_IF_ERROR(metrics_->RegisterHttpHandlers(metrics_webserver_.get()));
+    RETURN_IF_ERROR(metrics_webserver_->Start());
+  }
   catalogd_client_cache_->InitMetrics(metrics_.get(), "catalog.server");
   RETURN_IF_ERROR(RegisterMemoryMetrics(
       metrics_.get(), true, buffer_reservation_.get(), buffer_pool_.get()));
diff --git a/be/src/runtime/exec-env.h b/be/src/runtime/exec-env.h
index 0299c89..d0f9125 100644
--- a/be/src/runtime/exec-env.h
+++ b/be/src/runtime/exec-env.h
@@ -124,6 +124,7 @@ class ExecEnv {
   HBaseTableFactory* htable_factory() { return htable_factory_.get(); }
   io::DiskIoMgr* disk_io_mgr() { return disk_io_mgr_.get(); }
   Webserver* webserver() { return webserver_.get(); }
+  Webserver* metrics_webserver() { return metrics_webserver_.get(); }
   MetricGroup* metrics() { return metrics_.get(); }
   MetricGroup* rpc_metrics() { return rpc_metrics_; }
   MemTracker* process_mem_tracker() { return mem_tracker_.get(); }
@@ -188,6 +189,7 @@ class ExecEnv {
   boost::scoped_ptr<HBaseTableFactory> htable_factory_;
   boost::scoped_ptr<io::DiskIoMgr> disk_io_mgr_;
   boost::scoped_ptr<Webserver> webserver_;
+  boost::scoped_ptr<Webserver> metrics_webserver_;
   boost::scoped_ptr<MemTracker> mem_tracker_;
   boost::scoped_ptr<PoolMemTrackerRegistry> pool_mem_trackers_;
   boost::scoped_ptr<ThreadResourceMgr> thread_mgr_;
diff --git a/be/src/service/impala-http-handler.cc b/be/src/service/impala-http-handler.cc
index 1f2338b..de6b3f5 100644
--- a/be/src/service/impala-http-handler.cc
+++ b/be/src/service/impala-http-handler.cc
@@ -91,7 +91,7 @@ static Status ParseIdFromRequest(const Webserver::WebRequest& req, TUniqueId* id
 
 }
 
-void ImpalaHttpHandler::RegisterHandlers(Webserver* webserver) {
+void ImpalaHttpHandler::RegisterHandlers(Webserver* webserver, bool metrics_only) {
   DCHECK(webserver != NULL);
 
   Webserver::RawUrlCallback healthz_callback =
@@ -100,6 +100,8 @@ void ImpalaHttpHandler::RegisterHandlers(Webserver* webserver) {
     };
   webserver->RegisterUrlCallback("/healthz", healthz_callback);
 
+  if (metrics_only) return;
+
   webserver->RegisterUrlCallback("/backends", "backends.tmpl",
       MakeCallback(this, &ImpalaHttpHandler::BackendsHandler), true);
 
diff --git a/be/src/service/impala-http-handler.h b/be/src/service/impala-http-handler.h
index 8ecb7a1..a491f11 100644
--- a/be/src/service/impala-http-handler.h
+++ b/be/src/service/impala-http-handler.h
@@ -35,8 +35,9 @@ class ImpalaHttpHandler {
  public:
   ImpalaHttpHandler(ImpalaServer* server) : server_(server) { }
 
-  /// Registers all the per-Impalad webserver callbacks
-  void RegisterHandlers(Webserver* webserver);
+  /// Registers per-Impalad webserver callbacks. If 'metrics_only' is true, only registers
+  /// the callbacks needed by the metrics server, i.e. /healthz.
+  void RegisterHandlers(Webserver* webserver, bool metrics_only = false);
 
  private:
   ImpalaServer* server_;
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 3ab48cf..f1017ea 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -2525,6 +2525,10 @@ Status ImpalaServer::Start(int32_t thrift_be_port, int32_t beeswax_port, int32_t
   // ExecEnv singleton and will receive a nullptr.
   http_handler_.reset(new ImpalaHttpHandler(this));
   http_handler_->RegisterHandlers(exec_env_->webserver());
+  if (exec_env_->metrics_webserver() != nullptr) {
+    http_handler_->RegisterHandlers(
+        exec_env_->metrics_webserver(), /* metrics_only */ true);
+  }
 
   if (!FLAGS_is_coordinator && !FLAGS_is_executor) {
     return Status("Impala does not have a valid role configured. "
diff --git a/be/src/statestore/statestored-main.cc b/be/src/statestore/statestored-main.cc
index 205447c..b738aa3 100644
--- a/be/src/statestore/statestored-main.cc
+++ b/be/src/statestore/statestored-main.cc
@@ -37,6 +37,8 @@
 DECLARE_int32(state_store_port);
 DECLARE_int32(webserver_port);
 DECLARE_bool(enable_webserver);
+DECLARE_string(metrics_webserver_interface);
+DECLARE_int32(metrics_webserver_port);
 
 #include "common/names.h"
 
@@ -54,13 +56,20 @@ int StatestoredMain(int argc, char** argv) {
 
   if (FLAGS_enable_webserver) {
     AddDefaultUrlCallbacks(webserver.get(), metrics.get());
+    ABORT_IF_ERROR(metrics->RegisterHttpHandlers(webserver.get()));
     ABORT_IF_ERROR(webserver->Start());
   } else {
     LOG(INFO) << "Not starting webserver";
   }
 
-  ABORT_IF_ERROR(
-      metrics->Init(FLAGS_enable_webserver ? webserver.get() : nullptr));
+  scoped_ptr<Webserver> metrics_webserver;
+  if (FLAGS_metrics_webserver_port > 0) {
+    metrics_webserver.reset(new Webserver(FLAGS_metrics_webserver_interface,
+        FLAGS_metrics_webserver_port, metrics.get(), Webserver::AuthMode::NONE));
+    ABORT_IF_ERROR(metrics->RegisterHttpHandlers(metrics_webserver.get()));
+    ABORT_IF_ERROR(metrics_webserver->Start());
+  }
+
   ABORT_IF_ERROR(RegisterMemoryMetrics(metrics.get(), false, nullptr, nullptr));
   ABORT_IF_ERROR(StartMemoryMaintenanceThread());
   ABORT_IF_ERROR(
diff --git a/be/src/util/metrics.cc b/be/src/util/metrics.cc
index 5182543..d967a8a 100644
--- a/be/src/util/metrics.cc
+++ b/be/src/util/metrics.cc
@@ -137,7 +137,7 @@ TMetricDef MetricDefs::Get(const string& key, const string& arg) {
 MetricGroup::MetricGroup(const string& name)
     : obj_pool_(new ObjectPool()), name_(name) { }
 
-Status MetricGroup::Init(Webserver* webserver) {
+Status MetricGroup::RegisterHttpHandlers(Webserver* webserver) {
   if (webserver != NULL) {
     Webserver::UrlCallback default_callback =
         bind<void>(mem_fn(&MetricGroup::CMCompatibleCallback), this, _1, _2);
diff --git a/be/src/util/metrics.h b/be/src/util/metrics.h
index e6971e8..0970caf 100644
--- a/be/src/util/metrics.h
+++ b/be/src/util/metrics.h
@@ -423,7 +423,7 @@ class MetricGroup {
 
   /// Register page callbacks with the webserver. Only the root of any metric group
   /// hierarchy needs to do this.
-  Status Init(Webserver* webserver);
+  Status RegisterHttpHandlers(Webserver* webserver);
 
   /// Converts this metric group (and optionally all of its children recursively) to JSON.
   void ToJson(bool include_children, rapidjson::Document* document,
diff --git a/be/src/util/webserver.cc b/be/src/util/webserver.cc
index a9ac1eb..8986dba 100644
--- a/be/src/util/webserver.cc
+++ b/be/src/util/webserver.cc
@@ -107,7 +107,8 @@ DEFINE_string(webserver_authentication_domain, "",
     "Domain used for debug webserver authentication");
 DEFINE_string(webserver_password_file, "",
     "(Optional) Location of .htpasswd file containing user names and hashed passwords for"
-    " debug webserver authentication");
+    " debug webserver authentication. Cannot be used with --webserver_require_ldap or "
+    "--webserver_require_spnego.");
 
 DEFINE_string(webserver_x_frame_options, "DENY",
     "webserver will add X-Frame-Options HTTP header with this value");
@@ -117,11 +118,12 @@ DEFINE_int32(webserver_max_post_length_bytes, 1024 * 1024,
 
 DEFINE_bool(webserver_require_spnego, false,
     "Require connections to the web server to authenticate via Kerberos using SPNEGO. "
-    "Cannot be used with --webserver_require_ldap.");
+    "Cannot be used with --webserver_require_ldap or --webserver_password_file.");
 
 DEFINE_bool(webserver_require_ldap, false,
     "Require connections to the web server to authenticate via LDAP using HTTP Basic "
-    "authentication. Cannot be used with --webserver_require_spnego.");
+    "authentication. Cannot be used with --webserver_require_spnego or "
+    "--webserver_password_file.");
 DEFINE_string(webserver_ldap_user_filter, "",
     "Comma separated list of usernames. If specified, users must be on this list for "
     "LDAP athentication to the webserver to succeed.");
@@ -258,29 +260,32 @@ kudu::Status RunSpnegoStep(
 } // anonymous namespace
 
 Webserver::Webserver(MetricGroup* metrics)
-  : Webserver(FLAGS_webserver_interface, FLAGS_webserver_port, metrics) {}
+  : Webserver(FLAGS_webserver_interface, FLAGS_webserver_port, metrics,
+        GetConfiguredAuthMode()) {}
 
-Webserver::Webserver(const string& interface, const int port, MetricGroup* metrics)
-  : context_(nullptr),
+Webserver::Webserver(const string& interface, const int port, MetricGroup* metrics,
+    const AuthMode& auth_mode)
+  : auth_mode_(auth_mode),
+    context_(nullptr),
     error_handler_(UrlHandler(
         bind<void>(&Webserver::ErrorHandler, this, _1, _2), "error.tmpl", false)),
     use_cookies_(FLAGS_max_cookie_lifetime_s > 0) {
   http_address_ = MakeNetworkAddress(interface.empty() ? "0.0.0.0" : interface, port);
   Init();
 
-  if (FLAGS_webserver_require_spnego) {
+  if (auth_mode_ == AuthMode::SPNEGO) {
     total_negotiate_auth_success_ =
         metrics->AddCounter("impala.webserver.total-negotiate-auth-success", 0);
     total_negotiate_auth_failure_ =
         metrics->AddCounter("impala.webserver.total-negotiate-auth-failure", 0);
   }
-  if (FLAGS_webserver_require_ldap) {
+  if (auth_mode_ == AuthMode::LDAP) {
     total_basic_auth_success_ =
         metrics->AddCounter("impala.webserver.total-basic-auth-success", 0);
     total_basic_auth_failure_ =
         metrics->AddCounter("impala.webserver.total-basic-auth-failure", 0);
   }
-  if (use_cookies_ && (FLAGS_webserver_require_spnego || FLAGS_webserver_require_ldap)) {
+  if (use_cookies_ && (auth_mode_ == AuthMode::SPNEGO || auth_mode_ == AuthMode::LDAP)) {
     total_cookie_auth_success_ =
         metrics->AddCounter("impala.webserver.total-cookie-auth-success", 0);
     total_cookie_auth_failure_ =
@@ -396,12 +401,7 @@ Status Webserver::Start() {
     options.push_back(FLAGS_webserver_password_file.c_str());
   }
 
-  if (FLAGS_webserver_require_spnego && FLAGS_webserver_require_ldap) {
-    return Status("Securing the web server with both Kerberos and LDAP is not "
-                  "currently supported.");
-  }
-
-  if (FLAGS_webserver_require_spnego) {
+  if (auth_mode_ == AuthMode::SPNEGO) {
     // If Kerberos has been configured, security::InitKerberosForServer() will
     // already have been called, ensuring that the keytab path has been
     // propagated into this environment variable where the GSSAPI calls will
@@ -412,9 +412,10 @@ Status Webserver::Start() {
       return Status("Unable to configure web server for SPNEGO authentication: "
                     "must configure a keytab file for the server");
     }
+    LOG(INFO) << "Webserver: secured with SPNEGO authentication.";
   }
 
-  if (FLAGS_webserver_require_ldap) {
+  if (auth_mode_ == AuthMode::LDAP) {
     if (!FLAGS_enable_ldap_auth) {
       return Status("Unable to secure web server with LDAP: LDAP authentication must be "
                     "configured for this daemon.");
@@ -427,6 +428,7 @@ Status Webserver::Start() {
     ldap_.reset(new ImpalaLdap());
     RETURN_IF_ERROR(
         ldap_->Init(FLAGS_webserver_ldap_user_filter, FLAGS_webserver_ldap_group_filter));
+    LOG(INFO) << "Webserver: secured with LDAP authentication.";
   }
 
   options.push_back("listening_ports");
@@ -572,7 +574,7 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
   }
 
   vector<string> response_headers;
-  bool authenticated = !FLAGS_webserver_require_spnego && !FLAGS_webserver_require_ldap;
+  bool authenticated = auth_mode_ != AuthMode::SPNEGO && auth_mode_ != AuthMode::LDAP;
   // Try authenticating with a cookie first, if enabled.
   if (!authenticated && use_cookies_) {
     const char* cookie_header = sq_get_header(connection, "Cookie");
@@ -593,7 +595,7 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
   }
 
   if (!authenticated) {
-    if (FLAGS_webserver_require_spnego) {
+    if (auth_mode_ == AuthMode::SPNEGO) {
       sq_callback_result_t spnego_result =
           HandleSpnego(connection, request_info, &response_headers);
       if (spnego_result == SQ_CONTINUE_HANDLING) {
@@ -604,7 +606,7 @@ sq_callback_result_t Webserver::BeginRequestCallback(struct sq_connection* conne
         return spnego_result;
       }
     } else {
-      DCHECK(FLAGS_webserver_require_ldap);
+      DCHECK(auth_mode_ == AuthMode::LDAP);
       Status basic_status = HandleBasic(connection, request_info, &response_headers);
       if (basic_status.ok()) {
         // Basic auth was successful.
@@ -886,4 +888,15 @@ const string Webserver::GetMimeType(const ContentType& content_type) {
       return "";
   }
 }
+
+Webserver::AuthMode Webserver::GetConfiguredAuthMode() {
+  if (!FLAGS_webserver_password_file.empty()) {
+    return AuthMode::HTPASSWD;
+  } else if (FLAGS_webserver_require_spnego) {
+    return AuthMode::SPNEGO;
+  } else if (FLAGS_webserver_require_ldap) {
+    return AuthMode::LDAP;
+  }
+  return AuthMode::NONE;
+}
 }
diff --git a/be/src/util/webserver.h b/be/src/util/webserver.h
index feed82b..9af0245 100644
--- a/be/src/util/webserver.h
+++ b/be/src/util/webserver.h
@@ -83,9 +83,12 @@ class Webserver {
   /// as pretty printed JSON plain text.
   static const char* ENABLE_PLAIN_JSON_KEY;
 
+  enum AuthMode { NONE, HTPASSWD, SPNEGO, LDAP };
+
   /// Using this constructor, the webserver will bind to 'interface', or all available
   /// interfaces if not specified.
-  Webserver(const std::string& interface, const int port, MetricGroup* metrics);
+  Webserver(const std::string& interface, const int port, MetricGroup* metrics,
+      const AuthMode& auth_mode = GetConfiguredAuthMode());
 
   /// Uses FLAGS_webserver_{port, interface}
   Webserver(MetricGroup* metrics);
@@ -127,6 +130,9 @@ class Webserver {
   /// Returns the appropriate MIME type for a given ContentType.
   static const std::string GetMimeType(const ContentType& content_type);
 
+  /// Returns the authentication mode configured by the startup flags.
+  static AuthMode GetConfiguredAuthMode();
+
  private:
   /// Contains all information relevant to rendering one Url. Each Url has one callback
   /// that produces the output to render. The callback either produces a Json document
@@ -238,6 +244,8 @@ class Webserver {
   /// The resolved hostname from 'http_address_'.
   std::string hostname_;
 
+  AuthMode auth_mode_;
+
   /// Handle to Squeasel context; owned and freed by Squeasel internally
   struct sq_context* context_;
 
diff --git a/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
index 8e58394..f8ddb5c 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
@@ -43,14 +43,20 @@ class CustomClusterRunner {
     return StartImpalaCluster(args, new HashMap<String, String>());
   }
 
-  /**
-   * Starts Impala, setting environment variables in 'env', and passes 'args' to the
-   * impalads, catalogd, and statestored.
-   */
   public static int StartImpalaCluster(String args, Map<String, String> env)
       throws IOException, InterruptedException {
-    ProcessBuilder pb = new ProcessBuilder(new String[] {"start-impala-cluster.py",
-        "--impalad_args", args, "--catalogd_args", args, "--state_store_args", args});
+    return StartImpalaCluster(args, env, "");
+  }
+
+  /**
+   * Starts Impala, setting environment variables in 'env', and passing 'args' to the
+   * impalads, catalogd, and statestored, and 'startArgs' to start-impala-cluster.py.
+   */
+  public static int StartImpalaCluster(String args, Map<String, String> env,
+      String startArgs) throws IOException, InterruptedException {
+    ProcessBuilder pb =
+        new ProcessBuilder(new String[] {"start-impala-cluster.py", "--impalad_args",
+            args, "--catalogd_args", args, "--state_store_args", args, startArgs});
     pb.redirectErrorStream(true);
     Map<String, String> origEnv = pb.environment();
     origEnv.putAll(env);
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
index e23189b..aa49b54 100644
--- a/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapWebserverTest.java
@@ -19,6 +19,7 @@ package org.apache.impala.customcluster;
 
 import static org.apache.impala.testutil.LdapUtil.*;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -54,7 +55,7 @@ public class LdapWebserverTest {
 
   Metrics metrics_ = new Metrics(TEST_USER_1, TEST_PASSWORD_1);
 
-  public void setUp(String extraArgs) throws Exception {
+  public void setUp(String extraArgs, String startArgs) throws Exception {
     String uri =
         String.format("ldap://localhost:%s", serverRule.getLdapServer().getPort());
     String dn = "cn=#UID,ou=Users,dc=myorg,dc=com";
@@ -65,7 +66,7 @@ public class LdapWebserverTest {
     Map<String, String> env = new HashMap<>();
     env.put("IMPALA_WEBSERVER_USERNAME", TEST_USER_1);
     env.put("IMPALA_WEBSERVER_PASSWORD", TEST_PASSWORD_1);
-    int ret = CustomClusterRunner.StartImpalaCluster(impalaArgs, env);
+    int ret = CustomClusterRunner.StartImpalaCluster(impalaArgs, env, startArgs);
     assertEquals(ret, 0);
   }
 
@@ -98,7 +99,7 @@ public class LdapWebserverTest {
 
   @Test
   public void testWebserver() throws Exception {
-    setUp("");
+    setUp("", "");
     // start-impala-cluster contacts the webui to confirm the impalads have started, so
     // there will already be some successful auth attempts.
     verifyMetrics(Range.atLeast(1L), zero, Range.atLeast(1L), zero);
@@ -130,7 +131,7 @@ public class LdapWebserverTest {
             + "--ldap_group_dn_pattern=ou=Groups,dc=myorg,dc=com "
             + "--ldap_group_membership_key=uniqueMember "
             + "--ldap_group_class_key=groupOfUniqueNames",
-        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3));
+        TEST_USER_GROUP, TEST_USER_1, TEST_USER_3), "");
     // start-impala-cluster contacts the webui to confirm the impalads have started, so
     // there will already be some successful auth attempts.
     verifyMetrics(Range.atLeast(1L), zero, Range.atLeast(1L), zero);
@@ -158,4 +159,38 @@ public class LdapWebserverTest {
     // Check that there is now three unsuccessful auth attempts.
     verifyMetrics(Range.atLeast(1L), Range.closed(3L, 3L), Range.atLeast(1L), zero);
   }
+
+  /**
+   * Tests that the metrics webserver servers the correct endpoints and without security
+   * even if LDAP auth is turned on for the regular webserver.
+   */
+  @Test
+  public void testMetricsWebserver() throws Exception {
+    // Use 'per_impalad_args' to turn the metrics webserver on only for the first impalad.
+    setUp("", "--per_impalad_args=--metrics_webserver_port=25030");
+    // Attempt to access the regular webserver without a username/password, should fail.
+    Metrics noUsername = new Metrics();
+    String result = noUsername.readContent("/");
+    assertTrue(result, result.contains("Must authenticate with Basic authentication."));
+
+    // Attempt to access the regular webserver with invalid username/password.
+    Metrics invalidUserPass = new Metrics("invalid", "invalid");
+    result = invalidUserPass.readContent("/");
+    assertTrue(result, result.contains("Must authenticate with Basic authentication."));
+
+    // Attempt to access the metrics webserver without a username/password.
+    Metrics noUsernameMetrics = new Metrics(25030);
+    // Should succeed for the metrics endpoints.
+    for (String endpoint :
+        new String[] {"/metrics", "/jsonmetrics", "/metrics_prometheus", "/healthz"}) {
+      result = noUsernameMetrics.readContent(endpoint);
+      assertFalse(
+          result, result.contains("Must authenticate with Basic authentication."));
+    }
+
+    for (String endpoint : new String[] {"/varz", "/backends"}) {
+      result = noUsernameMetrics.readContent(endpoint);
+      assertTrue(result, result.contains("No URI handler for"));
+    }
+  }
 }
diff --git a/fe/src/test/java/org/apache/impala/util/Metrics.java b/fe/src/test/java/org/apache/impala/util/Metrics.java
index 75a20f2..8b8ea92 100644
--- a/fe/src/test/java/org/apache/impala/util/Metrics.java
+++ b/fe/src/test/java/org/apache/impala/util/Metrics.java
@@ -41,18 +41,26 @@ import org.json.simple.parser.JSONParser;
  */
 public class Metrics {
   private final static String WEBSERVER_HOST = "localhost";
-  private final static int WEBSERVER_PORT = 25000;
+  private final static int DEFAULT_WEBSERVER_PORT = 25000;
   private final static String JSON_METRICS = "/jsonmetrics?json";
 
   private CloseableHttpClient httpClient_;
+  private int port_;
   private String username_;
   private String password_;
 
-  public Metrics() { this("", ""); }
+  public Metrics() { this("", "", DEFAULT_WEBSERVER_PORT); }
+
+  public Metrics(int port) { this("", "", port); }
 
   public Metrics(String username, String password) {
+    this(username, password, DEFAULT_WEBSERVER_PORT);
+  }
+
+  public Metrics(String username, String password, int port) {
     this.username_ = username;
     this.password_ = password;
+    this.port_ = port;
     httpClient_ = HttpClients.createDefault();
   }
 
@@ -78,11 +86,11 @@ public class Metrics {
    * Retrieves the page at 'path' and returns its contents.
    */
   public String readContent(String path) throws IOException {
-    HttpHost targetHost = new HttpHost(WEBSERVER_HOST, WEBSERVER_PORT, "http");
+    HttpHost targetHost = new HttpHost(WEBSERVER_HOST, port_, "http");
     HttpClientContext context = HttpClientContext.create();
     if (!username_.equals("")) {
       CredentialsProvider credsProvider = new BasicCredentialsProvider();
-      credsProvider.setCredentials(new AuthScope(WEBSERVER_HOST, WEBSERVER_PORT),
+      credsProvider.setCredentials(new AuthScope(WEBSERVER_HOST, port_),
           new UsernamePasswordCredentials(username_, password_));
       AuthCache authCache = new BasicAuthCache();
       authCache.put(targetHost, new BasicScheme());