You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2017/03/03 01:25:40 UTC

[5/7] kudu git commit: server: offer a flag to completely disable the web server

server: offer a flag to completely disable the web server

This adds a new --webserver_enabled flag, which can be used to
completely disable the embedded web server. When disabled, the web
server does not listen at all.

The patch required a bit of cleanup to other spots in the code that
required all servers to have associated HTTP addresses. A new unit test
smoke tests a cluster configured in this manner.

Change-Id: I97dacfff075f21cde9248a11fcf19efefb030aec
Reviewed-on: http://gerrit.cloudera.org:8080/6216
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/kudu/repo
Commit: http://git-wip-us.apache.org/repos/asf/kudu/commit/9cb5b24f
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/9cb5b24f
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/9cb5b24f

Branch: refs/heads/master
Commit: 9cb5b24f7f957a3ecca1ae1bf034d3ad9defa03a
Parents: f6a1a60
Author: Todd Lipcon <to...@apache.org>
Authored: Wed Mar 1 15:29:00 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Mar 3 00:09:10 2017 +0000

----------------------------------------------------------------------
 .../integration-tests/external_mini_cluster.cc  | 17 ++++++---
 .../integration-tests/external_mini_cluster.h   |  4 +++
 src/kudu/integration-tests/security-itest.cc    | 38 ++++++++++++++------
 src/kudu/master/master.cc                       | 16 +++++----
 src/kudu/master/ts_descriptor.cc                | 10 +++---
 src/kudu/server/server_base.cc                  | 31 +++++++++++-----
 src/kudu/tserver/heartbeater.cc                 | 12 ++++---
 src/kudu/tserver/tablet_server.cc               |  4 ++-
 src/kudu/util/net/net_util.h                    |  4 +++
 src/kudu/util/thread.cc                         |  8 +++--
 src/kudu/util/thread.h                          |  2 +-
 11 files changed, 100 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/integration-tests/external_mini_cluster.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.cc b/src/kudu/integration-tests/external_mini_cluster.cc
index 5ead31b..a57f92a 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -915,7 +915,9 @@ Sockaddr ExternalDaemon::bound_rpc_addr() const {
 
 HostPort ExternalDaemon::bound_http_hostport() const {
   CHECK(status_);
-  CHECK_GE(status_->bound_http_addresses_size(), 1);
+  if (status_->bound_http_addresses_size() == 0) {
+    return HostPort();
+  }
   HostPort ret;
   CHECK_OK(HostPortFromPB(status_->bound_http_addresses(0), &ret));
   return ret;
@@ -936,6 +938,7 @@ Status ExternalDaemon::GetInt64Metric(const MetricEntityPrototype* entity_proto,
                                       const MetricPrototype* metric_proto,
                                       const char* value_field,
                                       int64_t* value) const {
+  CHECK(bound_http_hostport().Initialized());
   // Fetch metrics whose name matches the given prototype.
   string url = Substitute(
       "http://$0/jsonmetricz?metrics=$1",
@@ -1034,7 +1037,9 @@ Status ExternalMaster::Restart() {
   }
 
   vector<string> flags(GetCommonFlags());
-  flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
+  if (bound_http_.Initialized()) {
+    flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
+  }
   flags.push_back("--rpc_bind_addresses=" + bound_rpc_.ToString());
 
   return StartProcess(flags);
@@ -1141,9 +1146,11 @@ Status ExternalTabletServer::Restart() {
   flags.push_back("--rpc_bind_addresses=" + bound_rpc_.ToString());
   flags.push_back(Substitute("--local_ip_for_outbound_sockets=$0",
                              get_rpc_bind_address()));
-  flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
-  flags.push_back(Substitute("--webserver_interface=$0",
-                             bound_http_.host()));
+  if (bound_http_.Initialized()) {
+    flags.push_back(Substitute("--webserver_port=$0", bound_http_.port()));
+    flags.push_back(Substitute("--webserver_interface=$0",
+                               bound_http_.host()));
+  }
   flags.push_back("--tserver_master_addrs=" + master_addrs_);
   RETURN_NOT_OK(StartProcess(flags));
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/integration-tests/external_mini_cluster.h
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster.h b/src/kudu/integration-tests/external_mini_cluster.h
index c4518c3..57fdba1 100644
--- a/src/kudu/integration-tests/external_mini_cluster.h
+++ b/src/kudu/integration-tests/external_mini_cluster.h
@@ -345,7 +345,11 @@ class ExternalDaemon : public RefCountedThreadSafe<ExternalDaemon> {
 
   HostPort bound_rpc_hostport() const;
   Sockaddr bound_rpc_addr() const;
+
+  // Return the host/port that this daemon is bound to for HTTP.
+  // May return an uninitialized HostPort if HTTP is disabled.
   HostPort bound_http_hostport() const;
+
   const NodeInstancePB& instance_id() const;
   const std::string& uuid() const;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/integration-tests/security-itest.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index abfdaf5..cdac377 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -39,11 +39,12 @@ namespace kudu {
 
 class SecurityITest : public KuduTest {
  public:
+  SecurityITest() {
+    cluster_opts_.enable_kerberos = true;
+    cluster_opts_.num_tablet_servers = 3;
+  }
   void StartCluster() {
-    ExternalMiniClusterOptions opts;
-    opts.enable_kerberos = true;
-    opts.num_tablet_servers = 3;
-    cluster_.reset(new ExternalMiniCluster(opts));
+    cluster_.reset(new ExternalMiniCluster(cluster_opts_));
     ASSERT_OK(cluster_->Start());
   }
 
@@ -63,6 +64,9 @@ class SecurityITest : public KuduTest {
     return proxy.SetFlag(req, &resp, &controller);
   }
 
+  // Create a table, insert a row, scan it back, and delete the table.
+  void SmokeTestCluster();
+
   Status TryRegisterAsTS() {
     // Make a new messenger so that we don't reuse any cached connections from
     // the minicluster startup sequence.
@@ -89,16 +93,12 @@ class SecurityITest : public KuduTest {
   }
 
  protected:
+  ExternalMiniClusterOptions cluster_opts_;
   unique_ptr<ExternalMiniCluster> cluster_;
 };
 
-// Test creating a table, writing some data, reading data, and dropping
-// the table.
-TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
+void SecurityITest::SmokeTestCluster() {
   const char* kTableName = "test-table";
-  StartCluster();
-
-  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
   client::sp::shared_ptr<KuduClient> client;
   ASSERT_OK(cluster_->CreateClient(nullptr, &client));
 
@@ -127,6 +127,15 @@ TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
 
   // Delete the table.
   ASSERT_OK(client->DeleteTable(kTableName));
+}
+
+// Test creating a table, writing some data, reading data, and dropping
+// the table.
+TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
+  StartCluster();
+
+  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
+  NO_FATALS(SmokeTestCluster());
 
   // Non-superuser clients should not be able to set flags.
   Status s = TrySetFlagOnTS();
@@ -188,5 +197,14 @@ TEST_F(SecurityITest, TestAuthorizedSuperuser) {
 
 }
 
+// Test that the web UIs can be entirely disabled, for users who feel they
+// are a security risk.
+TEST_F(SecurityITest, TestDisableWebUI) {
+  cluster_opts_.extra_master_flags.push_back("--webserver_enabled=0");
+  cluster_opts_.extra_tserver_flags.push_back("--webserver_enabled=0");
+  StartCluster();
+  NO_FATALS(SmokeTestCluster());
+}
+
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index eda44c7..1d1d78a 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -112,7 +112,9 @@ Status Master::Init() {
 
   RETURN_NOT_OK(ServerBase::Init());
 
-  RETURN_NOT_OK(path_handlers_->Register(web_server_.get()));
+  if (web_server_) {
+    RETURN_NOT_OK(path_handlers_->Register(web_server_.get()));
+  }
 
   // The certificate authority object is initialized upon loading
   // CA private key and certificate from the system table when the server
@@ -233,12 +235,14 @@ Status Master::InitMasterRegistration() {
   RETURN_NOT_OK_PREPEND(rpc_server()->GetBoundAddresses(&rpc_addrs),
                         "Couldn't get RPC addresses");
   RETURN_NOT_OK(AddHostPortPBs(rpc_addrs, reg.mutable_rpc_addresses()));
-  vector<Sockaddr> http_addrs;
-  web_server()->GetBoundAddresses(&http_addrs);
-  RETURN_NOT_OK(AddHostPortPBs(http_addrs, reg.mutable_http_addresses()));
-  reg.set_software_version(VersionInfo::GetShortVersionString());
 
-  reg.set_https_enabled(web_server()->IsSecure());
+  if (web_server()) {
+    vector<Sockaddr> http_addrs;
+    web_server()->GetBoundAddresses(&http_addrs);
+    RETURN_NOT_OK(AddHostPortPBs(http_addrs, reg.mutable_http_addresses()));
+    reg.set_https_enabled(web_server()->IsSecure());
+  }
+  reg.set_software_version(VersionInfo::GetShortVersionString());
 
   registration_.Swap(&reg);
   registration_initialized_.store(true);

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/master/ts_descriptor.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/ts_descriptor.cc b/src/kudu/master/ts_descriptor.cc
index 26228fa..6f93bb3 100644
--- a/src/kudu/master/ts_descriptor.cc
+++ b/src/kudu/master/ts_descriptor.cc
@@ -90,11 +90,10 @@ Status TSDescriptor::Register(const NodeInstancePB& instance,
   std::lock_guard<simple_spinlock> l(lock_);
   CHECK_EQ(instance.permanent_uuid(), permanent_uuid_);
 
-  // TODO(KUDU-418): we don't currently support changing IPs or hosts since the
+  // TODO(KUDU-418): we don't currently support changing RPC addresses since the
   // host/port is stored persistently in each tablet's metadata.
   if (registration_ &&
-      (!HostPortPBsEqual(registration_->rpc_addresses(), registration.rpc_addresses()) ||
-       !HostPortPBsEqual(registration_->http_addresses(), registration.http_addresses()))) {
+      !HostPortPBsEqual(registration_->rpc_addresses(), registration.rpc_addresses())) {
     string msg = strings::Substitute(
         "Tablet server $0 is attempting to re-register with a different host/port. "
         "This is not currently supported. Old: {$1} New: {$2}",
@@ -105,10 +104,9 @@ Status TSDescriptor::Register(const NodeInstancePB& instance,
     return Status::InvalidArgument(msg);
   }
 
-  if (registration.rpc_addresses().empty() ||
-      registration.http_addresses().empty()) {
+  if (registration.rpc_addresses().empty()) {
     return Status::InvalidArgument(
-        "invalid registration: must have at least one RPC and one HTTP address",
+        "invalid registration: must have at least one RPC address",
         SecureShortDebugString(registration));
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index b67b7f7..772207d 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -76,6 +76,11 @@ TAG_FLAG(min_negotiation_threads, advanced);
 DEFINE_int32(max_negotiation_threads, 50, "Maximum number of connection negotiation threads.");
 TAG_FLAG(max_negotiation_threads, advanced);
 
+DEFINE_bool(webserver_enabled, true, "Whether to enable the web server on this daemon. "
+            "NOTE: disabling the web server is also likely to prevent monitoring systems "
+            "from properly capturing metrics.");
+TAG_FLAG(webserver_enabled, advanced);
+
 DEFINE_string(superuser_acl, "",
               "The list of usernames to allow as super users, comma-separated. "
               "A '*' entry indicates that all authenticated users are allowed. "
@@ -129,7 +134,6 @@ ServerBase::ServerBase(string name, const ServerBaseOptions& options,
       metric_entity_(METRIC_ENTITY_server.Instantiate(metric_registry_.get(),
                                                       metric_namespace)),
       rpc_server_(new RpcServer(options.rpc_opts)),
-      web_server_(new Webserver(options.webserver_opts)),
       result_tracker_(new rpc::ResultTracker(shared_ptr<MemTracker>(
           MemTracker::CreateTracker(-1, "result-tracker", mem_tracker_)))),
       is_first_run_(false),
@@ -148,6 +152,10 @@ ServerBase::ServerBase(string name, const ServerBaseOptions& options,
     clock_ = LogicalClock::CreateStartingAt(Timestamp::kInitialTimestamp);
   }
 
+  if (FLAGS_webserver_enabled) {
+    web_server_.reset(new Webserver(options.webserver_opts));
+  }
+
   CHECK_OK(StartThreadInstrumentation(metric_entity_, web_server_.get()));
   CHECK_OK(codegen::CompilationManager::GetSingleton()->StartInstrumentation(
                metric_entity_));
@@ -166,6 +174,7 @@ Sockaddr ServerBase::first_rpc_address() const {
 }
 
 Sockaddr ServerBase::first_http_address() const {
+  CHECK(web_server_);
   vector<Sockaddr> addrs;
   WARN_NOT_OK(web_server_->GetBoundAddresses(&addrs),
               "Couldn't get bound webserver addresses");
@@ -287,7 +296,7 @@ void ServerBase::GetStatusPB(ServerStatusPB* status) const {
   }
 
   // HTTP ports
-  {
+  if (web_server_) {
     vector<Sockaddr> addrs;
     CHECK_OK(web_server_->GetBoundAddresses(&addrs));
     for (const Sockaddr& addr : addrs) {
@@ -443,12 +452,14 @@ Status ServerBase::Start() {
 
   RETURN_NOT_OK(rpc_server_->Start());
 
-  AddDefaultPathHandlers(web_server_.get());
-  AddRpczPathHandlers(messenger_, web_server_.get());
-  RegisterMetricsJsonHandler(web_server_.get(), metric_registry_.get());
-  TracingPathHandlers::RegisterHandlers(web_server_.get());
-  web_server_->set_footer_html(FooterHtml());
-  RETURN_NOT_OK(web_server_->Start());
+  if (web_server_) {
+    AddDefaultPathHandlers(web_server_.get());
+    AddRpczPathHandlers(messenger_, web_server_.get());
+    RegisterMetricsJsonHandler(web_server_.get(), metric_registry_.get());
+    TracingPathHandlers::RegisterHandlers(web_server_.get());
+    web_server_->set_footer_html(FooterHtml());
+    RETURN_NOT_OK(web_server_->Start());
+  }
 
   if (!options_.dump_info_path.empty()) {
     RETURN_NOT_OK_PREPEND(DumpServerInfo(options_.dump_info_path, options_.dump_info_format),
@@ -466,7 +477,9 @@ void ServerBase::Shutdown() {
   if (excess_log_deleter_thread_) {
     excess_log_deleter_thread_->Join();
   }
-  web_server_->Stop();
+  if (web_server_) {
+    web_server_->Stop();
+  }
   rpc_server_->Shutdown();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/tserver/heartbeater.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index 892cf84..093399e 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -306,13 +306,15 @@ Status Heartbeater::Thread::SetupRegistration(ServerRegistrationPB* reg) {
                         "Failed to add RPC addresses to registration");
 
   addrs.clear();
-  RETURN_NOT_OK_PREPEND(CHECK_NOTNULL(server_->web_server())->GetBoundAddresses(&addrs),
-                        "Unable to get bound HTTP addresses");
-  RETURN_NOT_OK_PREPEND(AddHostPortPBs(addrs, reg->mutable_http_addresses()),
-                        "Failed to add HTTP addresses to registration");
+  if (server_->web_server()) {
+    RETURN_NOT_OK_PREPEND(server_->web_server()->GetBoundAddresses(&addrs),
+                          "Unable to get bound HTTP addresses");
+    RETURN_NOT_OK_PREPEND(AddHostPortPBs(addrs, reg->mutable_http_addresses()),
+                          "Failed to add HTTP addresses to registration");
+    reg->set_https_enabled(server_->web_server()->IsSecure());
+  }
   reg->set_software_version(VersionInfo::GetShortVersionString());
 
-  reg->set_https_enabled(server_->web_server()->IsSecure());
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/tserver/tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server.cc b/src/kudu/tserver/tablet_server.cc
index ea8e1b1..27a0c1d 100644
--- a/src/kudu/tserver/tablet_server.cc
+++ b/src/kudu/tserver/tablet_server.cc
@@ -88,7 +88,9 @@ Status TabletServer::Init() {
   RETURN_NOT_OK(ValidateMasterAddressResolution());
 
   RETURN_NOT_OK(ServerBase::Init());
-  RETURN_NOT_OK(path_handlers_->Register(web_server_.get()));
+  if (web_server_) {
+    RETURN_NOT_OK(path_handlers_->Register(web_server_.get()));
+  }
 
   heartbeater_.reset(new Heartbeater(opts_, this));
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/util/net/net_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/net_util.h b/src/kudu/util/net/net_util.h
index 3086f4a..863d0f3 100644
--- a/src/kudu/util/net/net_util.h
+++ b/src/kudu/util/net/net_util.h
@@ -33,6 +33,10 @@ class HostPort {
   HostPort(std::string host, uint16_t port);
   explicit HostPort(const Sockaddr& addr);
 
+  bool Initialized() const {
+    return !host_.empty();
+  }
+
   // Parse a "host:port" pair into this object.
   // If there is no port specified in the string, then 'default_port' is used.
   Status ParseString(const std::string& str, uint16_t default_port);

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/util/thread.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/thread.cc b/src/kudu/util/thread.cc
index 971568e..71aa400 100644
--- a/src/kudu/util/thread.cc
+++ b/src/kudu/util/thread.cc
@@ -268,9 +268,11 @@ Status ThreadMgr::StartInstrumentation(const scoped_refptr<MetricEntity>& metric
       METRIC_involuntary_context_switches.InstantiateFunctionGauge(metrics,
         Bind(&GetInVoluntaryContextSwitches)));
 
-  WebCallbackRegistry::PathHandlerCallback thread_callback =
-      bind<void>(mem_fn(&ThreadMgr::ThreadPathHandler), this, _1, _2);
-  DCHECK_NOTNULL(web)->RegisterPathHandler("/threadz", "Threads", thread_callback);
+  if (web) {
+    WebCallbackRegistry::PathHandlerCallback thread_callback =
+        bind<void>(mem_fn(&ThreadMgr::ThreadPathHandler), this, _1, _2);
+    DCHECK_NOTNULL(web)->RegisterPathHandler("/threadz", "Threads", thread_callback);
+  }
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9cb5b24f/src/kudu/util/thread.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/thread.h b/src/kudu/util/thread.h
index 5aa7620..46e2505 100644
--- a/src/kudu/util/thread.h
+++ b/src/kudu/util/thread.h
@@ -354,7 +354,7 @@ class Thread : public RefCountedThreadSafe<Thread> {
 };
 
 // Registers /threadz with the debug webserver, and creates thread-tracking metrics under
-// the given entity.
+// the given entity. If 'web' is NULL, does not register the path handler.
 Status StartThreadInstrumentation(const scoped_refptr<MetricEntity>& server_metrics,
                                   WebCallbackRegistry* web);
 } // namespace kudu