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/02/15 18:54:55 UTC

[1/3] kudu git commit: KUDU-1877. Properly link to https:// URLs when SSL is enabled

Repository: kudu
Updated Branches:
  refs/heads/master 07c6efb58 -> 56402bd1a


KUDU-1877. Properly link to https:// URLs when SSL is enabled

This adds an https_enabled field to the registration protobuf, and makes
the web pages that generate links use https:// URLs when it is set.

Change-Id: I0ee1ac2c577c0a1a9a6b67f26c4515c3d1ce6dad
Reviewed-on: http://gerrit.cloudera.org:8080/6007
Tested-by: Kudu Jenkins
Reviewed-by: Adar Dembo <ad...@cloudera.com>


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

Branch: refs/heads/master
Commit: 91c93d317d359ebe48f7ecc71d7b9f6b6fd75faa
Parents: 07c6efb
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 14 15:47:44 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 15 18:11:14 2017 +0000

----------------------------------------------------------------------
 src/kudu/common/wire_protocol.proto             |  5 +++
 src/kudu/integration-tests/registration-test.cc | 42 ++++++++++++++++++--
 src/kudu/master/master-path-handlers.cc         | 10 +++--
 src/kudu/master/master.cc                       |  2 +
 src/kudu/tserver/heartbeater.cc                 |  1 +
 5 files changed, 54 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/91c93d31/src/kudu/common/wire_protocol.proto
----------------------------------------------------------------------
diff --git a/src/kudu/common/wire_protocol.proto b/src/kudu/common/wire_protocol.proto
index 93186d0..5203cc7 100644
--- a/src/kudu/common/wire_protocol.proto
+++ b/src/kudu/common/wire_protocol.proto
@@ -86,6 +86,11 @@ message ServerRegistrationPB {
   repeated HostPortPB rpc_addresses = 1;
   repeated HostPortPB http_addresses = 2;
   optional string software_version = 3;
+
+  // True if HTTPS has been enabled for the web interface.
+  // In this case, https:// URLs should be generated for the above
+  // 'http_addresses' field.
+  optional bool https_enabled = 4;
 }
 
 message ServerEntryPB {

http://git-wip-us.apache.org/repos/asf/kudu/blob/91c93d31/src/kudu/integration-tests/registration-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/registration-test.cc b/src/kudu/integration-tests/registration-test.cc
index 6603245..0b82ae6 100644
--- a/src/kudu/integration-tests/registration-test.cc
+++ b/src/kudu/integration-tests/registration-test.cc
@@ -24,12 +24,14 @@
 #include "kudu/common/wire_protocol-test-util.h"
 #include "kudu/fs/fs_manager.h"
 #include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/integration-tests/mini_cluster.h"
 #include "kudu/master/master-test-util.h"
 #include "kudu/master/master.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/mini_master.h"
 #include "kudu/master/ts_descriptor.h"
+#include "kudu/security/test/test_certs.h"
 #include "kudu/security/tls_context.h"
 #include "kudu/tserver/mini_tablet_server.h"
 #include "kudu/tserver/tablet_server.h"
@@ -46,6 +48,7 @@ namespace kudu {
 
 using std::vector;
 using std::shared_ptr;
+using std::string;
 using master::MiniMaster;
 using master::TSDescriptor;
 using master::TabletLocationsPB;
@@ -73,21 +76,25 @@ class RegistrationTest : public KuduTest {
     cluster_->Shutdown();
   }
 
-  void CheckTabletServersPage() {
+  void CheckTabletServersPage(string* contents = nullptr) {
     EasyCurl c;
     faststring buf;
     string addr = cluster_->mini_master()->bound_http_addr().ToString();
     ASSERT_OK(c.FetchURL(strings::Substitute("http://$0/tablet-servers", addr),
                                 &buf));
+    string buf_str = buf.ToString();
 
     // Should include the TS UUID
     string expected_uuid =
       cluster_->mini_tablet_server(0)->server()->instance_pb().permanent_uuid();
-    ASSERT_STR_CONTAINS(buf.ToString(), expected_uuid);
+    ASSERT_STR_CONTAINS(buf_str, expected_uuid);
 
     // Should check that the TS software version is included on the page.
     // tserver version should be the same as returned by GetShortVersionString()
-    ASSERT_STR_CONTAINS(buf.ToString(), VersionInfo::GetShortVersionString());
+    ASSERT_STR_CONTAINS(buf_str, VersionInfo::GetShortVersionString());
+    if (contents != nullptr) {
+      *contents = std::move(buf_str);
+    }
   }
 
 
@@ -208,4 +215,33 @@ TEST_F(RegistrationTest, TestTSGetsSignedX509Certificate) {
     }, MonoDelta::FromSeconds(10));
 }
 
+// Test that, if the tserver has HTTPS enabled, the master links to it
+// via https:// URLs and not http://.
+TEST_F(RegistrationTest, TestExposeHttpsURLs) {
+  MiniTabletServer* ts = cluster_->mini_tablet_server(0);
+  string password;
+  WebserverOptions* opts = &ts->options()->webserver_opts;
+  ASSERT_OK(security::CreateTestSSLCerts(GetTestDataDirectory(),
+                                         &opts->certificate_file,
+                                         &opts->private_key_file,
+                                         &password));
+  opts->private_key_password_cmd = strings::Substitute("echo $0", password);
+  ts->Shutdown();
+  ASSERT_OK(ts->Start());
+
+  // The URL displayed on the page uses a hostname. Rather than
+  // dealing with figuring out what the hostname should be, just
+  // use a more permissive regex which doesn't check the host.
+  string expected_url_regex = strings::Substitute(
+      "https://[a-zA-Z0-9.-]+:$0/", opts->port);
+
+  // Need "eventually" here because the tserver may take a few seconds
+  // to re-register while starting up.
+  AssertEventually([&](){
+      string contents;
+      NO_FATALS(CheckTabletServersPage(&contents));
+      ASSERT_STR_MATCHES(contents, expected_url_regex);
+    });
+}
+
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/91c93d31/src/kudu/master/master-path-handlers.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master-path-handlers.cc b/src/kudu/master/master-path-handlers.cc
index 113b85d..240335d 100644
--- a/src/kudu/master/master-path-handlers.cc
+++ b/src/kudu/master/master-path-handlers.cc
@@ -539,7 +539,9 @@ void MasterPathHandlers::HandleDumpEntities(const Webserver::WebRequest& req,
     jw.String("http_addrs");
     jw.StartArray();
     for (const HostPortPB& host_port : reg.http_addresses()) {
-      jw.String(Substitute("http://$0:$1", host_port.host(), host_port.port()));
+      jw.String(Substitute("$0://$1:$2",
+                           reg.https_enabled() ? "https" : "http",
+                           host_port.host(), host_port.port()));
     }
     jw.EndArray();
 
@@ -586,7 +588,8 @@ string MasterPathHandlers::TSDescriptorToHtml(const TSDescriptor& desc,
   desc.GetRegistration(&reg);
 
   if (reg.http_addresses().size() > 0) {
-    return Substitute("<a href=\"http://$0:$1/tablet?id=$2\">$3:$4</a>",
+    return Substitute("<a href=\"$0://$1:$2/tablet?id=$3\">$4:$5</a>",
+                      reg.https_enabled() ? "https" : "http",
                       reg.http_addresses(0).host(),
                       reg.http_addresses(0).port(),
                       EscapeForHtmlToString(tablet_id),
@@ -602,7 +605,8 @@ string MasterPathHandlers::RegistrationToHtml(
     const std::string& link_text) const {
   string link_html = EscapeForHtmlToString(link_text);
   if (reg.http_addresses().size() > 0) {
-    link_html = Substitute("<a href=\"http://$0:$1/\">$2</a>",
+    link_html = Substitute("<a href=\"$0://$1:$2/\">$3</a>",
+                           reg.https_enabled() ? "https" : "http",
                            reg.http_addresses(0).host(),
                            reg.http_addresses(0).port(), link_html);
   }

http://git-wip-us.apache.org/repos/asf/kudu/blob/91c93d31/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index 79051c3..341531b 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -223,6 +223,8 @@ Status Master::InitMasterRegistration() {
   RETURN_NOT_OK(AddHostPortPBs(http_addrs, reg.mutable_http_addresses()));
   reg.set_software_version(VersionInfo::GetShortVersionString());
 
+  reg.set_https_enabled(web_server()->IsSecure());
+
   registration_.Swap(&reg);
   registration_initialized_.store(true);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/91c93d31/src/kudu/tserver/heartbeater.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/heartbeater.cc b/src/kudu/tserver/heartbeater.cc
index d95c836..f0438ad 100644
--- a/src/kudu/tserver/heartbeater.cc
+++ b/src/kudu/tserver/heartbeater.cc
@@ -311,6 +311,7 @@ Status Heartbeater::Thread::SetupRegistration(ServerRegistrationPB* reg) {
                         "Failed to add HTTP addresses to registration");
   reg->set_software_version(VersionInfo::GetShortVersionString());
 
+  reg->set_https_enabled(server_->web_server()->IsSecure());
   return Status::OK();
 }
 


[2/3] kudu git commit: tls_socket: fix handling of syscall errors

Posted by to...@apache.org.
tls_socket: fix handling of syscall errors

This prevents a case where we'd see an error like 'TlsSocket::Recv: Success'
when trying to read from a TLS socket which got abruptly shut down by the
remote side.

Change-Id: I9a0a63f861d71bd3186567ff98148476795530ab
Reviewed-on: http://gerrit.cloudera.org:8080/5954
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/4a5ab736
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/4a5ab736
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/4a5ab736

Branch: refs/heads/master
Commit: 4a5ab7368c9c3be39779d44f6bf701b1a925ce09
Parents: 91c93d3
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Feb 9 00:30:08 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 15 18:24:51 2017 +0000

----------------------------------------------------------------------
 src/kudu/security/tls_socket.cc | 29 ++++++++++++++++++++++++++---
 1 file changed, 26 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/4a5ab736/src/kudu/security/tls_socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/tls_socket.cc b/src/kudu/security/tls_socket.cc
index b510906..77f633f 100644
--- a/src/kudu/security/tls_socket.cc
+++ b/src/kudu/security/tls_socket.cc
@@ -24,6 +24,7 @@
 #include "kudu/gutil/basictypes.h"
 #include "kudu/security/cert.h"
 #include "kudu/security/openssl_util.h"
+#include "kudu/util/errno.h"
 
 namespace kudu {
 namespace security {
@@ -58,7 +59,8 @@ Status TlsSocket::Write(const uint8_t *buf, int32_t amt, int32_t *nwritten) {
       *nwritten = 0;
       return Status::OK();
     }
-    return Status::NetworkError("TlsSocket::Write", GetSSLErrorDescription(error_code));
+    return Status::NetworkError("failed to write to TLS socket",
+                                GetSSLErrorDescription(error_code));
   }
   *nwritten = bytes_written;
   return Status::OK();
@@ -84,12 +86,16 @@ Status TlsSocket::Writev(const struct ::iovec *iov, int iov_len, int32_t *nwritt
 }
 
 Status TlsSocket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
+  const char* kErrString = "failed to read from TLS socket";
+
   CHECK(ssl_);
   ERR_clear_error();
+  errno = 0;
   int32_t bytes_read = SSL_read(ssl_.get(), buf, amt);
+  int save_errno = errno;
   if (bytes_read <= 0) {
     if (bytes_read == 0 && SSL_get_shutdown(ssl_.get()) == SSL_RECEIVED_SHUTDOWN) {
-      return Status::NetworkError("TlsSocket::Recv", "received remote shutdown", ESHUTDOWN);
+      return Status::NetworkError(kErrString, ErrnoToString(ESHUTDOWN), ESHUTDOWN);
     }
     auto error_code = SSL_get_error(ssl_.get(), bytes_read);
     if (error_code == SSL_ERROR_WANT_READ) {
@@ -97,7 +103,24 @@ Status TlsSocket::Recv(uint8_t *buf, int32_t amt, int32_t *nread) {
       *nread = 0;
       return Status::OK();
     }
-    return Status::NetworkError("TlsSocket::Recv", GetSSLErrorDescription(error_code));
+    if (error_code == SSL_ERROR_SYSCALL && ERR_peek_error() == 0) {
+      // From the OpenSSL docs:
+      //   Some I/O error occurred.  The OpenSSL error queue may contain more
+      //   information on the error.  If the error queue is empty (i.e.
+      //   ERR_get_error() returns 0), ret can be used to find out more about
+      //   the error: If ret == 0, an EOF was observed that violates the pro-
+      //   tocol.  If ret == -1, the underlying BIO reported an I/O error (for
+      //   socket I/O on Unix systems, consult errno for details).
+      if (bytes_read == 0) {
+        // "EOF was observed that violates the protocol" (eg the other end disconnected)
+        return Status::NetworkError(kErrString, ErrnoToString(ECONNRESET), ECONNRESET);
+      }
+      if (bytes_read == -1 && save_errno != 0) {
+        return Status::NetworkError(kErrString, ErrnoToString(save_errno), save_errno);
+      }
+      return Status::NetworkError(kErrString, "unknown ERROR_SYSCALL");
+    }
+    return Status::NetworkError(kErrString, GetSSLErrorDescription(error_code));
   }
   *nread = bytes_read;
   return Status::OK();


[3/3] kudu git commit: Supress compile-time '((unused)) attribute ignored' warning

Posted by to...@apache.org.
Supress compile-time '((unused)) attribute ignored' warning

On Linux systems, we still use the user_signal_handler_thread_running_
variable, causing the compiler to complain that we've marked it
incorrectly as ATTRIBUTE_UNUSED.

It is only unused on MacOS, so it can be #ifndef'd out.

Change-Id: Ic579b8e66ec883f79facb666bfab0b98785679eb
Reviewed-on: http://gerrit.cloudera.org:8080/6011
Reviewed-by: Todd Lipcon <to...@apache.org>
Reviewed-by: Mike Percy <mp...@apache.org>
Tested-by: Kudu Jenkins


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

Branch: refs/heads/master
Commit: 56402bd1a48efaed486c2f92f5897096bce2ddbb
Parents: 4a5ab73
Author: Sailesh Mukil <sa...@apache.org>
Authored: Tue Feb 14 17:56:39 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Wed Feb 15 18:36:05 2017 +0000

----------------------------------------------------------------------
 src/kudu/util/minidump.h | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/56402bd1/src/kudu/util/minidump.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/minidump.h b/src/kudu/util/minidump.h
index 58702d7..459639b 100644
--- a/src/kudu/util/minidump.h
+++ b/src/kudu/util/minidump.h
@@ -82,7 +82,10 @@ class MinidumpExceptionHandler {
   // at a time, as a sanity check.
   static std::atomic<int> current_num_instances_;
 
-  std::atomic<bool> user_signal_handler_thread_running_ ATTRIBUTE_UNUSED; // Unused in macOS build.
+  #ifndef __APPLE__
+  std::atomic<bool> user_signal_handler_thread_running_;// Unused in macOS build.
+  #endif
+
   scoped_refptr<Thread> user_signal_handler_thread_;
 
   // Breakpad ExceptionHandler. It registers its own signal handlers to write