You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by gr...@apache.org on 2019/06/20 12:46:32 UTC

[kudu] branch branch-1.10.x updated (0a45b4d -> 86a0dc2)

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

granthenke pushed a change to branch branch-1.10.x
in repository https://gitbox.apache.org/repos/asf/kudu.git.


    from 0a45b4d  [sentry] add require_db_privileges flag for ListTables
     new 2d61c8f  KUDU-2870: allow super-user to skip authz checks in Checksum
     new 86a0dc2  KUDU-2871 (part 1): disable TLS 1.3.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 src/kudu/integration-tests/security-itest.cc | 76 ++++++++++++++++++++++++----
 src/kudu/rpc/client_negotiation.cc           |  4 +-
 src/kudu/security/tls_context.cc             |  7 +++
 src/kudu/server/server_base.cc               |  7 ++-
 src/kudu/server/server_base.h                |  3 ++
 src/kudu/tools/kudu-tool-test.cc             | 30 +++++++++++
 src/kudu/tserver/tablet_service.cc           |  7 ++-
 7 files changed, 119 insertions(+), 15 deletions(-)


[kudu] 01/02: KUDU-2870: allow super-user to skip authz checks in Checksum

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

granthenke pushed a commit to branch branch-1.10.x
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 2d61c8faf4350bed5612a1828e82d6a011ef0d40
Author: Andrew Wong <aw...@apache.org>
AuthorDate: Wed Jun 19 14:05:59 2019 -0700

    KUDU-2870: allow super-user to skip authz checks in Checksum
    
    In order to allow for the Kudu CLI to run a checksum scan (a process
    which currently doesn't fetch an authz token from the Master), this
    patch allows Checksum to proceed if the requesting user is a super-user.
    
    Testing:
    - A test is added to run the CLI against a tserver that enforces
      fine-grained access control.
    - A new test is added to test the super-user permissions for the
      Checksum endpoint.
    
    Change-Id: I9da21f41702da747a081ab037d75865748d981a8
    Reviewed-on: http://gerrit.cloudera.org:8080/13681
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
    (cherry picked from commit 2c0161db1117d27bb4c2961901dfc54b76437dc9)
    Reviewed-on: http://gerrit.cloudera.org:8080/13686
    Tested-by: Grant Henke <gr...@apache.org>
---
 src/kudu/integration-tests/security-itest.cc | 76 ++++++++++++++++++++++++----
 src/kudu/server/server_base.cc               |  7 ++-
 src/kudu/server/server_base.h                |  3 ++
 src/kudu/tools/kudu-tool-test.cc             | 30 +++++++++++
 src/kudu/tserver/tablet_service.cc           |  7 ++-
 5 files changed, 109 insertions(+), 14 deletions(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 92d7322..1003461 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -34,10 +34,11 @@
 #include "kudu/client/shared_ptr.h"
 #include "kudu/client/write_op.h"
 #include "kudu/common/partial_row.h"
+#include "kudu/common/schema.h"
+#include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/consensus/consensus.pb.h"
 #include "kudu/consensus/consensus.proxy.h"
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/master/master.pb.h"
 #include "kudu/master/master.proxy.h"
@@ -50,7 +51,9 @@
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tablet/key_value_test_schema.h"
+#include "kudu/tablet/tablet.pb.h"
 #include "kudu/tserver/tserver.pb.h"
+#include "kudu/tserver/tserver_service.pb.h"
 #include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/env.h"
 #include "kudu/util/monotime.h"
@@ -82,6 +85,10 @@ using strings::Substitute;
 
 namespace kudu {
 
+static const char* kTableName = "test-table";
+static const Schema kTestSchema = CreateKeyValueTestSchema();
+static const KuduSchema kTestKuduSchema = client::KuduSchema::FromSchema(kTestSchema);
+
 class SecurityITest : public KuduTest {
  public:
   SecurityITest() {
@@ -111,6 +118,15 @@ class SecurityITest : public KuduTest {
     return proxy.SetFlag(req, &resp, &controller);
   }
 
+  Status CreateTestTable(const client::sp::shared_ptr<KuduClient>& client) {
+    unique_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
+    return table_creator->table_name(kTableName)
+        .set_range_partition_columns({ "key" })
+        .schema(&kTestKuduSchema)
+        .num_replicas(3)
+        .Create();
+  }
+
   // Create a table, insert a row, scan it back, and delete the table.
   void SmokeTestCluster();
 
@@ -129,7 +145,7 @@ class SecurityITest : public KuduTest {
     return proxy.TSHeartbeat(req, &resp, &rpc);
   }
 
-  Status TryListTablets() {
+  Status TryListTablets(vector<string>* tablet_ids = nullptr) {
     auto messenger = NewMessengerOrDie();
     const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
     tserver::TabletServerServiceProxy proxy(messenger, addr, addr.host());
@@ -137,7 +153,28 @@ class SecurityITest : public KuduTest {
     rpc::RpcController rpc;
     tserver::ListTabletsRequestPB req;
     tserver::ListTabletsResponsePB resp;
-    return proxy.ListTablets(req, &resp, &rpc);
+    RETURN_NOT_OK(proxy.ListTablets(req, &resp, &rpc));
+    if (tablet_ids) {
+      for (int i = 0; i < resp.status_and_schema_size(); i++) {
+        tablet_ids->emplace_back(resp.status_and_schema(i).tablet_status().tablet_id());
+      }
+    }
+    return Status::OK();
+  }
+
+  // Sends a request to checksum the given tablet without an authz token.
+  Status TryChecksumWithoutAuthzToken(const string& tablet_id) {
+    auto messenger = NewMessengerOrDie();
+    const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
+    tserver::TabletServerServiceProxy proxy(messenger, addr, addr.host());
+
+    rpc::RpcController rpc;
+    tserver::ChecksumRequestPB req;
+    tserver::NewScanRequestPB* scan = req.mutable_new_request();
+    scan->set_tablet_id(tablet_id);
+    RETURN_NOT_OK(SchemaToColumnPBs(kTestSchema, scan->mutable_projected_columns()));
+    tserver::ChecksumResponsePB resp;
+    return proxy.Checksum(req, &resp, &rpc);
   }
 
  private:
@@ -156,18 +193,11 @@ class SecurityITest : public KuduTest {
 };
 
 void SecurityITest::SmokeTestCluster() {
-  const char* kTableName = "test-table";
   client::sp::shared_ptr<KuduClient> client;
   ASSERT_OK(cluster_->CreateClient(nullptr, &client));
 
   // Create a table.
-  KuduSchema schema = client::KuduSchema::FromSchema(CreateKeyValueTestSchema());
-  gscoped_ptr<KuduTableCreator> table_creator(client->NewTableCreator());
-  ASSERT_OK(table_creator->table_name(kTableName)
-            .set_range_partition_columns({ "key" })
-            .schema(&schema)
-            .num_replicas(3)
-            .Create());
+  ASSERT_OK(CreateTestTable(client));
 
   // Insert a row.
   client::sp::shared_ptr<KuduTable> table;
@@ -201,6 +231,30 @@ TEST_F(SecurityITest, TestAuthorizationOnListTablets) {
   ASSERT_OK(TryListTablets());
 }
 
+TEST_F(SecurityITest, TestAuthorizationOnChecksum) {
+  cluster_opts_.extra_tserver_flags.emplace_back("--tserver_enforce_access_control");
+  ASSERT_OK(StartCluster());
+  client::sp::shared_ptr<KuduClient> client;
+  ASSERT_OK(cluster_->CreateClient(nullptr, &client));
+  ASSERT_OK(CreateTestTable(client));
+  vector<string> tablet_ids;
+  ASSERT_OK(TryListTablets(&tablet_ids));
+
+  // As a regular user, we shouldn't be authorized if we didn't send an authz
+  // token.
+  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
+  for (const auto& tablet_id : tablet_ids) {
+    Status s = TryChecksumWithoutAuthzToken(tablet_id);
+    ASSERT_STR_CONTAINS(s.ToString(), "Not authorized: no authorization token presented");
+  }
+  // As a super-user (e.g. if running the CLI as an admin), this should be
+  // allowed.
+  ASSERT_OK(cluster_->kdc()->Kinit("test-admin"));
+  for (const auto& tablet_id : tablet_ids) {
+    ASSERT_OK(TryChecksumWithoutAuthzToken(tablet_id));
+  }
+}
+
 // Test creating a table, writing some data, reading data, and dropping
 // the table.
 TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index cdc3bc9..6ce8949 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -574,9 +574,12 @@ void ServerBase::LogUnauthorizedAccess(rpc::RpcContext* rpc) const {
                << " from " << rpc->requestor_string();
 }
 
+bool ServerBase::IsFromSuperUser(const rpc::RpcContext* rpc) {
+  return superuser_acl_.UserAllowed(rpc->remote_user().username());
+}
+
 bool ServerBase::Authorize(rpc::RpcContext* rpc, uint32_t allowed_roles) {
-  if ((allowed_roles & SUPER_USER) &&
-      superuser_acl_.UserAllowed(rpc->remote_user().username())) {
+  if ((allowed_roles & SUPER_USER) && IsFromSuperUser(rpc)) {
     return true;
   }
 
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index a2aee01..426249a 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -119,6 +119,9 @@ class ServerBase {
     SERVICE_USER = 1 << 2
   };
 
+  // Returns whether or not the rpc is from a super-user.
+  bool IsFromSuperUser(const rpc::RpcContext* rpc);
+
   // Authorize an RPC. 'allowed_roles' is a bitset of which roles from the above
   // enum should be allowed to make hthe RPC.
   //
diff --git a/src/kudu/tools/kudu-tool-test.cc b/src/kudu/tools/kudu-tool-test.cc
index 8761522..be454f6 100644
--- a/src/kudu/tools/kudu-tool-test.cc
+++ b/src/kudu/tools/kudu-tool-test.cc
@@ -4995,5 +4995,35 @@ TEST_P(Is343ReplicaUtilTest, Is343Cluster) {
   }
 }
 
+class AuthzTServerChecksumTest : public ToolTest {
+ public:
+  void SetUp() override {
+    ExternalMiniClusterOptions opts;
+    opts.extra_tserver_flags.emplace_back("--tserver_enforce_access_control=true");
+    NO_FATALS(StartExternalMiniCluster(std::move(opts)));
+  }
+};
+
+// Test the authorization of Checksum scans via the CLI.
+TEST_F(AuthzTServerChecksumTest, TestAuthorizeChecksum) {
+  // First, let's create a table.
+  const vector<string> loadgen_args = {
+    "perf", "loadgen",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    "--keep_auto_table",
+    "--num_rows_per_thread=0",
+  };
+  ASSERT_OK(RunKuduTool(loadgen_args));
+
+  // Running a checksum scan should succeed since the tool is run as the OS
+  // user, which is the default super-user.
+  const vector<string> checksum_args = {
+    "cluster", "ksck",
+    cluster_->master()->bound_rpc_addr().ToString(),
+    "--checksum_scan"
+  };
+  ASSERT_OK(RunKuduTool(checksum_args));
+}
+
 } // namespace tools
 } // namespace kudu
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index b1b1792..c752dcd 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -1933,7 +1933,12 @@ void TabletServiceImpl::Checksum(const ChecksumRequestPB* req,
   ScanResultChecksummer collector;
   bool has_more = false;
   TabletServerErrorPB::Code error_code = TabletServerErrorPB::UNKNOWN_ERROR;
-  if (FLAGS_tserver_enforce_access_control && req->has_new_request()) {
+  // TODO(KUDU-2870): the CLI tool doesn't currently fetch authz tokens when
+  // checksumming. Until it does, allow the super-user to avoid fine-grained
+  // privilege checking.
+  if (FLAGS_tserver_enforce_access_control &&
+      !server_->IsFromSuperUser(context) &&
+      req->has_new_request()) {
     const NewScanRequestPB& new_req = req->new_request();
     TokenPB token;
     if (!VerifyAuthzTokenOrRespond(server_->token_verifier(), req->new_request(),


[kudu] 02/02: KUDU-2871 (part 1): disable TLS 1.3.

Posted by gr...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

granthenke pushed a commit to branch branch-1.10.x
in repository https://gitbox.apache.org/repos/asf/kudu.git

commit 86a0dc29fcfd3b6fd2eb8089839e0379b8dd62f4
Author: Todd Lipcon <to...@apache.org>
AuthorDate: Wed Jun 19 23:49:26 2019 -0700

    KUDU-2871 (part 1): disable TLS 1.3.
    
    This disables TLS 1.3 for our RPC negotiations, since the 1.5-RTT
    optimization breaks an assumption that the server sends the last token
    in the negotiation exchange.
    
    Tested that this fixes tls_handshake-test with Ubuntu 18 and libssl
    1.1.1.
    
    Change-Id: I431a1352ce1b8cca61b60c2dafbebadb4303e08a
    Reviewed-on: http://gerrit.cloudera.org:8080/13683
    Reviewed-by: Alexey Serbin <as...@cloudera.com>
    Tested-by: Kudu Jenkins
    (cherry picked from commit efc3f372e8b9254ab6b65d1f884381016329611c)
    Reviewed-on: http://gerrit.cloudera.org:8080/13684
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 src/kudu/rpc/client_negotiation.cc | 4 +++-
 src/kudu/security/tls_context.cc   | 7 +++++++
 2 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index d74bdbe..b43f55d 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -481,7 +481,9 @@ Status ClientNegotiation::HandleTlsHandshake(const NegotiatePB& response) {
     return Status::NotAuthorized("expected TLS_HANDSHAKE step",
                                  NegotiatePB::NegotiateStep_Name(response.step()));
   }
-  TRACE("Received TLS_HANDSHAKE response from server");
+  if (!response.tls_handshake().empty()) {
+    TRACE("Received TLS_HANDSHAKE response from server");
+  }
 
   if (PREDICT_FALSE(!response.has_tls_handshake())) {
     return Status::NotAuthorized("No TLS handshake token in TLS_HANDSHAKE response from server");
diff --git a/src/kudu/security/tls_context.cc b/src/kudu/security/tls_context.cc
index 9bf433d..a01b779 100644
--- a/src/kudu/security/tls_context.cc
+++ b/src/kudu/security/tls_context.cc
@@ -61,6 +61,9 @@
 #ifndef SSL_OP_NO_TLSv1_1
 #define SSL_OP_NO_TLSv1_1 0x10000000U
 #endif
+#ifndef SSL_OP_NO_TLSv1_3
+#define SSL_OP_NO_TLSv1_3 0x20000000U
+#endif
 #ifndef TLS1_1_VERSION
 #define TLS1_1_VERSION 0x0302
 #endif
@@ -165,6 +168,10 @@ Status TlsContext::Init() {
                                    tls_min_protocol_);
   }
 
+  // We don't currently support TLS 1.3 because the one-and-a-half-RTT negotiation
+  // confuses our RPC negotiation protocol. See KUDU-2871.
+  options |= SSL_OP_NO_TLSv1_3;
+
   SSL_CTX_set_options(ctx_.get(), options);
 
   OPENSSL_RET_NOT_OK(