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/01/23 18:17:58 UTC

[2/2] kudu git commit: TLS-negotiation [2/n]: remove SASL ANONYMOUS mechanism

TLS-negotiation [2/n]: remove SASL ANONYMOUS mechanism

ANONYMOUS has not been enabled on any released server version going back
all the way to 1.0. Adding it back an enabling it would be a backwards
compatible change, so there is no harm in removing the currently unused
code.

Change-Id: I5badb63bd8e416fbcbdc7bb6f8fa1e3ec9e6e036
Reviewed-on: http://gerrit.cloudera.org:8080/5756
Tested-by: Kudu Jenkins
Reviewed-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/73dcce64
Tree: http://git-wip-us.apache.org/repos/asf/kudu/tree/73dcce64
Diff: http://git-wip-us.apache.org/repos/asf/kudu/diff/73dcce64

Branch: refs/heads/master
Commit: 73dcce642e85518d619ec3a4e9efcbef87a54c5c
Parents: 13014ca
Author: Dan Burkert <da...@apache.org>
Authored: Wed Dec 14 20:11:45 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Mon Jan 23 18:17:38 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/connection.cc    |  2 --
 src/kudu/rpc/sasl_client.cc   | 26 +++++++++-----------------
 src/kudu/rpc/sasl_client.h    |  4 ----
 src/kudu/rpc/sasl_common.cc   |  5 -----
 src/kudu/rpc/sasl_common.h    |  2 --
 src/kudu/rpc/sasl_helper.cc   | 11 -----------
 src/kudu/rpc/sasl_helper.h    |  7 -------
 src/kudu/rpc/sasl_rpc-test.cc | 33 ++++++---------------------------
 src/kudu/rpc/sasl_server.cc   |  5 -----
 src/kudu/rpc/sasl_server.h    |  4 ----
 10 files changed, 15 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index 8506b99..c7a87e9 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -656,8 +656,6 @@ Status Connection::InitSaslClient() {
                             << "servers requiring Kerberos authentication.";
     }
   }
-  // TODO(todd): we dont seem to ever use ANONYMOUS. Should we remove it?
-  RETURN_NOT_OK(sasl_client().EnableAnonymous());
   RETURN_NOT_OK(sasl_client().EnablePlain(user_credentials().real_user(), ""));
   RETURN_NOT_OK(sasl_client().Init(kSaslProtoName));
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_client.cc b/src/kudu/rpc/sasl_client.cc
index 83b1f97..379aec8 100644
--- a/src/kudu/rpc/sasl_client.cc
+++ b/src/kudu/rpc/sasl_client.cc
@@ -101,11 +101,6 @@ SaslClient::SaslClient(string app_name, Socket* socket)
   callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
 }
 
-Status SaslClient::EnableAnonymous() {
-  DCHECK_EQ(client_state_, SaslNegotiationState::NEW);
-  return helper_.EnableAnonymous();
-}
-
 Status SaslClient::EnablePlain(const string& user, const string& pass) {
   DCHECK_EQ(client_state_, SaslNegotiationState::NEW);
   RETURN_NOT_OK(helper_.EnablePlain());
@@ -207,7 +202,6 @@ Status SaslClient::Negotiate() {
 
   // We set nego_ok_ = true when the SASL library returns SASL_OK to us.
   // We set nego_response_expected_ = true each time we send a request to the server.
-  // When using ANONYMOUS, we get SASL_OK back immediately but still send INITIATE to the server.
   while (!nego_ok_ || nego_response_expected_) {
     ResponseHeader header;
     Slice param_buf;
@@ -457,9 +451,13 @@ int SaslClient::GetOptionCb(const char* plugin_name, const char* option,
   return helper_.GetOptionCb(plugin_name, option, result, len);
 }
 
-// Used for PLAIN and ANONYMOUS.
+// Used for PLAIN.
 // SASL callback for SASL_CB_USER, SASL_CB_AUTHNAME, SASL_CB_LANGUAGE
 int SaslClient::SimpleCb(int id, const char** result, unsigned* len) {
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "SASL Client: Simple callback called, but PLAIN auth is not enabled";
+    return SASL_FAIL;
+  }
   if (PREDICT_FALSE(result == nullptr)) {
     LOG(DFATAL) << "SASL Client: result outparam is NULL";
     return SASL_BADPARAM;
@@ -469,19 +467,13 @@ int SaslClient::SimpleCb(int id, const char** result, unsigned* len) {
     // For impersonation, USER is the impersonated user, AUTHNAME is the "sudoer".
     case SASL_CB_USER:
       TRACE("SASL Client: callback for SASL_CB_USER");
-      if (helper_.IsPlainEnabled()) {
-        *result = plain_auth_user_.c_str();
-        if (len != nullptr) *len = plain_auth_user_.length();
-      } else if (helper_.IsAnonymousEnabled()) {
-        *result = nullptr;
-      }
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
       break;
     case SASL_CB_AUTHNAME:
       TRACE("SASL Client: callback for SASL_CB_AUTHNAME");
-      if (helper_.IsPlainEnabled()) {
-        *result = plain_auth_user_.c_str();
-        if (len != nullptr) *len = plain_auth_user_.length();
-      }
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
       break;
     case SASL_CB_LANGUAGE:
       LOG(DFATAL) << "SASL Client: Unable to handle SASL callback type SASL_CB_LANGUAGE"

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_client.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_client.h b/src/kudu/rpc/sasl_client.h
index f8478c3..e35ff2a 100644
--- a/src/kudu/rpc/sasl_client.h
+++ b/src/kudu/rpc/sasl_client.h
@@ -48,10 +48,6 @@ class SaslClient {
   // Does not take ownership of the socket indicated by the fd.
   SaslClient(string app_name, Socket* socket);
 
-  // Enable ANONYMOUS authentication.
-  // Must be called after Init().
-  Status EnableAnonymous();
-
   // Enable PLAIN authentication.
   // Must be called after Init().
   Status EnablePlain(const string& user, const string& pass);

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.cc b/src/kudu/rpc/sasl_common.cc
index 4e9aed5..359dc80 100644
--- a/src/kudu/rpc/sasl_common.cc
+++ b/src/kudu/rpc/sasl_common.cc
@@ -39,7 +39,6 @@ using std::set;
 namespace kudu {
 namespace rpc {
 
-const char* const kSaslMechAnonymous = "ANONYMOUS";
 const char* const kSaslMechPlain = "PLAIN";
 const char* const kSaslMechGSSAPI = "GSSAPI";
 
@@ -380,9 +379,6 @@ sasl_callback_t SaslBuildCallback(int id, int (*proc)(void), void* context) {
 }
 
 SaslMechanism::Type SaslMechanism::value_of(const string& mech) {
-  if (boost::iequals(mech, "ANONYMOUS")) {
-    return ANONYMOUS;
-  }
   if (boost::iequals(mech, "PLAIN")) {
     return PLAIN;
   }
@@ -394,7 +390,6 @@ SaslMechanism::Type SaslMechanism::value_of(const string& mech) {
 
 const char* SaslMechanism::name_of(SaslMechanism::Type val) {
   switch (val) {
-    case ANONYMOUS: return "ANONYMOUS";
     case PLAIN: return "PLAIN";
     case GSSAPI: return "GSSAPI";
     default:

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_common.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.h b/src/kudu/rpc/sasl_common.h
index cf1ee27..419fc5f 100644
--- a/src/kudu/rpc/sasl_common.h
+++ b/src/kudu/rpc/sasl_common.h
@@ -36,7 +36,6 @@ namespace rpc {
 using std::string;
 
 // Constants
-extern const char* const kSaslMechAnonymous;
 extern const char* const kSaslMechPlain;
 extern const char* const kSaslMechGSSAPI;
 
@@ -102,7 +101,6 @@ struct SaslNegotiationState {
 struct SaslMechanism {
   enum Type {
     INVALID,
-    ANONYMOUS,
     PLAIN,
     GSSAPI
   };

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_helper.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_helper.cc b/src/kudu/rpc/sasl_helper.cc
index 06fc82c..60be7a6 100644
--- a/src/kudu/rpc/sasl_helper.cc
+++ b/src/kudu/rpc/sasl_helper.cc
@@ -46,7 +46,6 @@ using google::protobuf::MessageLite;
 SaslHelper::SaslHelper(PeerType peer_type)
   : peer_type_(peer_type),
     conn_header_exchanged_(false),
-    anonymous_enabled_(false),
     plain_enabled_(false),
     gssapi_enabled_(false) {
   tag_ = (peer_type_ == SERVER) ? "Sasl Server" : "Sasl Client";
@@ -124,16 +123,6 @@ int SaslHelper::GetOptionCb(const char* plugin_name, const char* option,
   return SASL_FAIL;
 }
 
-Status SaslHelper::EnableAnonymous() {
-  RETURN_NOT_OK(EnableMechanism(kSaslMechAnonymous));
-  anonymous_enabled_ = true;
-  return Status::OK();
-}
-
-bool SaslHelper::IsAnonymousEnabled() const {
-  return anonymous_enabled_;
-}
-
 Status SaslHelper::EnablePlain() {
   RETURN_NOT_OK(EnableMechanism(kSaslMechPlain));
   plain_enabled_ = true;

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_helper.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_helper.h b/src/kudu/rpc/sasl_helper.h
index 408b488..45adbe3 100644
--- a/src/kudu/rpc/sasl_helper.h
+++ b/src/kudu/rpc/sasl_helper.h
@@ -84,12 +84,6 @@ class SaslHelper {
   // Implements the client_mech_list / mech_list callbacks.
   int GetOptionCb(const char* plugin_name, const char* option, const char** result, unsigned* len);
 
-  // Enable the ANONYMOUS SASL mechanism.
-  Status EnableAnonymous();
-
-  // Check for the ANONYMOUS SASL mechanism.
-  bool IsAnonymousEnabled() const;
-
   // Enable the PLAIN SASL mechanism.
   Status EnablePlain();
 
@@ -125,7 +119,6 @@ class SaslHelper {
   std::set<string> mechs_;    // Active mechanisms.
   mutable string mech_list_;  // Mechanism list string returned by callbacks.
 
-  bool anonymous_enabled_;
   bool plain_enabled_;
   bool gssapi_enabled_;
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_rpc-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_rpc-test.cc b/src/kudu/rpc/sasl_rpc-test.cc
index 7aecbb7..6220e72 100644
--- a/src/kudu/rpc/sasl_rpc-test.cc
+++ b/src/kudu/rpc/sasl_rpc-test.cc
@@ -73,10 +73,10 @@ class TestSaslRpc : public RpcTestBase {
 // Test basic initialization of the objects.
 TEST_F(TestSaslRpc, TestBasicInit) {
   SaslServer server(kSaslAppName, nullptr);
-  server.EnableAnonymous();
+  server.EnablePlain();
   ASSERT_OK(server.Init(kSaslAppName));
   SaslClient client(kSaslAppName, nullptr);
-  client.EnableAnonymous();
+  client.EnablePlain("test", "test");
   ASSERT_OK(client.Init(kSaslAppName));
 }
 
@@ -123,27 +123,6 @@ static void RunNegotiationTest(const SocketCallable& server_runner,
 
 ////////////////////////////////////////////////////////////////////////////////
 
-static void RunAnonNegotiationServer(Socket* conn) {
-  SaslServer sasl_server(kSaslAppName, conn);
-  CHECK_OK(sasl_server.EnableAnonymous());
-  CHECK_OK(sasl_server.Init(kSaslAppName));
-  CHECK_OK(sasl_server.Negotiate());
-}
-
-static void RunAnonNegotiationClient(Socket* conn) {
-  SaslClient sasl_client(kSaslAppName, conn);
-  CHECK_OK(sasl_client.EnableAnonymous());
-  CHECK_OK(sasl_client.Init(kSaslAppName));
-  CHECK_OK(sasl_client.Negotiate());
-}
-
-// Test SASL negotiation using the ANONYMOUS mechanism over a socket.
-TEST_F(TestSaslRpc, TestAnonNegotiation) {
-  RunNegotiationTest(RunAnonNegotiationServer, RunAnonNegotiationClient);
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
 static void RunPlainNegotiationServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn);
   CHECK_OK(sasl_server.EnablePlain());
@@ -430,7 +409,7 @@ TEST_F(TestSaslRpc, TestPreflight) {
 
 static void RunTimeoutExpectingServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn);
-  CHECK_OK(sasl_server.EnableAnonymous());
+  CHECK_OK(sasl_server.EnablePlain());
   CHECK_OK(sasl_server.Init(kSaslAppName));
   Status s = sasl_server.Negotiate();
   ASSERT_TRUE(s.IsNetworkError()) << "Expected client to time out and close the connection. Got: "
@@ -439,7 +418,7 @@ static void RunTimeoutExpectingServer(Socket* conn) {
 
 static void RunTimeoutNegotiationClient(Socket* sock) {
   SaslClient sasl_client(kSaslAppName, sock);
-  CHECK_OK(sasl_client.EnableAnonymous());
+  CHECK_OK(sasl_client.EnablePlain("test", "test"));
   CHECK_OK(sasl_client.Init(kSaslAppName));
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   sasl_client.set_deadline(deadline);
@@ -457,7 +436,7 @@ TEST_F(TestSaslRpc, TestClientTimeout) {
 
 static void RunTimeoutNegotiationServer(Socket* sock) {
   SaslServer sasl_server(kSaslAppName, sock);
-  CHECK_OK(sasl_server.EnableAnonymous());
+  CHECK_OK(sasl_server.EnablePlain());
   CHECK_OK(sasl_server.Init(kSaslAppName));
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   sasl_server.set_deadline(deadline);
@@ -468,7 +447,7 @@ static void RunTimeoutNegotiationServer(Socket* sock) {
 
 static void RunTimeoutExpectingClient(Socket* conn) {
   SaslClient sasl_client(kSaslAppName, conn);
-  CHECK_OK(sasl_client.EnableAnonymous());
+  CHECK_OK(sasl_client.EnablePlain("test", "test"));
   CHECK_OK(sasl_client.Init(kSaslAppName));
   Status s = sasl_client.Negotiate();
   ASSERT_TRUE(s.IsNetworkError()) << "Expected server to time out and close the connection. Got: "

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.cc b/src/kudu/rpc/sasl_server.cc
index 6b3560d..6dae445 100644
--- a/src/kudu/rpc/sasl_server.cc
+++ b/src/kudu/rpc/sasl_server.cc
@@ -65,11 +65,6 @@ SaslServer::SaslServer(string app_name, Socket* socket)
   callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
 }
 
-Status SaslServer::EnableAnonymous() {
-  DCHECK_EQ(server_state_, SaslNegotiationState::NEW);
-  return helper_.EnableAnonymous();
-}
-
 Status SaslServer::EnablePlain() {
   DCHECK_EQ(server_state_, SaslNegotiationState::NEW);
   RETURN_NOT_OK(helper_.EnablePlain());

http://git-wip-us.apache.org/repos/asf/kudu/blob/73dcce64/src/kudu/rpc/sasl_server.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.h b/src/kudu/rpc/sasl_server.h
index f26f229..7cbf6fd 100644
--- a/src/kudu/rpc/sasl_server.h
+++ b/src/kudu/rpc/sasl_server.h
@@ -46,10 +46,6 @@ class SaslServer {
   // Does not take ownership of 'socket'.
   SaslServer(string app_name, Socket* socket);
 
-  // Enable ANONYMOUS authentication.
-  // Must be called after Init().
-  Status EnableAnonymous();
-
   // Enable PLAIN authentication.
   // Despite PLAIN authentication taking a username and password, we disregard
   // the password and use this as a "unauthenticated" mode.