You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by aw...@apache.org on 2019/04/09 20:39:10 UTC

[kudu] 02/04: authz: authorize ListTablets

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

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

commit 0f61255989fa92925433187b2eaaf900c8092f5c
Author: Andrew Wong <aw...@cloudera.com>
AuthorDate: Mon Oct 22 00:05:58 2018 -0700

    authz: authorize ListTablets
    
    See the comment in tablet_service.h for details.
    
    Change-Id: I4bb2d09f23d7b77729e21060dad41c0501b17ded
    Reviewed-on: http://gerrit.cloudera.org:8080/11752
    Tested-by: Andrew Wong <aw...@cloudera.com>
    Reviewed-by: Adar Dembo <ad...@cloudera.com>
    Reviewed-by: Hao Hao <ha...@cloudera.com>
---
 src/kudu/integration-tests/security-itest.cc | 27 +++++++++++++++++++++++++++
 src/kudu/tserver/tablet_service.cc           |  9 +++++++++
 src/kudu/tserver/tablet_service.h            |  9 +++++++++
 src/kudu/tserver/tserver_service.proto       |  2 +-
 4 files changed, 46 insertions(+), 1 deletion(-)

diff --git a/src/kudu/integration-tests/security-itest.cc b/src/kudu/integration-tests/security-itest.cc
index 3b9b0a7..92d7322 100644
--- a/src/kudu/integration-tests/security-itest.cc
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -50,6 +50,8 @@
 #include "kudu/server/server_base.pb.h"
 #include "kudu/server/server_base.proxy.h"
 #include "kudu/tablet/key_value_test_schema.h"
+#include "kudu/tserver/tserver.pb.h"
+#include "kudu/tserver/tserver_service.proxy.h"
 #include "kudu/util/env.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/net_util.h"
@@ -127,6 +129,17 @@ class SecurityITest : public KuduTest {
     return proxy.TSHeartbeat(req, &resp, &rpc);
   }
 
+  Status TryListTablets() {
+    auto messenger = NewMessengerOrDie();
+    const auto& addr = cluster_->tablet_server(0)->bound_rpc_addr();
+    tserver::TabletServerServiceProxy proxy(messenger, addr, addr.host());
+
+    rpc::RpcController rpc;
+    tserver::ListTabletsRequestPB req;
+    tserver::ListTabletsResponsePB resp;
+    return proxy.ListTablets(req, &resp, &rpc);
+  }
+
  private:
   std::shared_ptr<Messenger> NewMessengerOrDie() {
     std::shared_ptr<Messenger> messenger;
@@ -174,6 +187,20 @@ void SecurityITest::SmokeTestCluster() {
   ASSERT_OK(client->DeleteTable(kTableName));
 }
 
+// Test authorizing list tablets.
+TEST_F(SecurityITest, TestAuthorizationOnListTablets) {
+  // When enforcing access control, an operator of ListTablets must be
+  // superuser.
+  cluster_opts_.extra_tserver_flags.emplace_back("--tserver_enforce_access_control");
+  ASSERT_OK(StartCluster());
+  ASSERT_OK(cluster_->kdc()->Kinit("test-user"));
+  Status s = TryListTablets();
+  ASSERT_EQ("Remote error: Not authorized: unauthorized access to method: ListTablets",
+            s.ToString());
+  ASSERT_OK(cluster_->kdc()->Kinit("test-admin"));
+  ASSERT_OK(TryListTablets());
+}
+
 // Test creating a table, writing some data, reading data, and dropping
 // the table.
 TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 9d26b6b..d240b68 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -876,6 +876,15 @@ bool TabletServiceImpl::AuthorizeClientOrServiceUser(const google::protobuf::Mes
                             ServerBase::SERVICE_USER);
 }
 
+bool TabletServiceImpl::AuthorizeListTablets(const google::protobuf::Message* req,
+                                             google::protobuf::Message* resp,
+                                             rpc::RpcContext* context) {
+  if (FLAGS_tserver_enforce_access_control) {
+    return server_->Authorize(context, ServerBase::SUPER_USER);
+  }
+  return AuthorizeClient(req, resp, context);
+}
+
 bool TabletServiceImpl::AuthorizeClient(const google::protobuf::Message* /*req*/,
                                         google::protobuf::Message* /*resp*/,
                                         rpc::RpcContext* context) {
diff --git a/src/kudu/tserver/tablet_service.h b/src/kudu/tserver/tablet_service.h
index 57622a4..143b9ad 100644
--- a/src/kudu/tserver/tablet_service.h
+++ b/src/kudu/tserver/tablet_service.h
@@ -108,6 +108,15 @@ class TabletServiceImpl : public TabletServerServiceIf {
                                     google::protobuf::Message* resp,
                                     rpc::RpcContext* context) override;
 
+  // Note: we authorize ListTablets separately because our fine-grained access
+  // model is simpler when authorization is scoped to a single table, which
+  // isn't the case for ListTablets. Rather than authorizing multiple tables at
+  // once, if enforcing access control, we require the super-user role and omit
+  // checking table privileges, and authorize as a client otherwise.
+  bool AuthorizeListTablets(const google::protobuf::Message* req,
+                            google::protobuf::Message* resp,
+                            rpc::RpcContext* context) override;
+
   virtual void Ping(const PingRequestPB* req,
                     PingResponsePB* resp,
                     rpc::RpcContext* context) OVERRIDE;
diff --git a/src/kudu/tserver/tserver_service.proto b/src/kudu/tserver/tserver_service.proto
index 0f8c25b..07ea0c9 100644
--- a/src/kudu/tserver/tserver_service.proto
+++ b/src/kudu/tserver/tserver_service.proto
@@ -42,7 +42,7 @@ service TabletServerService {
     option (kudu.rpc.authz_method) = "AuthorizeClient";
   }
   rpc ListTablets(ListTabletsRequestPB) returns (ListTabletsResponsePB) {
-    option (kudu.rpc.authz_method) = "AuthorizeClient";
+    option (kudu.rpc.authz_method) = "AuthorizeListTablets";
   }
   rpc SplitKeyRange(SplitKeyRangeRequestPB) returns (SplitKeyRangeResponsePB) {
     option (kudu.rpc.authz_method) = "AuthorizeClient";