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

[1/2] kudu git commit: [client, server]_negotiation: replace gscoped_ptr with unique_ptr

Repository: kudu
Updated Branches:
  refs/heads/master fea449ace -> 758cf8b8d


[client, server]_negotiation: replace gscoped_ptr with unique_ptr

Change-Id: Ibe0151aeb54af4f54b7dbb8e019127329cdd526b
Reviewed-on: http://gerrit.cloudera.org:8080/6136
Reviewed-by: Alexey Serbin <as...@cloudera.com>
Reviewed-by: Todd Lipcon <to...@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/f76facd4
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/f76facd4
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/f76facd4

Branch: refs/heads/master
Commit: f76facd40f3473733f6c7c7f2f268bb799c44c3b
Parents: fea449a
Author: Dan Burkert <da...@apache.org>
Authored: Thu Feb 23 13:57:51 2017 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Fri Feb 24 23:35:39 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/client_negotiation.cc | 1 +
 src/kudu/rpc/client_negotiation.h  | 6 +++---
 src/kudu/rpc/server_negotiation.h  | 3 +--
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/f76facd4/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index 95efa7d..ec27855 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -111,6 +111,7 @@ ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
       tls_context_(tls_context),
       tls_negotiated_(false),
       authn_token_(authn_token),
+      psecret_(nullptr, std::free),
       negotiated_authn_(AuthenticationType::INVALID),
       negotiated_mech_(SaslMechanism::INVALID),
       deadline_(MonoTime::Max()) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/f76facd4/src/kudu/rpc/client_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.h b/src/kudu/rpc/client_negotiation.h
index c68e9ec..dc92204 100644
--- a/src/kudu/rpc/client_negotiation.h
+++ b/src/kudu/rpc/client_negotiation.h
@@ -17,6 +17,7 @@
 
 #pragma once
 
+#include <cstdlib>
 #include <memory>
 #include <set>
 #include <string>
@@ -25,7 +26,6 @@
 #include <boost/optional.hpp>
 #include <sasl/sasl.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/rpc/negotiation.h"
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/rpc/sasl_common.h"
@@ -201,7 +201,7 @@ class ClientNegotiation {
 
   // SASL state.
   std::vector<sasl_callback_t> callbacks_;
-  gscoped_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
+  std::unique_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
   SaslHelper helper_;
 
   // TLS state.
@@ -215,7 +215,7 @@ class ClientNegotiation {
   // Authentication state.
   std::string plain_auth_user_;
   std::string plain_pass_;
-  gscoped_ptr<sasl_secret_t, FreeDeleter> psecret_;
+  std::unique_ptr<sasl_secret_t, decltype(std::free)*> psecret_;
 
   // The set of features advertised by the client. Filled in when we send
   // the first message. This is not necessarily constant since some features

http://git-wip-us.apache.org/repos/asf/kudu/blob/f76facd4/src/kudu/rpc/server_negotiation.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.h b/src/kudu/rpc/server_negotiation.h
index 38202c3..23c9c34 100644
--- a/src/kudu/rpc/server_negotiation.h
+++ b/src/kudu/rpc/server_negotiation.h
@@ -24,7 +24,6 @@
 
 #include <sasl/sasl.h>
 
-#include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/rpc/negotiation.h"
 #include "kudu/rpc/remote_user.h"
 #include "kudu/rpc/rpc_header.pb.h"
@@ -212,7 +211,7 @@ class ServerNegotiation {
 
   // SASL state.
   std::vector<sasl_callback_t> callbacks_;
-  gscoped_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
+  std::unique_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
   SaslHelper helper_;
 
   // TLS state.


[2/2] kudu git commit: security: authorize all RPCs against coarse-grained ACLs

Posted by da...@apache.org.
security: authorize all RPCs against coarse-grained ACLs

This adds two new flags: 'superuser_acl' and 'user_acl'.
Cluster-admin operations (eg things like SetFlags) are authorized
against superuser_acl, and read/write/DDL type operations are authorized
against user_acl.

Internal-facing RPCs are authorized against the service user, which is
assumed to be a matching principal across all of the hosts.

Most of the "service" RPCs are also allowed to be accessed by
superusers, so that operator tools can take advantage of them. The one
exception is TSHeartbeat, which is locked down to _only_ the service
user, since it's the endpoint that exports signed IPKI certs.

The default service user and superuser ACL are set based on the identity
of the server: if the server is logged in from a keytab, we use that
username. Otherwise, we use the local Unix username. This means that
tests which rely on superuser things like SetFlags, etc, should continue
to work as before, since the test client runs as the same Unix user as
the server.

A new unit test smoke tests the various authorization levels using a
combination of the real client and hand-crafted RPCs.

Change-Id: Id24a6429273aff355e70e127086a26b7e4a03cd8
Reviewed-on: http://gerrit.cloudera.org:8080/5998
Reviewed-by: Dan Burkert <da...@apache.org>
Tested-by: Todd Lipcon <to...@apache.org>


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

Branch: refs/heads/master
Commit: 758cf8b8d65973b650c3262f79ff7e0c843736ce
Parents: f76facd
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 14 00:00:58 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Fri Feb 24 23:56:04 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/MiniKuduCluster.java |   2 +
 src/kudu/consensus/consensus.proto              |   2 +
 src/kudu/integration-tests/CMakeLists.txt       |   1 +
 .../external_mini_cluster-test.cc               |   2 +-
 .../integration-tests/external_mini_cluster.cc  |  15 +-
 src/kudu/integration-tests/security-itest.cc    | 184 +++++++++++++++++++
 src/kudu/master/master.cc                       |   4 +-
 src/kudu/master/master.proto                    |  67 +++++--
 src/kudu/master/master_service.cc               |  37 +++-
 src/kudu/master/master_service.h                |  14 ++
 src/kudu/rpc/rpc_context.cc                     |   8 +
 src/kudu/rpc/rpc_context.h                      |   6 +
 src/kudu/security/CMakeLists.txt                |   2 +
 src/kudu/security/init.cc                       |  15 +-
 src/kudu/security/init.h                        |   3 +
 src/kudu/security/kerberos_util.cc              |  36 ++++
 src/kudu/security/kerberos_util.h               |  29 +++
 src/kudu/security/simple_acl.cc                 |  87 +++++++++
 src/kudu/security/simple_acl.h                  |  60 ++++++
 src/kudu/server/generic_service.cc              |  14 ++
 src/kudu/server/generic_service.h               |   8 +
 src/kudu/server/server_base.cc                  |  88 +++++++++
 src/kudu/server/server_base.h                   |  26 ++-
 src/kudu/server/server_base.proto               |  19 +-
 src/kudu/tserver/tablet_copy.proto              |   2 +
 src/kudu/tserver/tablet_copy_service.cc         |  25 ++-
 src/kudu/tserver/tablet_copy_service.h          |  15 +-
 src/kudu/tserver/tablet_server.cc               |   6 +-
 src/kudu/tserver/tablet_service.cc              |  42 ++++-
 src/kudu/tserver/tablet_service.h               |  27 ++-
 src/kudu/tserver/tserver_admin.proto            |   3 +
 src/kudu/tserver/tserver_service.proto          |  26 ++-
 32 files changed, 801 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
index 546fa6c..3a60ba0 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/MiniKuduCluster.java
@@ -183,6 +183,7 @@ public class MiniKuduCluster implements AutoCloseable {
         commandLine.add("--keytab_file=" + keytab);
         commandLine.add("--principal=kudu/" + bindHost);
         commandLine.add("--rpc_authentication=required");
+        commandLine.add("--superuser_acl=testuser");
       }
 
       commandLine.addAll(extraTserverFlags);
@@ -264,6 +265,7 @@ public class MiniKuduCluster implements AutoCloseable {
         commandLine.add("--keytab_file=" + keytab);
         commandLine.add("--principal=kudu/" + bindHost);
         commandLine.add("--rpc_authentication=required");
+        commandLine.add("--superuser_acl=testuser");
       }
 
       commandLine.addAll(extraMasterFlags);

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/consensus/consensus.proto
----------------------------------------------------------------------
diff --git a/src/kudu/consensus/consensus.proto b/src/kudu/consensus/consensus.proto
index 699ebcc..e36e0bb 100644
--- a/src/kudu/consensus/consensus.proto
+++ b/src/kudu/consensus/consensus.proto
@@ -484,6 +484,8 @@ message StartTabletCopyResponsePB {
 
 // A Raft implementation.
 service ConsensusService {
+  option (kudu.rpc.default_authz_method) = "AuthorizeServiceUser";
+
   // Analogous to AppendEntries in Raft, but only used for followers.
   rpc UpdateConsensus(ConsensusRequestPB) returns (ConsensusResponsePB);
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/integration-tests/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/CMakeLists.txt b/src/kudu/integration-tests/CMakeLists.txt
index 5b6a2e8..7a0d32a 100644
--- a/src/kudu/integration-tests/CMakeLists.txt
+++ b/src/kudu/integration-tests/CMakeLists.txt
@@ -76,6 +76,7 @@ ADD_KUDU_TEST(master-stress-test RESOURCE_LOCK "master-rpc-ports")
 ADD_KUDU_TEST(open-readonly-fs-itest)
 ADD_KUDU_TEST(raft_consensus-itest RUN_SERIAL true)
 ADD_KUDU_TEST(registration-test RESOURCE_LOCK "master-web-port")
+ADD_KUDU_TEST(security-itest)
 ADD_KUDU_TEST(table_locations-itest)
 ADD_KUDU_TEST(tablet_copy-itest)
 ADD_KUDU_TEST(tablet_copy_client_session-itest)

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/integration-tests/external_mini_cluster-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/integration-tests/external_mini_cluster-test.cc b/src/kudu/integration-tests/external_mini_cluster-test.cc
index 5714591..b73343f 100644
--- a/src/kudu/integration-tests/external_mini_cluster-test.cc
+++ b/src/kudu/integration-tests/external_mini_cluster-test.cc
@@ -62,7 +62,7 @@ void SmokeTestKerberizedCluster(const ExternalMiniClusterOptions& opts) {
   SleepFor(MonoDelta::FromSeconds(10));
 
   // Re-kinit for the client, since the client's ticket would have expired as well.
-  ASSERT_OK(cluster.kdc()->Kinit("testuser"));
+  ASSERT_OK(cluster.kdc()->Kinit("test-admin"));
 
   // Restart the master, and make sure the tserver is still able to reconnect and
   // authenticate.

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/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 dcf024e..6dc84ae 100644
--- a/src/kudu/integration-tests/external_mini_cluster.cc
+++ b/src/kudu/integration-tests/external_mini_cluster.cc
@@ -148,10 +148,15 @@ Status ExternalMiniCluster::Start() {
   if (opts_.enable_kerberos) {
     kdc_.reset(new MiniKdc(opts_.mini_kdc_options));
     RETURN_NOT_OK(kdc_->Start());
-    RETURN_NOT_OK_PREPEND(kdc_->CreateUserPrincipal("testuser"),
-                          "could not create client principal");
-    RETURN_NOT_OK_PREPEND(kdc_->Kinit("testuser"),
-                          "could not kinit as client");
+    RETURN_NOT_OK_PREPEND(kdc_->CreateUserPrincipal("test-admin"),
+                          "could not create admin principal");
+    RETURN_NOT_OK_PREPEND(kdc_->CreateUserPrincipal("test-user"),
+                          "could not create user principal");
+    RETURN_NOT_OK_PREPEND(kdc_->CreateUserPrincipal("joe-interloper"),
+                          "could not create unauthorized principal");
+
+    RETURN_NOT_OK_PREPEND(kdc_->Kinit("test-admin"),
+                          "could not kinit as admin");
     RETURN_NOT_OK_PREPEND(kdc_->SetKrb5Environment(),
                           "could not set krb5 client env");
   }
@@ -607,6 +612,8 @@ Status ExternalDaemon::EnableKerberos(MiniKdc* kdc, const string& bind_host) {
   extra_flags_.push_back(Substitute("--keytab_file=$0", ktpath));
   extra_flags_.push_back(Substitute("--principal=$0", spn));
   extra_flags_.push_back("--rpc_authentication=required");
+  extra_flags_.push_back("--superuser_acl=test-admin");
+  extra_flags_.push_back("--user_acl=test-user");
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/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
new file mode 100644
index 0000000..a3268ef
--- /dev/null
+++ b/src/kudu/integration-tests/security-itest.cc
@@ -0,0 +1,184 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include <memory>
+
+#include "kudu/client/client.h"
+#include "kudu/client/client-test-util.h"
+#include "kudu/master/master.proxy.h"
+#include "kudu/integration-tests/external_mini_cluster.h"
+#include "kudu/tablet/key_value_test_schema.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/server/server_base.proxy.h"
+#include "kudu/util/test_util.h"
+
+using kudu::client::KuduClient;
+using kudu::client::KuduInsert;
+using kudu::client::KuduSchema;
+using kudu::client::KuduSession;
+using kudu::client::KuduTable;
+using kudu::client::KuduTableCreator;
+using kudu::rpc::Messenger;
+using std::unique_ptr;
+
+namespace kudu {
+
+class SecurityITest : public KuduTest {
+ public:
+  void StartCluster() {
+    ExternalMiniClusterOptions opts;
+    opts.enable_kerberos = true;
+    opts.num_tablet_servers = 3;
+    cluster_.reset(new ExternalMiniCluster(opts));
+    ASSERT_OK(cluster_->Start());
+  }
+
+  Status TrySetFlagOnTS() {
+    // Make a new messenger so that we don't reuse any cached connections from
+    // the minicluster startup sequence.
+    auto messenger = NewMessengerOrDie();
+    server::GenericServiceProxy proxy(
+        messenger, cluster_->tablet_server(0)->bound_rpc_addr());
+
+    rpc::RpcController controller;
+    controller.set_timeout(MonoDelta::FromSeconds(30));
+    server::SetFlagRequestPB req;
+    server::SetFlagResponsePB resp;
+    req.set_flag("non-existent");
+    req.set_value("xx");
+    return proxy.SetFlag(req, &resp, &controller);
+  }
+
+  Status TryRegisterAsTS() {
+    // Make a new messenger so that we don't reuse any cached connections from
+    // the minicluster startup sequence.
+    auto messenger = NewMessengerOrDie();
+    master::MasterServiceProxy proxy(
+        messenger, cluster_->master(0)->bound_rpc_addr());
+
+    rpc::RpcController rpc;
+    master::TSHeartbeatRequestPB req;
+    master::TSHeartbeatResponsePB resp;
+    req.mutable_common()->mutable_ts_instance()->set_permanent_uuid("x");
+    req.mutable_common()->mutable_ts_instance()->set_instance_seqno(1);
+    return proxy.TSHeartbeat(req, &resp, &rpc);
+  }
+
+ private:
+  std::shared_ptr<Messenger> NewMessengerOrDie() {
+    std::shared_ptr<Messenger> messenger;
+    CHECK_OK(rpc::MessengerBuilder("test-messenger")
+             .set_num_reactors(1)
+             .set_max_negotiation_threads(1)
+             .Build(&messenger));
+    return messenger;
+  }
+
+ protected:
+  unique_ptr<ExternalMiniCluster> cluster_;
+};
+
+// Test creating a table, writing some data, reading data, and dropping
+// the table.
+TEST_F(SecurityITest, SmokeTestAsAuthorizedUser) {
+  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));
+
+  // Create a table.
+  KuduSchema schema = client::KuduSchemaFromSchema(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());
+
+  // Insert a row.
+  client::sp::shared_ptr<KuduTable> table;
+  ASSERT_OK(client->OpenTable(kTableName, &table));
+  client::sp::shared_ptr<KuduSession> session = client->NewSession();
+  session->SetTimeoutMillis(60000);
+  unique_ptr<KuduInsert> ins(table->NewInsert());
+  ASSERT_OK(ins->mutable_row()->SetInt32(0, 12345));
+  ASSERT_OK(ins->mutable_row()->SetInt32(1, 54321));
+  ASSERT_OK(session->Apply(ins.release()));
+  FlushSessionOrDie(session);
+
+  // Read it back.
+  ASSERT_EQ(1, CountTableRows(table.get()));
+
+  // Delete the table.
+  ASSERT_OK(client->DeleteTable(kTableName));
+
+  // Non-superuser clients should not be able to set flags.
+  Status s = TrySetFlagOnTS();
+  ASSERT_EQ("Remote error: Not authorized: unauthorized access to method: SetFlag",
+            s.ToString());
+
+  // Nor should they be able to send TS RPCs.
+  s = TryRegisterAsTS();
+  ASSERT_EQ("Remote error: Not authorized: unauthorized access to method: TSHeartbeat",
+            s.ToString());
+}
+
+// Test trying to access the cluster with no Kerberos credentials at all.
+TEST_F(SecurityITest, TestNoKerberosCredentials) {
+  StartCluster();
+  ASSERT_OK(cluster_->kdc()->Kdestroy());
+
+  client::sp::shared_ptr<KuduClient> client;
+  Status s = cluster_->CreateClient(nullptr, &client);
+  ASSERT_STR_MATCHES(s.ToString(),
+                     "Not authorized: Could not connect to the cluster: "
+                     "Client connection negotiation failed: client connection "
+                     "to .*: No Kerberos credentials available");
+}
+
+// Test cluster access by a user who is not authorized as a client.
+TEST_F(SecurityITest, TestUnauthorizedClientKerberosCredentials) {
+  StartCluster();
+  ASSERT_OK(cluster_->kdc()->Kinit("joe-interloper"));
+  client::sp::shared_ptr<KuduClient> client;
+  Status s = cluster_->CreateClient(nullptr, &client);
+  ASSERT_EQ("Remote error: Could not connect to the cluster: "
+            "Not authorized: unauthorized access to method: ConnectToMaster",
+            s.ToString());
+}
+
+// Test superuser actions when authorized as a superuser.
+TEST_F(SecurityITest, TestAuthorizedSuperuser) {
+  StartCluster();
+
+  ASSERT_OK(cluster_->kdc()->Kinit("test-admin"));
+
+  // Superuser can set flags.
+  ASSERT_OK(TrySetFlagOnTS());
+
+  // Even superusers can't pretend to be tablet servers.
+  Status s = TryRegisterAsTS();
+
+  ASSERT_EQ("Remote error: Not authorized: unauthorized access to method: TSHeartbeat",
+            s.ToString());
+
+}
+
+
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/master/master.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.cc b/src/kudu/master/master.cc
index ce7eb08..984f50a 100644
--- a/src/kudu/master/master.cc
+++ b/src/kudu/master/master.cc
@@ -141,9 +141,9 @@ Status Master::StartAsync() {
 
   gscoped_ptr<ServiceIf> impl(new MasterServiceImpl(this));
   gscoped_ptr<ServiceIf> consensus_service(new ConsensusServiceImpl(
-      metric_entity(), result_tracker(), catalog_manager_.get()));
+      this, catalog_manager_.get()));
   gscoped_ptr<ServiceIf> tablet_copy_service(new TabletCopyServiceImpl(
-      fs_manager_.get(), catalog_manager_.get(), metric_entity(), result_tracker()));
+      this, catalog_manager_.get()));
 
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(impl)));
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(consensus_service)));

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/master/master.proto
----------------------------------------------------------------------
diff --git a/src/kudu/master/master.proto b/src/kudu/master/master.proto
index 9846968..79de60f 100644
--- a/src/kudu/master/master.proto
+++ b/src/kudu/master/master.proto
@@ -21,6 +21,7 @@ option java_package = "org.apache.kudu.master";
 import "kudu/common/common.proto";
 import "kudu/common/wire_protocol.proto";
 import "kudu/consensus/metadata.proto";
+import "kudu/rpc/rpc_header.proto";
 import "kudu/security/token.proto";
 import "kudu/tablet/metadata.proto";
 import "kudu/util/pb_util.proto";
@@ -674,35 +675,67 @@ enum MasterFeatures {
 }
 
 service MasterService {
+  // Set the default authz method to something invalid, so that if
+  // we forget to set the option on a new RPC call, we'll get a build
+  // failure.
+  option (kudu.rpc.default_authz_method) = "MUST_SET_AUTHZ_PER_RPC";
+
   // TS->Master RPCs
   // ------------------------------------------------------------
-  rpc TSHeartbeat(TSHeartbeatRequestPB) returns (TSHeartbeatResponsePB);
+  rpc TSHeartbeat(TSHeartbeatRequestPB) returns (TSHeartbeatResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeService";
+  }
 
   // Client->Master RPCs
   // ------------------------------------------------------------
 
   // Used only by Kudu 1.3 and later.
-  rpc ConnectToMaster(ConnectToMasterRequestPB) returns (ConnectToMasterResponsePB);
-
-  rpc GetTabletLocations(GetTabletLocationsRequestPB) returns (GetTabletLocationsResponsePB);
+  rpc ConnectToMaster(ConnectToMasterRequestPB) returns (ConnectToMasterResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
-  rpc CreateTable(CreateTableRequestPB) returns (CreateTableResponsePB);
-  rpc IsCreateTableDone(IsCreateTableDoneRequestPB) returns (IsCreateTableDoneResponsePB);
+  rpc GetTabletLocations(GetTabletLocationsRequestPB) returns (GetTabletLocationsResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
-  rpc DeleteTable(DeleteTableRequestPB) returns (DeleteTableResponsePB);
+  rpc CreateTable(CreateTableRequestPB) returns (CreateTableResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc IsCreateTableDone(IsCreateTableDoneRequestPB) returns (IsCreateTableDoneResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc DeleteTable(DeleteTableRequestPB) returns (DeleteTableResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
-  rpc AlterTable(AlterTableRequestPB) returns (AlterTableResponsePB);
-  rpc IsAlterTableDone(IsAlterTableDoneRequestPB) returns (IsAlterTableDoneResponsePB);
+  rpc AlterTable(AlterTableRequestPB) returns (AlterTableResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc IsAlterTableDone(IsAlterTableDoneRequestPB) returns (IsAlterTableDoneResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
-  rpc ListTables(ListTablesRequestPB) returns (ListTablesResponsePB);
-  rpc GetTableLocations(GetTableLocationsRequestPB) returns (GetTableLocationsResponsePB);
-  rpc GetTableSchema(GetTableSchemaRequestPB) returns (GetTableSchemaResponsePB);
+  rpc ListTables(ListTablesRequestPB) returns (ListTablesResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc GetTableLocations(GetTableLocationsRequestPB) returns (GetTableLocationsResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc GetTableSchema(GetTableSchemaRequestPB) returns (GetTableSchemaResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
   // Administrative/monitoring RPCs
   // ------------------------------------------------------------
-  rpc ListTabletServers(ListTabletServersRequestPB) returns (ListTabletServersResponsePB);
-  rpc ListMasters(ListMastersRequestPB) returns (ListMastersResponsePB);
-  rpc Ping(PingRequestPB) returns (PingResponsePB);
+  rpc ListTabletServers(ListTabletServersRequestPB) returns (ListTabletServersResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc ListMasters(ListMastersRequestPB) returns (ListMastersResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc Ping(PingRequestPB) returns (PingResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClientOrService";
+  }
 
   // Master->Master RPCs
   // ------------------------------------------------------------
@@ -710,5 +743,7 @@ service MasterService {
   // NOTE: this RPC is also used by Kudu client <= 1.2 when first connecting to the
   // cluster.
   rpc GetMasterRegistration(GetMasterRegistrationRequestPB) returns
-    (GetMasterRegistrationResponsePB);
+      (GetMasterRegistrationResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClientOrService";
+  }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index f35d4b9..7ba1bcf 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -18,6 +18,7 @@
 #include "kudu/master/master_service.h"
 
 #include <gflags/gflags.h>
+#include <google/protobuf/message.h>
 #include <memory>
 #include <string>
 #include <vector>
@@ -50,14 +51,16 @@ TAG_FLAG(master_support_connect_to_master_rpc, unsafe);
 TAG_FLAG(master_support_connect_to_master_rpc, hidden);
 
 using kudu::security::SignedTokenPB;
+using google::protobuf::Message;
+using std::string;
+using std::vector;
+using std::shared_ptr;
 
 namespace kudu {
 namespace master {
 
-using consensus::RaftPeerPB;
-using std::string;
-using std::vector;
-using std::shared_ptr;
+using server::ServerBase;
+using security::SignedTokenPB;
 using strings::Substitute;
 
 namespace {
@@ -80,8 +83,30 @@ MasterServiceImpl::MasterServiceImpl(Master* server)
     server_(server) {
 }
 
-void MasterServiceImpl::Ping(const PingRequestPB* req,
-                             PingResponsePB* resp,
+bool MasterServiceImpl::AuthorizeClient(const Message* /*req*/,
+                                        Message* /*resp*/,
+                                        rpc::RpcContext* context) {
+  return server_->Authorize(context, ServerBase::SUPER_USER | ServerBase::USER);
+}
+
+bool MasterServiceImpl::AuthorizeService(const Message* /*req*/,
+                                         Message* /*resp*/,
+                                         rpc::RpcContext* context) {
+  // We don't allow superusers to pretend to be tablet servers -- there are no
+  // operator tools that do anything like this and since we sign requests for
+  // tablet servers, we should be extra tight here.
+  return server_->Authorize(context, ServerBase::SERVICE_USER);
+}
+
+bool MasterServiceImpl::AuthorizeClientOrService(const Message* /*req*/,
+                                                 Message* /*resp*/,
+                                                 rpc::RpcContext* context) {
+  return server_->Authorize(context, ServerBase::SUPER_USER | ServerBase::USER |
+                            ServerBase::SERVICE_USER);
+}
+
+void MasterServiceImpl::Ping(const PingRequestPB* /*req*/,
+                             PingResponsePB* /*resp*/,
                              rpc::RpcContext* rpc) {
   rpc->RespondSuccess();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/master/master_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.h b/src/kudu/master/master_service.h
index 4aa87eb..70eafaf 100644
--- a/src/kudu/master/master_service.h
+++ b/src/kudu/master/master_service.h
@@ -36,6 +36,20 @@ class MasterServiceImpl : public MasterServiceIf {
  public:
   explicit MasterServiceImpl(Master* server);
 
+  // Authorize an RPC call which must be from a client.
+  bool AuthorizeClient(const google::protobuf::Message* req,
+                       google::protobuf::Message* resp,
+                       rpc::RpcContext *context) override;
+
+  // Authorize an RPC call which must be from within the Kudu service.
+  bool AuthorizeService(const google::protobuf::Message* req,
+                        google::protobuf::Message* resp,
+                        rpc::RpcContext *context) override;
+
+  bool AuthorizeClientOrService(const google::protobuf::Message* req,
+                                google::protobuf::Message* resp,
+                                rpc::RpcContext *context) override;
+
   virtual void Ping(const PingRequestPB* req,
                     PingResponsePB* resp,
                     rpc::RpcContext* rpc) OVERRIDE;

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/rpc/rpc_context.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.cc b/src/kudu/rpc/rpc_context.cc
index 2e45085..a0e634c 100644
--- a/src/kudu/rpc/rpc_context.cc
+++ b/src/kudu/rpc/rpc_context.cc
@@ -158,6 +158,14 @@ std::string RpcContext::requestor_string() const {
     call_->remote_address().ToString();
 }
 
+std::string RpcContext::method_name() const {
+  return call_->remote_method().method_name();
+}
+
+std::string RpcContext::service_name() const {
+  return call_->remote_method().service_name();
+}
+
 MonoTime RpcContext::GetClientDeadline() const {
   return call_->GetClientDeadline();
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/rpc/rpc_context.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_context.h b/src/kudu/rpc/rpc_context.h
index f0c7e04..b95a9ce 100644
--- a/src/kudu/rpc/rpc_context.h
+++ b/src/kudu/rpc/rpc_context.h
@@ -165,6 +165,12 @@ class RpcContext {
   // Suitable for use in log messages.
   std::string requestor_string() const;
 
+  // Return the name of the RPC service method being called.
+  std::string method_name() const;
+
+  // Return the name of the RPC service being called.
+  std::string service_name() const;
+
   const google::protobuf::Message *request_pb() const { return request_pb_.get(); }
   google::protobuf::Message *response_pb() const { return response_pb_.get(); }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/security/CMakeLists.txt b/src/kudu/security/CMakeLists.txt
index 9126f4b..c24deef 100644
--- a/src/kudu/security/CMakeLists.txt
+++ b/src/kudu/security/CMakeLists.txt
@@ -58,9 +58,11 @@ set(SECURITY_SRCS
   ca/cert_management.cc
   cert.cc
   crypto.cc
+  kerberos_util.cc
   init.cc
   openssl_util.cc
   ${PORTED_X509_CHECK_HOST_CC}
+  simple_acl.cc
   tls_context.cc
   tls_handshake.cc
   tls_socket.cc

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/init.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/init.cc b/src/kudu/security/init.cc
index fe4659b..2e7d940 100644
--- a/src/kudu/security/init.cc
+++ b/src/kudu/security/init.cc
@@ -107,6 +107,7 @@ class KinitContext {
   int32_t GetBackedOffRenewInterval(int32_t time_remaining, uint32_t num_retries);
 
   const string& principal_str() const { return principal_str_; }
+  const string& username_str() const { return username_str_; }
 
  private:
   krb5_principal principal_;
@@ -114,8 +115,8 @@ class KinitContext {
   krb5_ccache ccache_;
   krb5_get_init_creds_opt* opts_;
 
-  // The stringified principal that we are logged in as.
-  string principal_str_;
+  // The stringified principal and username that we are logged in as.
+  string principal_str_, username_str_;
 
   // This is the time that the current TGT in use expires.
   int32_t ticket_end_timestamp_;
@@ -343,8 +344,11 @@ Status KinitContext::Kinit(const string& keytab_path, const string& principal) {
   // configuration if not originally specified.
   RETURN_NOT_OK_PREPEND(Krb5UnparseName(principal_, &principal_str_),
                         "could not stringify the logged-in principal");
+  RETURN_NOT_OK_PREPEND(MapPrincipalToLocalName(principal_str_, &username_str_),
+                        "could not map own logged-in principal to a short username");
 
-  LOG(INFO) << "Logged in from keytab as " << principal_str_;
+  LOG(INFO) << "Logged in from keytab as " << principal_str_
+            << " (short username " << username_str_ << ")";
 
   return Status::OK();
 }
@@ -418,6 +422,11 @@ boost::optional<string> GetLoggedInPrincipalFromKeytab() {
   return g_kinit_ctx->principal_str();
 }
 
+boost::optional<string> GetLoggedInUsernameFromKeytab() {
+  if (!g_kinit_ctx) return boost::none;
+  return g_kinit_ctx->username_str();
+}
+
 Status InitKerberosForServer() {
   if (FLAGS_keytab_file.empty()) return Status::OK();
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/init.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/init.h b/src/kudu/security/init.h
index c62346f..61c9577 100644
--- a/src/kudu/security/init.h
+++ b/src/kudu/security/init.h
@@ -42,6 +42,9 @@ RWMutex* KerberosReinitLock();
 // If the server has not logged in from a keytab, returns boost::none.
 boost::optional<std::string> GetLoggedInPrincipalFromKeytab();
 
+// Same, but returns the mapped short username.
+boost::optional<std::string> GetLoggedInUsernameFromKeytab();
+
 // Canonicalize the given principal name by adding '@DEFAULT_REALM' in the case that
 // the principal has no realm.
 //

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/kerberos_util.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/kerberos_util.cc b/src/kudu/security/kerberos_util.cc
new file mode 100644
index 0000000..09f1b82
--- /dev/null
+++ b/src/kudu/security/kerberos_util.cc
@@ -0,0 +1,36 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/kerberos_util.h"
+#include "kudu/gutil/strings/split.h"
+
+#include <array>
+#include <utility>
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal) {
+
+  std::pair<StringPiece, StringPiece> user_realm = strings::Split(principal, "@");
+  std::pair<StringPiece, StringPiece> princ_host = strings::Split(user_realm.first, "/");
+  return {{princ_host.first, princ_host.second, user_realm.second}};
+}
+
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/kerberos_util.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/kerberos_util.h b/src/kudu/security/kerberos_util.h
new file mode 100644
index 0000000..4c27a86
--- /dev/null
+++ b/src/kudu/security/kerberos_util.h
@@ -0,0 +1,29 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <array>
+
+class StringPiece;
+
+namespace kudu {
+namespace security {
+
+std::array<StringPiece, 3> SplitKerberosPrincipal(StringPiece principal);
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/simple_acl.cc
----------------------------------------------------------------------
diff --git a/src/kudu/security/simple_acl.cc b/src/kudu/security/simple_acl.cc
new file mode 100644
index 0000000..75b06a2
--- /dev/null
+++ b/src/kudu/security/simple_acl.cc
@@ -0,0 +1,87 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "kudu/security/simple_acl.h"
+
+#include <ctype.h>
+
+#include <vector>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/util/status.h"
+
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace security {
+
+SimpleAcl::SimpleAcl() {
+}
+
+SimpleAcl::~SimpleAcl() {
+}
+
+Status SimpleAcl::ParseFlag(const string& flag) {
+  vector<StringPiece> fields = strings::Split(flag, ",", strings::SkipWhitespace());
+  set<string> users;
+  for (const auto& field : fields) {
+    if (field.empty()) continue;
+    // if any field is a wildcard, no need to include the rest.
+    if (flag == "*") {
+      Reset({"*"});
+      return Status::OK();
+    }
+
+
+    // Leave open the use of various special characters at the start of each
+    // username. We reserve some special characters that might be useful in
+    // ACLs:
+    // '!': might be interpreted as "not"
+    // '@': often used to read data from a file
+    // '#': comments
+    // '$': maybe variable expansion?
+    // '%': used by sudoers for groups
+    // '*': only allowed for special wildcard ACL above
+    // '-', '+', '=': useful for allow/deny style ACLs
+    // <quote characters>: in case we want to add quoted strings
+    // whitespace: down right confusing
+    static const char* kReservedStartingCharacters = "!@#$%*-=+'\"";
+    if (strchr(kReservedStartingCharacters, field[0]) ||
+        isspace(field[0])) {
+      return Status::NotSupported("invalid username", field.ToString());
+    }
+
+    users.insert(field.ToString());
+  }
+
+  Reset(std::move(users));
+  return Status::OK();
+}
+
+void SimpleAcl::Reset(set<string> users) {
+  users_ = std::move(users);
+}
+
+bool SimpleAcl::UserAllowed(const string& username) {
+  return ContainsKey(users_, "*") || ContainsKey(users_, username);
+}
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/security/simple_acl.h
----------------------------------------------------------------------
diff --git a/src/kudu/security/simple_acl.h b/src/kudu/security/simple_acl.h
new file mode 100644
index 0000000..96ca598
--- /dev/null
+++ b/src/kudu/security/simple_acl.h
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include "kudu/gutil/macros.h"
+
+#include <set>
+#include <string>
+
+namespace kudu {
+class Status;
+
+namespace security {
+
+// Represent a very simple access control list which contains a set of users.
+//
+// This is basically just a wrapper around a set<string> with a bit of parsing logic and
+// support for the '*' wildcard.
+class SimpleAcl {
+ public:
+  SimpleAcl();
+  ~SimpleAcl();
+
+  // Parse a flag value, which should be of the form 'user1,user2,user3' to indicate a
+  // list of users, or '*' to indicate a wildcard. This syntax may be expanded later to
+  // include groups, "allow/deny" style access, etc.
+  //
+  // Thread-unsafe: must be called before the ACL may be consulted, and may not be
+  // called a second time concurrent with reads of the ACL.
+  Status ParseFlag(const std::string& flag);
+
+  // Return true if the given user is allowed by the ACL.
+  //
+  // Thread-safe after initialization.
+  bool UserAllowed(const std::string& username);
+
+  // Reset the ACL to the specific set of usernames.
+  void Reset(std::set<std::string> users);
+
+ private:
+  // The set of users, or a set with the single value '*' for the wildcard.
+  std::set<std::string> users_;
+};
+
+} // namespace security
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/server/generic_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/generic_service.cc b/src/kudu/server/generic_service.cc
index 42cb88e..f4562b3 100644
--- a/src/kudu/server/generic_service.cc
+++ b/src/kudu/server/generic_service.cc
@@ -23,6 +23,7 @@
 
 #include "kudu/gutil/map-util.h"
 #include "kudu/rpc/rpc_context.h"
+#include "kudu/rpc/remote_user.h"
 #include "kudu/server/clock.h"
 #include "kudu/server/hybrid_clock.h"
 #include "kudu/server/server_base.h"
@@ -47,6 +48,19 @@ GenericServiceImpl::GenericServiceImpl(ServerBase* server)
 GenericServiceImpl::~GenericServiceImpl() {
 }
 
+bool GenericServiceImpl::AuthorizeSuperUser(const google::protobuf::Message* /*req*/,
+                                            google::protobuf::Message* /*resp*/,
+                                            rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER);
+}
+
+bool GenericServiceImpl::AuthorizeClient(const google::protobuf::Message* /*req*/,
+                                         google::protobuf::Message* /*resp*/,
+                                         rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::USER);
+}
+
+
 void GenericServiceImpl::SetFlag(const SetFlagRequestPB* req,
                                  SetFlagResponsePB* resp,
                                  rpc::RpcContext* rpc) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/server/generic_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/generic_service.h b/src/kudu/server/generic_service.h
index 1ef5423..d3038f2 100644
--- a/src/kudu/server/generic_service.h
+++ b/src/kudu/server/generic_service.h
@@ -30,6 +30,14 @@ class GenericServiceImpl : public GenericServiceIf {
   explicit GenericServiceImpl(ServerBase* server);
   virtual ~GenericServiceImpl();
 
+  bool AuthorizeSuperUser(const google::protobuf::Message* req,
+                          google::protobuf::Message* resp,
+                          rpc::RpcContext* rpc) override;
+
+  bool AuthorizeClient(const google::protobuf::Message* req,
+                       google::protobuf::Message* resp,
+                       rpc::RpcContext* rpc) override;
+
   virtual void SetFlag(const SetFlagRequestPB* req,
                        SetFlagResponsePB* resp,
                        rpc::RpcContext* rpc) OVERRIDE;

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/server/server_base.cc
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.cc b/src/kudu/server/server_base.cc
index 1c981d1..b67b7f7 100644
--- a/src/kudu/server/server_base.cc
+++ b/src/kudu/server/server_base.cc
@@ -21,16 +21,21 @@
 #include <vector>
 
 #include <boost/algorithm/string/predicate.hpp>
+#include <boost/optional.hpp>
 #include <gflags/gflags.h>
 
 #include "kudu/codegen/compilation_manager.h"
 #include "kudu/common/wire_protocol.h"
 #include "kudu/common/wire_protocol.pb.h"
 #include "kudu/fs/fs_manager.h"
+#include "kudu/gutil/strings/split.h"
 #include "kudu/gutil/strings/strcat.h"
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/walltime.h"
 #include "kudu/rpc/messenger.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/rpc_context.h"
+#include "kudu/security/kerberos_util.h"
 #include "kudu/security/init.h"
 #include "kudu/server/default-path-handlers.h"
 #include "kudu/server/generic_service.h"
@@ -59,6 +64,7 @@
 #include "kudu/util/rolling_log.h"
 #include "kudu/util/spinlock_profiling.h"
 #include "kudu/util/thread.h"
+#include "kudu/util/user.h"
 #include "kudu/util/version_info.h"
 
 DEFINE_int32(num_reactor_threads, 4, "Number of libev reactor threads to start.");
@@ -70,6 +76,23 @@ 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_string(superuser_acl, "",
+              "The list of usernames to allow as super users, comma-separated. "
+              "A '*' entry indicates that all authenticated users are allowed. "
+              "If this is left unset or blank, the default behavior is that the "
+              "identity of the daemon itself determines the superuser. If the "
+              "daemon is logged in from a Keytab, then the local username from "
+              "the Kerberos principal is used; otherwise, the local Unix "
+              "username is used.");
+TAG_FLAG(superuser_acl, stable);
+TAG_FLAG(superuser_acl, sensitive);
+
+DEFINE_string(user_acl, "*",
+              "The list of usernames who may access the cluster, comma-separated. "
+              "A '*' entry indicates that all authenticated users are allowed.");
+TAG_FLAG(user_acl, stable);
+TAG_FLAG(user_acl, sensitive);
+
 DECLARE_bool(use_hybrid_clock);
 
 using std::ostringstream;
@@ -187,6 +210,8 @@ Status ServerBase::Init() {
   }
   RETURN_NOT_OK_PREPEND(s, "Failed to load FS layout");
 
+  RETURN_NOT_OK(InitAcls());
+
   // Create the Messenger.
   rpc::MessengerBuilder builder(name_);
 
@@ -210,6 +235,41 @@ Status ServerBase::Init() {
   return Status::OK();
 }
 
+Status ServerBase::InitAcls() {
+
+  string service_user;
+  boost::optional<string> keytab_user = security::GetLoggedInUsernameFromKeytab();
+  if (keytab_user) {
+    // If we're logged in from a keytab, then everyone should be, and we expect them
+    // to use the same mapped username.
+    service_user = *keytab_user;
+  } else {
+    // If we aren't logged in from a keytab, then just assume that the services
+    // will be running as the same Unix user as we are.
+    RETURN_NOT_OK_PREPEND(GetLoggedInUser(&service_user),
+                          "could not deterine local username");
+  }
+
+  // If the user has specified a superuser acl, use that. Otherwise, assume
+  // that the same user running the service acts as superuser.
+  if (!FLAGS_superuser_acl.empty()) {
+    RETURN_NOT_OK_PREPEND(superuser_acl_.ParseFlag(FLAGS_superuser_acl),
+                          "could not parse --superuser_acl flag");
+  } else {
+    superuser_acl_.Reset({ service_user });
+  }
+
+  RETURN_NOT_OK_PREPEND(user_acl_.ParseFlag(FLAGS_user_acl),
+                        "could not parse --user_acl flag");
+
+  // For the "service" ACL, we currently don't allow it to be user-configured,
+  // but instead assume that all of the services will be running the same
+  // way.
+  service_acl_.Reset({ service_user });
+
+  return Status::OK();
+}
+
 void ServerBase::GetStatusPB(ServerStatusPB* status) const {
   // Node instance
   status->mutable_node_instance()->CopyFrom(*instance_pb_);
@@ -241,6 +301,34 @@ void ServerBase::GetStatusPB(ServerStatusPB* status) const {
   VersionInfo::GetVersionInfoPB(status->mutable_version_info());
 }
 
+void ServerBase::LogUnauthorizedAccess(rpc::RpcContext* rpc) const {
+  LOG(WARNING) << "Unauthorized access attempt to method "
+               << rpc->service_name() << "." << rpc->method_name()
+               << " from " << rpc->requestor_string();
+}
+
+bool ServerBase::Authorize(rpc::RpcContext* rpc, uint32_t allowed_roles) {
+  if ((allowed_roles & SUPER_USER) &&
+      superuser_acl_.UserAllowed(rpc->remote_user().username())) {
+    return true;
+  }
+
+  if ((allowed_roles & USER) &&
+      user_acl_.UserAllowed(rpc->remote_user().username())) {
+    return true;
+  }
+
+  if ((allowed_roles & SERVICE_USER) &&
+      service_acl_.UserAllowed(rpc->remote_user().username())) {
+    return true;
+  }
+
+  LogUnauthorizedAccess(rpc);
+  rpc->RespondFailure(Status::NotAuthorized("unauthorized access to method",
+                                            rpc->method_name()));
+  return false;
+}
+
 Status ServerBase::DumpServerInfo(const string& path,
                                   const string& format) const {
   ServerStatusPB status;

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/server/server_base.h
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.h b/src/kudu/server/server_base.h
index a76a338..951c3c3 100644
--- a/src/kudu/server/server_base.h
+++ b/src/kudu/server/server_base.h
@@ -25,6 +25,7 @@
 #include "kudu/gutil/ref_counted.h"
 #include "kudu/rpc/messenger.h"
 #include "kudu/rpc/service_if.h"
+#include "kudu/security/simple_acl.h"
 #include "kudu/server/server_base_options.h"
 #include "kudu/util/status.h"
 
@@ -44,7 +45,7 @@ class Thread;
 class Webserver;
 
 namespace rpc {
-class ServiceIf;
+class RpcContext;
 } // namespace rpc
 
 namespace security {
@@ -101,6 +102,18 @@ class ServerBase {
   // Return a PB describing the status of the server (version info, bound ports, etc)
   void GetStatusPB(ServerStatusPB* status) const;
 
+  enum {
+    SUPER_USER = 1,
+    USER = 1 << 1,
+    SERVICE_USER = 1 << 2
+  };
+
+  // Authorize an RPC. 'allowed_roles' is a bitset of which roles from the above
+  // enum should be allowed to make hthe RPC.
+  //
+  // If authorization fails, return false and respond to the RPC.
+  bool Authorize(rpc::RpcContext* rpc, uint32_t allowed_roles);
+
  protected:
   ServerBase(std::string name, const ServerBaseOptions& options,
              const std::string& metric_namespace);
@@ -111,6 +124,8 @@ class ServerBase {
   Status Start();
   void Shutdown();
 
+  void LogUnauthorizedAccess(rpc::RpcContext* rpc) const;
+
   const std::string name_;
 
   std::unique_ptr<MinidumpExceptionHandler> minidump_handler_;
@@ -130,7 +145,16 @@ class ServerBase {
   // The instance identifier of this server.
   gscoped_ptr<NodeInstancePB> instance_pb_;
 
+  // The ACL of users who are allowed to act as superusers.
+  security::SimpleAcl superuser_acl_;
+
+  // The ACL of users who are allowed to access the cluster.
+  security::SimpleAcl user_acl_;
+
+  // The ACL of users who may act as part of the Kudu service.
+  security::SimpleAcl service_acl_;
  private:
+  Status InitAcls();
   void GenerateInstanceID();
   Status DumpServerInfo(const std::string& path,
                         const std::string& format) const;

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/server/server_base.proto
----------------------------------------------------------------------
diff --git a/src/kudu/server/server_base.proto b/src/kudu/server/server_base.proto
index ed031fc..c81d49b 100644
--- a/src/kudu/server/server_base.proto
+++ b/src/kudu/server/server_base.proto
@@ -20,6 +20,7 @@ option java_package = "org.apache.kudu.server";
 
 import "kudu/common/common.proto";
 import "kudu/common/wire_protocol.proto";
+import "kudu/rpc/rpc_header.proto";
 import "kudu/util/version_info.proto";
 
 // The status information dumped by a server after it starts.
@@ -33,14 +34,13 @@ message ServerStatusPB {
   repeated HostPortPB bound_rpc_addresses = 2;
   repeated HostPortPB bound_http_addresses = 3;
   optional VersionInfoPB version_info = 4;
+  // NOTE: this PB may be fetched by any authenticated user, not just superusers.
+  // So, do not expose anything here which may be sensitive!
 }
 
 // Attempt to set a command line flag.
 // Note that many command line flags do not take effect if changed
 // at runtime.
-//
-// TODO: We plan to add tags indicating which flags can be modified at
-// runtime. For now, this is an advanced option.
 message SetFlagRequestPB {
   required string flag = 1;
   required string value = 2;
@@ -123,6 +123,8 @@ message SetServerWallClockForTestsResponsePB {
 }
 
 service GenericService {
+  option (kudu.rpc.default_authz_method) = "AuthorizeSuperUser";
+
   rpc SetFlag(SetFlagRequestPB)
     returns (SetFlagResponsePB);
 
@@ -132,12 +134,15 @@ service GenericService {
   rpc CheckLeaks(CheckLeaksRequestPB)
     returns (CheckLeaksResponsePB);
 
-  rpc ServerClock(ServerClockRequestPB)
-    returns (ServerClockResponsePB);
+  rpc ServerClock(ServerClockRequestPB) returns (ServerClockResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 
+  // TODO(dralves): seems unused. can we remove this?
   rpc SetServerWallClockForTests(SetServerWallClockForTestsRequestPB)
     returns (SetServerWallClockForTestsResponsePB);
 
-  rpc GetStatus(GetStatusRequestPB)
-    returns (GetStatusResponsePB);
+  rpc GetStatus(GetStatusRequestPB) returns (GetStatusResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 }

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_copy.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy.proto b/src/kudu/tserver/tablet_copy.proto
index 5959ef4..1e31a5c 100644
--- a/src/kudu/tserver/tablet_copy.proto
+++ b/src/kudu/tserver/tablet_copy.proto
@@ -27,6 +27,8 @@ import "kudu/util/pb_util.proto";
 
 // RaftConfig tablet copy RPC calls.
 service TabletCopyService {
+  option (kudu.rpc.default_authz_method) = "AuthorizeServiceUser";
+
   // Establish a tablet copy session.
   rpc BeginTabletCopySession(BeginTabletCopySessionRequestPB)
       returns (BeginTabletCopySessionResponsePB);

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_copy_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service.cc b/src/kudu/tserver/tablet_copy_service.cc
index d3ff41b..c405470 100644
--- a/src/kudu/tserver/tablet_copy_service.cc
+++ b/src/kudu/tserver/tablet_copy_service.cc
@@ -28,6 +28,7 @@
 #include "kudu/gutil/strings/substitute.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/rpc/rpc_context.h"
+#include "kudu/server/server_base.h"
 #include "kudu/tserver/tablet_copy_source_session.h"
 #include "kudu/tserver/tablet_peer_lookup.h"
 #include "kudu/tablet/tablet_peer.h"
@@ -61,20 +62,22 @@ DEFINE_double(fault_crash_on_handle_tc_fetch_data, 0.0,
               "(For testing only!)");
 TAG_FLAG(fault_crash_on_handle_tc_fetch_data, unsafe);
 
+using strings::Substitute;
+
 namespace kudu {
-namespace tserver {
 
 using crc::Crc32c;
-using strings::Substitute;
+using server::ServerBase;
 using tablet::TabletPeer;
 
+namespace tserver {
+
 TabletCopyServiceImpl::TabletCopyServiceImpl(
-    FsManager* fs_manager,
-    TabletPeerLookupIf* tablet_peer_lookup,
-    const scoped_refptr<MetricEntity>& metric_entity,
-    const scoped_refptr<rpc::ResultTracker>& result_tracker)
-    : TabletCopyServiceIf(metric_entity, result_tracker),
-      fs_manager_(CHECK_NOTNULL(fs_manager)),
+    ServerBase* server,
+    TabletPeerLookupIf* tablet_peer_lookup)
+    : TabletCopyServiceIf(server->metric_entity(), server->result_tracker()),
+      server_(server),
+      fs_manager_(CHECK_NOTNULL(server->fs_manager())),
       tablet_peer_lookup_(CHECK_NOTNULL(tablet_peer_lookup)),
       shutdown_latch_(1) {
   CHECK_OK(Thread::Create("tablet-copy", "tc-session-exp",
@@ -82,6 +85,12 @@ TabletCopyServiceImpl::TabletCopyServiceImpl(
                           &session_expiration_thread_));
 }
 
+bool TabletCopyServiceImpl::AuthorizeServiceUser(const google::protobuf::Message* /*req*/,
+                                                 google::protobuf::Message* /*resp*/,
+                                                 rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::SERVICE_USER);
+}
+
 void TabletCopyServiceImpl::BeginTabletCopySession(
         const BeginTabletCopySessionRequestPB* req,
         BeginTabletCopySessionResponsePB* resp,

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_copy_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_copy_service.h b/src/kudu/tserver/tablet_copy_service.h
index 58602eb..8ece47d 100644
--- a/src/kudu/tserver/tablet_copy_service.h
+++ b/src/kudu/tserver/tablet_copy_service.h
@@ -33,6 +33,10 @@
 namespace kudu {
 class FsManager;
 
+namespace server {
+class ServerBase;
+} // namespace server
+
 namespace log {
 class ReadableLogSegment;
 } // namespace log
@@ -44,10 +48,12 @@ class TabletPeerLookupIf;
 
 class TabletCopyServiceImpl : public TabletCopyServiceIf {
  public:
-  TabletCopyServiceImpl(FsManager* fs_manager,
-                             TabletPeerLookupIf* tablet_peer_lookup,
-                             const scoped_refptr<MetricEntity>& metric_entity,
-                             const scoped_refptr<rpc::ResultTracker>& result_tracker);
+  TabletCopyServiceImpl(server::ServerBase* server,
+                        TabletPeerLookupIf* tablet_peer_lookup);
+
+  bool AuthorizeServiceUser(const google::protobuf::Message* req,
+                            google::protobuf::Message* resp,
+                            rpc::RpcContext* rpc) override;
 
   virtual void BeginTabletCopySession(const BeginTabletCopySessionRequestPB* req,
                                            BeginTabletCopySessionResponsePB* resp,
@@ -99,6 +105,7 @@ class TabletCopyServiceImpl : public TabletCopyServiceIf {
                             const string& message,
                             const Status& s);
 
+  server::ServerBase* server_;
   FsManager* fs_manager_;
   TabletPeerLookupIf* tablet_peer_lookup_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_server.cc b/src/kudu/tserver/tablet_server.cc
index a26c94b..ea8e1b1 100644
--- a/src/kudu/tserver/tablet_server.cc
+++ b/src/kudu/tserver/tablet_server.cc
@@ -111,11 +111,9 @@ Status TabletServer::Start() {
 
   gscoped_ptr<ServiceIf> ts_service(new TabletServiceImpl(this));
   gscoped_ptr<ServiceIf> admin_service(new TabletServiceAdminImpl(this));
-  gscoped_ptr<ServiceIf> consensus_service(new ConsensusServiceImpl(metric_entity(),
-                                                                    result_tracker(),
-                                                                    tablet_manager_.get()));
+  gscoped_ptr<ServiceIf> consensus_service(new ConsensusServiceImpl(this, tablet_manager_.get()));
   gscoped_ptr<ServiceIf> tablet_copy_service(new TabletCopyServiceImpl(
-      fs_manager_.get(), tablet_manager_.get(), metric_entity(), result_tracker()));
+      this, tablet_manager_.get()));
 
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(ts_service)));
   RETURN_NOT_OK(ServerBase::RegisterService(std::move(admin_service)));

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.cc b/src/kudu/tserver/tablet_service.cc
index 477d8ea..69ee7ab 100644
--- a/src/kudu/tserver/tablet_service.cc
+++ b/src/kudu/tserver/tablet_service.cc
@@ -117,9 +117,8 @@ using kudu::consensus::StartTabletCopyRequestPB;
 using kudu::consensus::StartTabletCopyResponsePB;
 using kudu::consensus::VoteRequestPB;
 using kudu::consensus::VoteResponsePB;
-using kudu::rpc::ResultTracker;
 using kudu::rpc::RpcContext;
-using kudu::server::HybridClock;
+using kudu::server::ServerBase;
 using kudu::tablet::AlterSchemaTransactionState;
 using kudu::tablet::Tablet;
 using kudu::tablet::TabletPeer;
@@ -513,8 +512,22 @@ TabletServiceImpl::TabletServiceImpl(TabletServer* server)
     server_(server) {
 }
 
-void TabletServiceImpl::Ping(const PingRequestPB* req,
-                             PingResponsePB* resp,
+bool TabletServiceImpl::AuthorizeClientOrServiceUser(const google::protobuf::Message* /*req*/,
+                                                 google::protobuf::Message* /*resp*/,
+                                                 rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::USER |
+                            ServerBase::SERVICE_USER);
+}
+
+bool TabletServiceImpl::AuthorizeClient(const google::protobuf::Message* /*req*/,
+                                        google::protobuf::Message* /*resp*/,
+                                        rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::USER);
+}
+
+
+void TabletServiceImpl::Ping(const PingRequestPB* /*req*/,
+                             PingResponsePB* /*resp*/,
                              rpc::RpcContext* context) {
   context->RespondSuccess();
 }
@@ -524,6 +537,12 @@ TabletServiceAdminImpl::TabletServiceAdminImpl(TabletServer* server)
     server_(server) {
 }
 
+bool TabletServiceAdminImpl::AuthorizeServiceUser(const google::protobuf::Message* /*req*/,
+                                                  google::protobuf::Message* /*resp*/,
+                                                  rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::SERVICE_USER);
+}
+
 void TabletServiceAdminImpl::AlterSchema(const AlterSchemaRequestPB* req,
                                          AlterSchemaResponsePB* resp,
                                          rpc::RpcContext* context) {
@@ -782,19 +801,24 @@ void TabletServiceImpl::Write(const WriteRequestPB* req,
                          TabletServerErrorPB::UNKNOWN_ERROR,
                          context);
   }
-  return;
 }
 
-ConsensusServiceImpl::ConsensusServiceImpl(const scoped_refptr<MetricEntity>& metric_entity,
-                                           const scoped_refptr<ResultTracker>& result_tracker,
+ConsensusServiceImpl::ConsensusServiceImpl(ServerBase* server,
                                            TabletPeerLookupIf* tablet_manager)
-  : ConsensusServiceIf(metric_entity, result_tracker),
-    tablet_manager_(tablet_manager) {
+    : ConsensusServiceIf(server->metric_entity(), server->result_tracker()),
+      server_(server),
+      tablet_manager_(tablet_manager) {
 }
 
 ConsensusServiceImpl::~ConsensusServiceImpl() {
 }
 
+bool ConsensusServiceImpl::AuthorizeServiceUser(const google::protobuf::Message* /*req*/,
+                                                google::protobuf::Message* /*resp*/,
+                                                rpc::RpcContext* rpc) {
+  return server_->Authorize(rpc, ServerBase::SUPER_USER | ServerBase::SERVICE_USER);
+}
+
 void ConsensusServiceImpl::UpdateConsensus(const ConsensusRequestPB* req,
                                            ConsensusResponsePB* resp,
                                            rpc::RpcContext* context) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tablet_service.h
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tablet_service.h b/src/kudu/tserver/tablet_service.h
index 3cdc95e..3290aa8 100644
--- a/src/kudu/tserver/tablet_service.h
+++ b/src/kudu/tserver/tablet_service.h
@@ -32,6 +32,10 @@ class Schema;
 class Status;
 class Timestamp;
 
+namespace server {
+class ServerBase;
+} // namespace server
+
 namespace tablet {
 class Tablet;
 class TabletPeer;
@@ -48,6 +52,14 @@ class TabletServiceImpl : public TabletServerServiceIf {
  public:
   explicit TabletServiceImpl(TabletServer* server);
 
+  bool AuthorizeClient(const google::protobuf::Message* req,
+                       google::protobuf::Message* resp,
+                       rpc::RpcContext* rpc) override;
+
+  bool AuthorizeClientOrServiceUser(const google::protobuf::Message* req,
+                                    google::protobuf::Message* resp,
+                                    rpc::RpcContext* rpc) override;
+
   virtual void Ping(const PingRequestPB* req,
                     PingResponsePB* resp,
                     rpc::RpcContext* context) OVERRIDE;
@@ -103,6 +115,11 @@ class TabletServiceImpl : public TabletServerServiceIf {
 class TabletServiceAdminImpl : public TabletServerAdminServiceIf {
  public:
   explicit TabletServiceAdminImpl(TabletServer* server);
+
+  bool AuthorizeServiceUser(const google::protobuf::Message* req,
+                            google::protobuf::Message* resp,
+                            rpc::RpcContext* rpc) override;
+
   virtual void CreateTablet(const CreateTabletRequestPB* req,
                             CreateTabletResponsePB* resp,
                             rpc::RpcContext* context) OVERRIDE;
@@ -121,12 +138,15 @@ class TabletServiceAdminImpl : public TabletServerAdminServiceIf {
 
 class ConsensusServiceImpl : public consensus::ConsensusServiceIf {
  public:
-  ConsensusServiceImpl(const scoped_refptr<MetricEntity>& metric_entity,
-                       const scoped_refptr<rpc::ResultTracker>& result_tracker,
-                       TabletPeerLookupIf* tablet_manager_);
+  ConsensusServiceImpl(server::ServerBase* server,
+                       TabletPeerLookupIf* tablet_manager);
 
   virtual ~ConsensusServiceImpl();
 
+  bool AuthorizeServiceUser(const google::protobuf::Message* req,
+                            google::protobuf::Message* resp,
+                            rpc::RpcContext* rpc) override;
+
   virtual void UpdateConsensus(const consensus::ConsensusRequestPB *req,
                                consensus::ConsensusResponsePB *resp,
                                rpc::RpcContext *context) OVERRIDE;
@@ -164,6 +184,7 @@ class ConsensusServiceImpl : public consensus::ConsensusServiceIf {
                                     rpc::RpcContext* context) OVERRIDE;
 
  private:
+  server::ServerBase* server_;
   TabletPeerLookupIf* tablet_manager_;
 };
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tserver_admin.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver_admin.proto b/src/kudu/tserver/tserver_admin.proto
index dcbcbde..45e9839 100644
--- a/src/kudu/tserver/tserver_admin.proto
+++ b/src/kudu/tserver/tserver_admin.proto
@@ -20,6 +20,7 @@ option java_package = "org.apache.kudu.tserver";
 
 import "kudu/common/common.proto";
 import "kudu/consensus/metadata.proto";
+import "kudu/rpc/rpc_header.proto";
 import "kudu/tablet/metadata.proto";
 import "kudu/tserver/tserver.proto";
 
@@ -123,6 +124,8 @@ enum TSTabletManagerStatePB {
 }
 
 service TabletServerAdminService {
+  option (kudu.rpc.default_authz_method) = "AuthorizeServiceUser";
+
   // Create a new, empty tablet with the specified parameters. Only used for
   // brand-new tablets, not for "moves".
   rpc CreateTablet(CreateTabletRequestPB) returns (CreateTabletResponsePB);

http://git-wip-us.apache.org/repos/asf/kudu/blob/758cf8b8/src/kudu/tserver/tserver_service.proto
----------------------------------------------------------------------
diff --git a/src/kudu/tserver/tserver_service.proto b/src/kudu/tserver/tserver_service.proto
index 665a073..1ed3249 100644
--- a/src/kudu/tserver/tserver_service.proto
+++ b/src/kudu/tserver/tserver_service.proto
@@ -22,21 +22,35 @@ import "kudu/rpc/rpc_header.proto";
 import "kudu/tserver/tserver.proto";
 
 service TabletServerService {
+  // Set the default authz method to something invalid, so that if
+  // we forget to set the option on a new RPC call, we'll get a build
+  // failure.
+  option (kudu.rpc.default_authz_method) = "MUST_SET_AUTHZ_PER_RPC";
 
-  rpc Ping(PingRequestPB) returns (PingResponsePB);
+  rpc Ping(PingRequestPB) returns (PingResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClientOrServiceUser";
+  }
   rpc Write(WriteRequestPB) returns (WriteResponsePB)  {
     option (kudu.rpc.track_rpc_result) = true;
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc Scan(ScanRequestPB) returns (ScanResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc ScannerKeepAlive(ScannerKeepAliveRequestPB) returns (ScannerKeepAliveResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
+  rpc ListTablets(ListTabletsRequestPB) returns (ListTabletsResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
   }
-  rpc Scan(ScanRequestPB) returns (ScanResponsePB);
-  rpc ScannerKeepAlive(ScannerKeepAliveRequestPB) returns (ScannerKeepAliveResponsePB);
-  rpc ListTablets(ListTabletsRequestPB) returns (ListTabletsResponsePB);
 
   // Run full-scan data checksum on a tablet to verify data integrity.
   //
   // TODO: Consider refactoring this as a scan that runs a checksum aggregation
   // function.
-  rpc Checksum(ChecksumRequestPB)
-      returns (ChecksumResponsePB);
+  rpc Checksum(ChecksumRequestPB) returns (ChecksumResponsePB) {
+    option (kudu.rpc.authz_method) = "AuthorizeClient";
+  }
 }
 
 message ChecksumRequestPB {