You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by to...@apache.org on 2016/10/29 00:18:54 UTC

[1/3] kudu git commit: Remove unused code for checking PLAIN authentication

Repository: kudu
Updated Branches:
  refs/heads/master 316b5f75d -> a1a8eef23


Remove unused code for checking PLAIN authentication

This removes the 'AuthStore' abstraction which theoretically supported
checking username/password authentication. In practice, we have no plans
to implement our own username/password authentication any time soon, so
this extra code just confuses the matter.

Additionally, this deprecates the user information that's passed as part
of the 'Connection Context'. The client still sends it, to satisfy older
servers, but the server ignores it and instead determines user
information from the SASL handshake.

Change-Id: Ie960fae30fe573b859f7ef0e27d656faac50d4c2
Reviewed-on: http://gerrit.cloudera.org:8080/4874
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: 732a5052acadc19b999553b4a96979f8801d5963
Parents: 316b5f7
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Oct 27 17:35:12 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Oct 29 00:00:25 2016 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/TabletClient.java    |  6 +-
 src/kudu/rpc/CMakeLists.txt                     |  1 -
 src/kudu/rpc/auth_store.cc                      | 61 -----------------
 src/kudu/rpc/auth_store.h                       | 71 --------------------
 src/kudu/rpc/connection.cc                      |  9 +--
 src/kudu/rpc/negotiation.cc                     | 37 ++++------
 src/kudu/rpc/outbound_call.cc                   | 30 +--------
 src/kudu/rpc/outbound_call.h                    | 13 ----
 src/kudu/rpc/rpc-test-base.h                    |  7 +-
 src/kudu/rpc/rpc_header.proto                   |  7 +-
 src/kudu/rpc/sasl_common.cc                     | 10 +++
 src/kudu/rpc/sasl_common.h                      |  1 +
 src/kudu/rpc/sasl_rpc-test.cc                   | 35 +---------
 src/kudu/rpc/sasl_server.cc                     | 21 ++----
 src/kudu/rpc/sasl_server.h                      | 11 ++-
 15 files changed, 54 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
index 5ff5e98..9f13fb2 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/TabletClient.java
@@ -812,10 +812,12 @@ public class TabletClient extends ReplayingDecoder<VoidEnum> {
 
   private ChannelBuffer header() {
     RpcHeader.ConnectionContextPB.Builder builder = RpcHeader.ConnectionContextPB.newBuilder();
+
+    // The UserInformationPB is deprecated, but used by servers prior to Kudu 1.1.
     RpcHeader.UserInformationPB.Builder userBuilder = RpcHeader.UserInformationPB.newBuilder();
-    userBuilder.setEffectiveUser(SecureRpcHelper.USER_AND_PASSWORD); // TODO set real user
+    userBuilder.setEffectiveUser(SecureRpcHelper.USER_AND_PASSWORD);
     userBuilder.setRealUser(SecureRpcHelper.USER_AND_PASSWORD);
-    builder.setUserInfo(userBuilder.build());
+    builder.setDEPRECATEDUserInfo(userBuilder.build());
     RpcHeader.ConnectionContextPB pb = builder.build();
     RpcHeader.RequestHeader header = RpcHeader.RequestHeader.newBuilder().setCallId
         (CONNECTION_CTX_CALL_ID).build();

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/CMakeLists.txt b/src/kudu/rpc/CMakeLists.txt
index 179eabf..6d10586 100644
--- a/src/kudu/rpc/CMakeLists.txt
+++ b/src/kudu/rpc/CMakeLists.txt
@@ -42,7 +42,6 @@ ADD_EXPORTABLE_LIBRARY(rpc_introspection_proto
 ### RPC library
 set(KRPC_SRCS
     acceptor_pool.cc
-    auth_store.cc
     blocking_ops.cc
     outbound_call.cc
     connection.cc

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/auth_store.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/auth_store.cc b/src/kudu/rpc/auth_store.cc
deleted file mode 100644
index ec28b54..0000000
--- a/src/kudu/rpc/auth_store.cc
+++ /dev/null
@@ -1,61 +0,0 @@
-// 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/rpc/auth_store.h"
-
-#include <string>
-#include <unordered_map>
-
-#include "kudu/util/status.h"
-
-namespace kudu {
-namespace rpc {
-
-AuthStore::AuthStore() {
-}
-
-AuthStore::~AuthStore() {
-}
-
-Status AuthStore::Add(const string& user, const string& pass) {
-  user_cred_map_[user] = pass;
-  return Status::OK();
-}
-
-Status AuthStore::Authenticate(const string& user, const string& pass) const {
-  auto it = user_cred_map_.find(user);
-  if (it == user_cred_map_.end()) {
-    return Status::NotFound("Unknown user", user);
-  }
-  if (it->second != pass) {
-    return Status::NotAuthorized("Invalid credentials for user", user);
-  }
-  return Status::OK();
-}
-
-DummyAuthStore::DummyAuthStore() {
-}
-
-DummyAuthStore::~DummyAuthStore() {
-}
-
-Status DummyAuthStore::Authenticate(const string& user, const string& password) const {
-  return Status::OK();
-}
-
-} // namespace rpc
-} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/auth_store.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/auth_store.h b/src/kudu/rpc/auth_store.h
deleted file mode 100644
index b6e937f..0000000
--- a/src/kudu/rpc/auth_store.h
+++ /dev/null
@@ -1,71 +0,0 @@
-// 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.
-
-#ifndef KUDU_RPC_AUTH_STORE_H
-#define KUDU_RPC_AUTH_STORE_H
-
-#include <unordered_map>
-#include <string>
-
-#include "kudu/gutil/macros.h"
-
-namespace kudu {
-
-class Status;
-
-namespace rpc {
-
-using std::string;
-using std::unordered_map;
-
-// This class stores username / password pairs in memory for use in PLAIN SASL auth.
-// Add() is NOT thread safe.
-// Authenticate() is safe to call from multiple threads.
-class AuthStore {
- public:
-  AuthStore();
-  virtual ~AuthStore();
-
-  // Add user to the auth store.
-  virtual Status Add(const string& user, const string& password);
-
-  // Validate whether user/password combination exists in auth store.
-  // Returns OK if the user has valid credentials.
-  // Returns NotFound if the user is not found.
-  // Returns NotAuthorized if the password is incorrect.
-  virtual Status Authenticate(const string& user, const string& password) const;
-
- private:
-  unordered_map<string, string> user_cred_map_;
-
-  DISALLOW_COPY_AND_ASSIGN(AuthStore);
-};
-
-// This class simply allows anybody through.
-class DummyAuthStore : public AuthStore {
- public:
-  DummyAuthStore();
-  virtual ~DummyAuthStore();
-
-  // Always returns OK
-  virtual Status Authenticate(const string& user, const string& password) const OVERRIDE;
-};
-
-} // namespace rpc
-} // namespace kudu
-
-#endif // KUDU_RPC_AUTH_STORE_H

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index 018b894..7c15a5d 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -30,7 +30,6 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/strings/human_readable.h"
 #include "kudu/gutil/strings/substitute.h"
-#include "kudu/rpc/auth_store.h"
 #include "kudu/rpc/rpc_introspection.pb.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/messenger.h"
@@ -621,18 +620,14 @@ std::string Connection::ToString() const {
 
 Status Connection::InitSaslClient() {
   RETURN_NOT_OK(sasl_client().EnableAnonymous());
-  RETURN_NOT_OK(sasl_client().EnablePlain(user_credentials().real_user(),
-                                          user_credentials().password()));
+  RETURN_NOT_OK(sasl_client().EnablePlain(user_credentials().real_user(), ""));
   RETURN_NOT_OK(sasl_client().Init(kSaslProtoName));
   return Status::OK();
 }
 
 Status Connection::InitSaslServer() {
-  // TODO: Do necessary configuration plumbing to enable user authentication.
-  // Right now we just enable PLAIN with a "dummy" auth store, which allows everyone in.
   RETURN_NOT_OK(sasl_server().Init(kSaslProtoName));
-  gscoped_ptr<AuthStore> auth_store(new DummyAuthStore());
-  RETURN_NOT_OK(sasl_server().EnablePlain(std::move(auth_store)));
+  RETURN_NOT_OK(sasl_server().EnablePlain());
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/negotiation.cc b/src/kudu/rpc/negotiation.cc
index afb91e7..be5b504 100644
--- a/src/kudu/rpc/negotiation.cc
+++ b/src/kudu/rpc/negotiation.cc
@@ -63,9 +63,10 @@ static Status SendConnectionContext(Connection* conn, const MonoTime& deadline)
   header.set_call_id(kConnectionContextCallId);
 
   ConnectionContextPB conn_context;
-  conn_context.mutable_user_info()->set_effective_user(conn->user_credentials().effective_user());
-  conn_context.mutable_user_info()->set_real_user(conn->user_credentials().real_user());
-
+  // This field is deprecated but used by servers <Kudu 1.1. Newer server versions ignore
+  // this and use the SASL-provided username instead.
+  conn_context.mutable_deprecated_user_info()->set_real_user(
+      conn->user_credentials().real_user());
   return SendFramedMessageBlocking(conn->socket(), header, conn_context, deadline);
 }
 
@@ -89,29 +90,15 @@ static Status RecvConnectionContext(Connection* conn, const MonoTime& deadline)
   ConnectionContextPB conn_context;
   if (!conn_context.ParseFromArray(param_buf.data(), param_buf.size())) {
     return Status::InvalidArgument("Invalid ConnectionContextPB message, missing fields",
-        conn_context.InitializationErrorString());
+                                   conn_context.InitializationErrorString());
   }
 
-  // Update the fields of our Connection object from the ConnectionContextPB.
-  if (conn_context.has_user_info()) {
-    // Validate real user against SASL impl.
-    if (conn->sasl_server().negotiated_mechanism() == SaslMechanism::PLAIN) {
-      if (conn->sasl_server().authenticated_user() != conn_context.user_info().real_user()) {
-        return Status::NotAuthorized(
-            "ConnectionContextPB specified different real user than sent in SASL negotiation",
-            StringPrintf("\"%s\" vs. \"%s\"",
-                conn_context.user_info().real_user().c_str(),
-                conn->sasl_server().authenticated_user().c_str()));
-      }
-    }
-    conn->mutable_user_credentials()->set_real_user(conn_context.user_info().real_user());
-
-    // TODO: Validate effective user when we implement impersonation.
-    if (conn_context.user_info().has_effective_user()) {
-      conn->mutable_user_credentials()->set_effective_user(
-        conn_context.user_info().effective_user());
-    }
+  if (conn->sasl_server().authenticated_user().empty()) {
+    return Status::NotAuthorized("No user was authenticated");
   }
+
+  conn->mutable_user_credentials()->set_real_user(conn->sasl_server().authenticated_user());
+
   return Status::OK();
 }
 
@@ -253,6 +240,10 @@ void Negotiation::RunNegotiation(const scoped_refptr<Connection>& conn,
       LOG(INFO) << "RPC negotiation tracing enabled. Trace:\n" << msg;
     }
   }
+
+  if (conn->direction() == Connection::SERVER && s.IsNotAuthorized()) {
+    LOG(WARNING) << "Unauthorized connection attempt: " << s.message().ToString();
+  }
   conn->CompleteNegotiation(s);
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/outbound_call.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/outbound_call.cc b/src/kudu/rpc/outbound_call.cc
index 2795e16..a26cbec 100644
--- a/src/kudu/rpc/outbound_call.cc
+++ b/src/kudu/rpc/outbound_call.cc
@@ -320,14 +320,6 @@ void OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
 
 UserCredentials::UserCredentials() {}
 
-bool UserCredentials::has_effective_user() const {
-  return !eff_user_.empty();
-}
-
-void UserCredentials::set_effective_user(const string& eff_user) {
-  eff_user_ = eff_user;
-}
-
 bool UserCredentials::has_real_user() const {
   return !real_user_.empty();
 }
@@ -336,43 +328,25 @@ void UserCredentials::set_real_user(const string& real_user) {
   real_user_ = real_user;
 }
 
-bool UserCredentials::has_password() const {
-  return !password_.empty();
-}
-
-void UserCredentials::set_password(const string& password) {
-  password_ = password;
-}
-
 void UserCredentials::CopyFrom(const UserCredentials& other) {
-  eff_user_ = other.eff_user_;
   real_user_ = other.real_user_;
-  password_ = other.password_;
 }
 
 string UserCredentials::ToString() const {
   // Does not print the password.
-  return StringPrintf("{real_user=%s, eff_user=%s}", real_user_.c_str(), eff_user_.c_str());
+  return StringPrintf("{real_user=%s}", real_user_.c_str());
 }
 
 size_t UserCredentials::HashCode() const {
   size_t seed = 0;
-  if (has_effective_user()) {
-    boost::hash_combine(seed, effective_user());
-  }
   if (has_real_user()) {
     boost::hash_combine(seed, real_user());
   }
-  if (has_password()) {
-    boost::hash_combine(seed, password());
-  }
   return seed;
 }
 
 bool UserCredentials::Equals(const UserCredentials& other) const {
-  return (effective_user() == other.effective_user()
-       && real_user() == other.real_user()
-       && password() == other.password());
+  return real_user() == other.real_user();
 }
 
 ///

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/outbound_call.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/outbound_call.h b/src/kudu/rpc/outbound_call.h
index f17bb34..45d02b2 100644
--- a/src/kudu/rpc/outbound_call.h
+++ b/src/kudu/rpc/outbound_call.h
@@ -61,22 +61,11 @@ class UserCredentials {
  public:
    UserCredentials();
 
-  // Effective user, in cases where impersonation is supported.
-  // If impersonation is not supported, this should be left empty.
-  bool has_effective_user() const;
-  void set_effective_user(const std::string& eff_user);
-  const std::string& effective_user() const { return eff_user_; }
-
   // Real user.
   bool has_real_user() const;
   void set_real_user(const std::string& real_user);
   const std::string& real_user() const { return real_user_; }
 
-  // The real user's password.
-  bool has_password() const;
-  void set_password(const std::string& password);
-  const std::string& password() const { return password_; }
-
   // Copy state from another object to this one.
   void CopyFrom(const UserCredentials& other);
 
@@ -88,9 +77,7 @@ class UserCredentials {
 
  private:
   // Remember to update HashCode() and Equals() when new fields are added.
-  std::string eff_user_;
   std::string real_user_;
-  std::string password_;
 
   DISALLOW_COPY_AND_ASSIGN(UserCredentials);
 };

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/rpc-test-base.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc-test-base.h b/src/kudu/rpc/rpc-test-base.h
index ff675fd..9aafe5d 100644
--- a/src/kudu/rpc/rpc-test-base.h
+++ b/src/kudu/rpc/rpc-test-base.h
@@ -220,11 +220,10 @@ class CalculatorService : public CalculatorServiceIf {
     context->RespondSuccess();
   }
 
-  void WhoAmI(const WhoAmIRequestPB* req, WhoAmIResponsePB* resp, RpcContext* context) override {
+  void WhoAmI(const WhoAmIRequestPB* /*req*/,
+              WhoAmIResponsePB* resp,
+              RpcContext* context) override {
     const UserCredentials& creds = context->user_credentials();
-    if (creds.has_effective_user()) {
-      resp->mutable_credentials()->set_effective_user(creds.effective_user());
-    }
     resp->mutable_credentials()->set_real_user(creds.real_user());
     resp->set_address(context->remote_address().ToString());
     context->RespondSuccess();

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_header.proto b/src/kudu/rpc/rpc_header.proto
index 8adc5e2..aa5abf1 100644
--- a/src/kudu/rpc/rpc_header.proto
+++ b/src/kudu/rpc/rpc_header.proto
@@ -46,7 +46,12 @@ message UserInformationPB {
 message ConnectionContextPB {
   // UserInfo beyond what is determined as part of security handshake
   // at connection time (kerberos, tokens etc).
-  optional UserInformationPB user_info = 2;
+  //
+  // DEPRECATED: No longer used in Kudu 1.1 and later.
+  // The 'real_user' should be taken from the SASL negotiation.
+  // Impersonation (effective user) was never supported, so we'll have
+  // to add that back at some point later.
+  optional UserInformationPB DEPRECATED_user_info = 2;
 }
 
 // Features supported by the RPC system itself.

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/sasl_common.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.cc b/src/kudu/rpc/sasl_common.cc
index 2803a9b..25e883c 100644
--- a/src/kudu/rpc/sasl_common.cc
+++ b/src/kudu/rpc/sasl_common.cc
@@ -319,5 +319,15 @@ SaslMechanism::Type SaslMechanism::value_of(const string& mech) {
   return INVALID;
 }
 
+const char* SaslMechanism::name_of(SaslMechanism::Type val) {
+  switch (val) {
+    case ANONYMOUS: return "ANONYMOUS";
+    case PLAIN: return "PLAIN";
+    case GSSAPI: return "GSSAPI";
+    default:
+      return "INVALID";
+  }
+}
+
 } // namespace rpc
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/sasl_common.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_common.h b/src/kudu/rpc/sasl_common.h
index 4ef657a..4492d82 100644
--- a/src/kudu/rpc/sasl_common.h
+++ b/src/kudu/rpc/sasl_common.h
@@ -104,6 +104,7 @@ struct SaslMechanism {
     GSSAPI
   };
   static Type value_of(const std::string& mech);
+  static const char* name_of(Type val);
 };
 
 } // namespace rpc

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/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 f66314d..b238dff 100644
--- a/src/kudu/rpc/sasl_rpc-test.cc
+++ b/src/kudu/rpc/sasl_rpc-test.cc
@@ -30,7 +30,6 @@
 #include "kudu/gutil/gscoped_ptr.h"
 #include "kudu/gutil/map-util.h"
 #include "kudu/rpc/constants.h"
-#include "kudu/rpc/auth_store.h"
 #include "kudu/rpc/sasl_client.h"
 #include "kudu/rpc/sasl_common.h"
 #include "kudu/rpc/sasl_server.h"
@@ -127,18 +126,16 @@ TEST_F(TestSaslRpc, TestAnonNegotiation) {
 
 static void RunPlainNegotiationServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
-  gscoped_ptr<AuthStore> authstore(new AuthStore());
-  CHECK_OK(authstore->Add("danger", "burrito"));
   CHECK_OK(sasl_server.Init(kSaslAppName));
-  CHECK_OK(sasl_server.EnablePlain(std::move(authstore)));
+  CHECK_OK(sasl_server.EnablePlain());
   CHECK_OK(sasl_server.Negotiate());
   CHECK(ContainsKey(sasl_server.client_features(), APPLICATION_FEATURE_FLAGS));
-  CHECK_EQ("danger", sasl_server.authenticated_user());
+  CHECK_EQ("my-username", sasl_server.authenticated_user());
 }
 
 static void RunPlainNegotiationClient(Socket* conn) {
   SaslClient sasl_client(kSaslAppName, conn->GetFd());
-  CHECK_OK(sasl_client.EnablePlain("danger", "burrito"));
+  CHECK_OK(sasl_client.EnablePlain("my-username", "ignored password"));
   CHECK_OK(sasl_client.Init(kSaslAppName));
   CHECK_OK(sasl_client.Negotiate());
   CHECK(ContainsKey(sasl_client.server_features(), APPLICATION_FEATURE_FLAGS));
@@ -270,32 +267,6 @@ TEST_F(TestSaslRpc, TestGSSAPINegotiation) {
 
 ////////////////////////////////////////////////////////////////////////////////
 
-
-static void RunPlainFailingNegotiationServer(Socket* conn) {
-  SaslServer sasl_server(kSaslAppName, conn->GetFd());
-  gscoped_ptr<AuthStore> authstore(new AuthStore());
-  CHECK_OK(authstore->Add("danger", "burrito"));
-  CHECK_OK(sasl_server.Init(kSaslAppName));
-  CHECK_OK(sasl_server.EnablePlain(std::move(authstore)));
-  Status s = sasl_server.Negotiate();
-  ASSERT_TRUE(s.IsNotAuthorized()) << "Expected auth failure! Got: " << s.ToString();
-}
-
-static void RunPlainFailingNegotiationClient(Socket* conn) {
-  SaslClient sasl_client(kSaslAppName, conn->GetFd());
-  CHECK_OK(sasl_client.EnablePlain("unknown", "burrito"));
-  CHECK_OK(sasl_client.Init(kSaslAppName));
-  Status s = sasl_client.Negotiate();
-  ASSERT_TRUE(s.IsNotAuthorized()) << "Expected auth failure! Got: " << s.ToString();
-}
-
-// Test SASL negotiation using the PLAIN mechanism over a socket.
-TEST_F(TestSaslRpc, TestPlainFailingNegotiation) {
-  RunNegotiationTest(RunPlainFailingNegotiationServer, RunPlainFailingNegotiationClient);
-}
-
-////////////////////////////////////////////////////////////////////////////////
-
 static void RunTimeoutExpectingServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
   CHECK_OK(sasl_server.Init(kSaslAppName));

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/sasl_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.cc b/src/kudu/rpc/sasl_server.cc
index 75000c4..bc1b94b 100644
--- a/src/kudu/rpc/sasl_server.cc
+++ b/src/kudu/rpc/sasl_server.cc
@@ -28,7 +28,6 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/stringprintf.h"
 #include "kudu/gutil/strings/split.h"
-#include "kudu/rpc/auth_store.h"
 #include "kudu/rpc/blocking_ops.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/serialization.h"
@@ -75,10 +74,9 @@ Status SaslServer::EnableAnonymous() {
   return helper_.EnableAnonymous();
 }
 
-Status SaslServer::EnablePlain(gscoped_ptr<AuthStore> authstore) {
+Status SaslServer::EnablePlain() {
   DCHECK_EQ(server_state_, SaslNegotiationState::INITIALIZED);
   RETURN_NOT_OK(helper_.EnablePlain());
-  authstore_.swap(authstore);
   return Status::OK();
 }
 
@@ -461,23 +459,14 @@ int SaslServer::GetOptionCb(const char* plugin_name, const char* option,
   return helper_.GetOptionCb(plugin_name, option, result, len);
 }
 
-int SaslServer::PlainAuthCb(sasl_conn_t *conn, const char *user, const char *pass,
-                            unsigned passlen, struct propctx *propctx) {
-  TRACE("SASL Server: Checking PLAIN auth credentials");
+int SaslServer::PlainAuthCb(sasl_conn_t * /*conn*/, const char * /*user*/, const char * /*pass*/,
+                            unsigned /*passlen*/, struct propctx * /*propctx*/) {
+  TRACE("SASL Server: Received PLAIN auth.");
   if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
     LOG(DFATAL) << "Password authentication callback called while PLAIN auth disabled";
     return SASL_BADPARAM;
   }
-  if (PREDICT_FALSE(!authstore_)) {
-    LOG(DFATAL) << "AuthStore not initialized";
-    return SASL_FAIL;
-  }
-  Status s = authstore_->Authenticate(user, string(pass, passlen));
-  TRACE("SASL Server: PLAIN user authentication status: $0", s.ToString());
-  if (!s.ok()) {
-    LOG(INFO) << "Failed login for user: " << user;
-    return SASL_FAIL;
-  }
+  // We always allow PLAIN authentication to succeed.
   return SASL_OK;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/732a5052/src/kudu/rpc/sasl_server.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.h b/src/kudu/rpc/sasl_server.h
index c5a1399..0f5946f 100644
--- a/src/kudu/rpc/sasl_server.h
+++ b/src/kudu/rpc/sasl_server.h
@@ -39,8 +39,6 @@ namespace rpc {
 
 using std::string;
 
-class AuthStore;
-
 // Class for doing SASL negotiation with a SaslClient over a bidirectional socket.
 // Operations on this class are NOT thread-safe.
 class SaslServer {
@@ -53,9 +51,11 @@ class SaslServer {
   // Must be called after Init().
   Status EnableAnonymous();
 
-  // Enable PLAIN authentication. TODO: Support impersonation.
+  // Enable PLAIN authentication.
+  // Despite PLAIN authentication taking a username and password, we disregard
+  // the password and use this as a "unauthenticated" mode.
   // Must be called after Init().
-  Status EnablePlain(gscoped_ptr<AuthStore> authstore);
+  Status EnablePlain();
 
   // Enable GSSAPI (Kerberos) authentication.
   // Call after Init().
@@ -153,9 +153,6 @@ class SaslServer {
   gscoped_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
   SaslHelper helper_;
 
-  // Authentication store used for PLAIN authentication.
-  gscoped_ptr<AuthStore> authstore_;
-
   // The set of features that the client supports. Filled in
   // after we receive the NEGOTIATE request from the client.
   std::set<RpcFeatureFlag> client_features_;


[3/3] kudu git commit: KUDU-100 make RLE encoder handle 64-bit integer.

Posted by to...@apache.org.
KUDU-100 make RLE encoder handle 64-bit integer.

Fix edge cases in bit stream.

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

Branch: refs/heads/master
Commit: a1a8eef23a3cdab198cbae88a999cefb38ff2aea
Parents: fd89792
Author: honghaijei <ho...@gmail.com>
Authored: Mon Oct 24 10:49:27 2016 +0000
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Oct 29 00:16:57 2016 +0000

----------------------------------------------------------------------
 src/kudu/util/bit-stream-utils.inline.h | 17 ++++++------
 src/kudu/util/bit-util-test.cc          |  8 ++++++
 src/kudu/util/bit-util.h                | 12 ++++++++
 src/kudu/util/rle-test.cc               | 41 ++++++++++++++++++++++------
 4 files changed, 61 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a1a8eef2/src/kudu/util/bit-stream-utils.inline.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/bit-stream-utils.inline.h b/src/kudu/util/bit-stream-utils.inline.h
index 6d6aad4..569197b 100644
--- a/src/kudu/util/bit-stream-utils.inline.h
+++ b/src/kudu/util/bit-stream-utils.inline.h
@@ -19,17 +19,18 @@
 
 #include <algorithm>
 
+#include "glog/logging.h"
 #include "kudu/util/bit-stream-utils.h"
 #include "kudu/util/alignment.h"
 
 namespace kudu {
 
 inline void BitWriter::PutValue(uint64_t v, int num_bits) {
-  // TODO: revisit this limit if necessary (can be raised to 64 by fixing some edge cases)
-  DCHECK_LE(num_bits, 32);
+  DCHECK_LE(num_bits, 64);
   // Truncate the higher-order bits. This is necessary to
   // support signed values.
-  v &= (1ULL << num_bits) - 1;
+  v &= ~0ULL >> (64 - num_bits);
+
 
   buffered_values_ |= v << bit_offset_;
   bit_offset_ += num_bits;
@@ -43,7 +44,7 @@ inline void BitWriter::PutValue(uint64_t v, int num_bits) {
     buffered_values_ = 0;
     byte_offset_ += 8;
     bit_offset_ -= 64;
-    buffered_values_ = v >> (num_bits - bit_offset_);
+    buffered_values_ = BitUtil::ShiftRightZeroOnOverflow(v, (num_bits - bit_offset_));
   }
   DCHECK_LT(bit_offset_, 64);
 }
@@ -109,8 +110,7 @@ inline void BitReader::BufferValues() {
 
 template<typename T>
 inline bool BitReader::GetValue(int num_bits, T* v) {
-  // TODO: revisit this limit if necessary
-  DCHECK_LE(num_bits, 32);
+  DCHECK_LE(num_bits, 64);
   DCHECK_LE(num_bits, sizeof(T) * 8);
 
   if (PREDICT_FALSE(byte_offset_ * 8 + bit_offset_ + num_bits > max_bytes_ * 8)) return false;
@@ -123,8 +123,9 @@ inline bool BitReader::GetValue(int num_bits, T* v) {
     bit_offset_ -= 64;
     BufferValues();
     // Read bits of v that crossed into new buffered_values_
-    *v |= BitUtil::TrailingBits(buffered_values_, bit_offset_)
-          << (num_bits - bit_offset_);
+    *v |= BitUtil::ShiftLeftZeroOnOverflow(
+        BitUtil::TrailingBits(buffered_values_, bit_offset_),
+        (num_bits - bit_offset_));
   }
   DCHECK_LE(bit_offset_, 64);
   return true;

http://git-wip-us.apache.org/repos/asf/kudu/blob/a1a8eef2/src/kudu/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/bit-util-test.cc b/src/kudu/util/bit-util-test.cc
index 8c43562..0d8eab4 100644
--- a/src/kudu/util/bit-util-test.cc
+++ b/src/kudu/util/bit-util-test.cc
@@ -32,6 +32,14 @@ TEST(BitUtil, TrailingBits) {
   EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 0), 0);
   EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 63), 0);
   EXPECT_EQ(BitUtil::TrailingBits(1LL << 63, 64), 1LL << 63);
+
+}
+
+TEST(BitUtil, ShiftBits) {
+  EXPECT_EQ(BitUtil::ShiftLeftZeroOnOverflow(1ULL, 64), 0ULL);
+  EXPECT_EQ(BitUtil::ShiftLeftZeroOnOverflow(0xFFFFFFFFFFFFFFFFULL, 32), 0xFFFFFFFF00000000ULL);
+  EXPECT_EQ(BitUtil::ShiftRightZeroOnOverflow(1ULL, 64), 0ULL);
+  EXPECT_EQ(BitUtil::ShiftRightZeroOnOverflow(0xFFFFFFFFFFFFFFFFULL, 32), 0x00000000FFFFFFFFULL);
 }
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/a1a8eef2/src/kudu/util/bit-util.h
----------------------------------------------------------------------
diff --git a/src/kudu/util/bit-util.h b/src/kudu/util/bit-util.h
index 7f11f05..21f1558 100644
--- a/src/kudu/util/bit-util.h
+++ b/src/kudu/util/bit-util.h
@@ -38,6 +38,18 @@ class BitUtil {
     int n = 64 - num_bits;
     return (v << n) >> n;
   }
+
+  static inline uint64_t ShiftLeftZeroOnOverflow(uint64_t v, int num_bits) {
+    if (num_bits >= 64) return 0;
+    return v << num_bits;
+  }
+
+  static inline uint64_t ShiftRightZeroOnOverflow(uint64_t v, int num_bits) {
+    if (num_bits >= 64) return 0;
+    return v >> num_bits;
+  }
+
+
 };
 
 } // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/a1a8eef2/src/kudu/util/rle-test.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/rle-test.cc b/src/kudu/util/rle-test.cc
index 3b71dc2..185fed5 100644
--- a/src/kudu/util/rle-test.cc
+++ b/src/kudu/util/rle-test.cc
@@ -35,7 +35,7 @@ using std::vector;
 
 namespace kudu {
 
-const int MAX_WIDTH = 32;
+const int MAX_WIDTH = 64;
 
 class TestRle : public KuduTest {};
 
@@ -205,7 +205,7 @@ void ValidateRle(const vector<T>& values, int bit_width,
 TEST(Rle, SpecificSequences) {
   const int kTestLen = 1024;
   uint8_t expected_buffer[kTestLen];
-  vector<int> values;
+  vector<uint64_t> values;
 
   // Test 50 0' followed by 50 1's
   values.resize(100);
@@ -251,10 +251,10 @@ TEST(Rle, SpecificSequences) {
 // ValidateRle on 'num_vals' values with width 'bit_width'. If 'value' != -1, that value
 // is used, otherwise alternating values are used.
 void TestRleValues(int bit_width, int num_vals, int value = -1) {
-  const uint64_t mod = (bit_width == 64) ? 1 : 1LL << bit_width;
-  vector<int> values;
-  for (int v = 0; v < num_vals; ++v) {
-    values.push_back((value != -1) ? value : (v % mod));
+  const uint64_t mod = bit_width == 64 ? 1ULL : 1ULL << bit_width;
+  vector<uint64_t> values;
+  for (uint64_t v = 0; v < num_vals; ++v) {
+    values.push_back((value != -1) ? value : (bit_width == 64 ? v : (v % mod)));
   }
   ValidateRle(values, bit_width, nullptr, -1);
 }
@@ -301,14 +301,14 @@ TEST_F(BitRle, Flush) {
   ValidateRle(values, 1, nullptr, -1);
 }
 
-// Test some random sequences.
-TEST_F(BitRle, Random) {
+// Test some random bool sequences.
+TEST_F(BitRle, RandomBools) {
   int iters = 0;
   const int n_iters = AllowSlowTests() ? 1000 : 20;
   while (iters < n_iters) {
     srand(iters++);
     if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters;
-    vector<int> values;
+    vector<uint64_t > values;
     bool parity = 0;
     for (int i = 0; i < 1000; ++i) {
       int group_size = rand() % 20 + 1; // NOLINT(*)
@@ -324,6 +324,29 @@ TEST_F(BitRle, Random) {
   }
 }
 
+// Test some random 64-bit sequences.
+TEST_F(BitRle, Random64Bit) {
+  int iters = 0;
+  const int n_iters = AllowSlowTests() ? 1000 : 20;
+  while (iters < n_iters) {
+    srand(iters++);
+    if (iters % 10000 == 0) LOG(ERROR) << "Seed: " << iters;
+    vector<uint64_t > values;
+    for (int i = 0; i < 1000; ++i) {
+      int group_size = rand() % 20 + 1; // NOLINT(*)
+      uint64_t cur_value = (static_cast<uint64_t>(rand()) << 32) + static_cast<uint64_t>(rand());
+      if (group_size > 16) {
+        group_size = 1;
+      }
+      for (int i = 0; i < group_size; ++i) {
+        values.push_back(cur_value);
+      }
+
+    }
+    ValidateRle(values, 64, nullptr, -1);
+  }
+}
+
 // Test a sequence of 1 0's, 2 1's, 3 0's. etc
 // e.g. 011000111100000
 TEST_F(BitRle, RepeatedPattern) {


[2/3] kudu git commit: Fix SASL mechanism negotiation on the server side

Posted by to...@apache.org.
Fix SASL mechanism negotiation on the server side

This fixes the order in which the SASL server is initialized such that
the mechanism list is set up before the sasl_server_init function is
called. This is equivalent to the change made in Change ID
I3c1b93045acd428ef3437597059c5106b03e25d0 on the client side.

Without this change, the server would use a null mechanism list, which
was being treated as a wildcard.

The new test case configures a server to only accept GSSAPI, and then
has a client configured to accept GSSAPI or PLAIN. Before the fix, it
would negotiate PLAIN despite the attempt to require GSSAPI on the
server side. Now, it properly negotiates.

Change-Id: I4741ce618525311258d5ba61dd85f6f4721caf66
Reviewed-on: http://gerrit.cloudera.org:8080/4875
Tested-by: Kudu Jenkins
Reviewed-by: Alexey Serbin <as...@cloudera.com>


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

Branch: refs/heads/master
Commit: fd89792a89a551902f69bb8d40f128df6dbe7bc4
Parents: 732a505
Author: Todd Lipcon <to...@apache.org>
Authored: Thu Oct 27 17:35:40 2016 -0700
Committer: Todd Lipcon <to...@apache.org>
Committed: Sat Oct 29 00:00:36 2016 +0000

----------------------------------------------------------------------
 src/kudu/rpc/connection.cc    |  2 +-
 src/kudu/rpc/sasl_helper.cc   |  4 +-
 src/kudu/rpc/sasl_rpc-test.cc | 84 +++++++++++++++++++++++++++++++++++---
 src/kudu/rpc/sasl_server.cc   |  6 +--
 4 files changed, 85 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/fd89792a/src/kudu/rpc/connection.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index 7c15a5d..bdc4f9d 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -626,8 +626,8 @@ Status Connection::InitSaslClient() {
 }
 
 Status Connection::InitSaslServer() {
-  RETURN_NOT_OK(sasl_server().Init(kSaslProtoName));
   RETURN_NOT_OK(sasl_server().EnablePlain());
+  RETURN_NOT_OK(sasl_server().Init(kSaslProtoName));
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd89792a/src/kudu/rpc/sasl_helper.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_helper.cc b/src/kudu/rpc/sasl_helper.cc
index 459b120..280f41e 100644
--- a/src/kudu/rpc/sasl_helper.cc
+++ b/src/kudu/rpc/sasl_helper.cc
@@ -93,7 +93,7 @@ const std::set<std::string>& SaslHelper::LocalMechs() const {
 
 const char* SaslHelper::LocalMechListString() const {
   JoinStrings(mechs_, " ", &mech_list_);
-  return mech_list_.empty() ? nullptr : mech_list_.c_str();
+  return mech_list_.c_str();
 }
 
 
@@ -119,7 +119,7 @@ int SaslHelper::GetOptionCb(const char* plugin_name, const char* option,
     if (cb_name == option) {
       *result = LocalMechListString();
       if (len != nullptr) *len = strlen(*result);
-      DVLOG(3) << tag_ << ": Enabled mech list: " << (*result == nullptr ? "NULL" : *result);
+      VLOG(4) << tag_ << ": Enabled mech list: " << *result;
       return SASL_OK;
     }
     VLOG(4) << tag_ << ": GetOptionCb: Unknown library option: " << option;

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd89792a/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 b238dff..61b3377 100644
--- a/src/kudu/rpc/sasl_rpc-test.cc
+++ b/src/kudu/rpc/sasl_rpc-test.cc
@@ -55,8 +55,10 @@ class TestSaslRpc : public RpcTestBase {
 // Test basic initialization of the objects.
 TEST_F(TestSaslRpc, TestBasicInit) {
   SaslServer server(kSaslAppName, -1);
+  server.EnableAnonymous();
   ASSERT_OK(server.Init(kSaslAppName));
   SaslClient client(kSaslAppName, -1);
+  client.EnableAnonymous();
   ASSERT_OK(client.Init(kSaslAppName));
 }
 
@@ -105,8 +107,8 @@ static void RunNegotiationTest(const SocketCallable& server_runner,
 
 static void RunAnonNegotiationServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
-  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.EnableAnonymous());
+  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.Negotiate());
 }
 
@@ -126,8 +128,8 @@ TEST_F(TestSaslRpc, TestAnonNegotiation) {
 
 static void RunPlainNegotiationServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
-  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.EnablePlain());
+  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.Negotiate());
   CHECK(ContainsKey(sasl_server.client_features(), APPLICATION_FEATURE_FLAGS));
   CHECK_EQ("my-username", sasl_server.authenticated_user());
@@ -159,8 +161,8 @@ static void RunGSSAPINegotiationServer(
     const CheckerFunction<SaslServer>& post_check) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
   sasl_server.set_server_fqdn("127.0.0.1");
-  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.EnableGSSAPI());
+  CHECK_OK(sasl_server.Init(kSaslAppName));
   post_check(sasl_server.Negotiate(), sasl_server);
 }
 
@@ -176,6 +178,78 @@ static void RunGSSAPINegotiationClient(
   post_check(sasl_client.Negotiate(), sasl_client);
 }
 
+// Test configuring a client to allow but not require Kerberos/GSSAPI,
+// and connect to a server which requires Kerberos/GSSAPI.
+//
+// They should negotiate to use Kerberos/GSSAPI.
+TEST_F(TestSaslRpc, TestRestrictiveServer_NonRestrictiveClient) {
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+
+  // Create the server principal and keytab.
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/localhost", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  // Create and kinit as a client user.
+  ASSERT_OK(kdc.CreateUserPrincipal("testuser"));
+  ASSERT_OK(kdc.Kinit("testuser"));
+  ASSERT_OK(kdc.SetKrb5Environment());
+
+  // Authentication should now succeed on both sides.
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, SaslServer& server) {
+                  CHECK_OK(s);
+                  CHECK_EQ(SaslMechanism::GSSAPI, server.negotiated_mechanism());
+                  CHECK_EQ("testuser", server.authenticated_user());
+                }),
+      [](Socket* conn) {
+        SaslClient sasl_client(kSaslAppName, conn->GetFd());
+        sasl_client.set_server_fqdn("127.0.0.1");
+        // The client enables both PLAIN and GSSAPI.
+        CHECK_OK(sasl_client.EnablePlain("foo", "bar"));
+        CHECK_OK(sasl_client.EnableGSSAPI());
+        CHECK_OK(sasl_client.Init(kSaslAppName));
+        CHECK_OK(sasl_client.Negotiate());
+        CHECK_EQ(SaslMechanism::GSSAPI, sasl_client.negotiated_mechanism());
+      });
+}
+
+// Test configuring a client to only support PLAIN, and a server which
+// only supports GSSAPI. This would happen, for example, if an old Kudu
+// client tries to talk to a secure-only cluster.
+TEST_F(TestSaslRpc, TestNoMatchingMechanisms) {
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+
+  // Create the server principal and keytab.
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/localhost", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, SaslServer& server) {
+                  // The client fails to find a matching mechanism and
+                  // doesn't send any failure message to the server.
+                  // Instead, it just disconnects.
+                  //
+                  // TODO(todd): this could produce a better message!
+                  ASSERT_STR_CONTAINS(s.ToString(), "got EOF from remote");
+                }),
+      [](Socket* conn) {
+        SaslClient sasl_client(kSaslAppName, conn->GetFd());
+        sasl_client.set_server_fqdn("127.0.0.1");
+        // The client enables both PLAIN and GSSAPI.
+        CHECK_OK(sasl_client.EnablePlain("foo", "bar"));
+        CHECK_OK(sasl_client.Init(kSaslAppName));
+        Status s = sasl_client.Negotiate();
+        ASSERT_STR_CONTAINS(s.ToString(), "no mechanism available: No worthy mechs found");
+      });
+}
+
 // Test SASL negotiation using the GSSAPI (kerberos) mechanism over a socket.
 TEST_F(TestSaslRpc, TestGSSAPINegotiation) {
   MiniKdc kdc;
@@ -269,8 +343,8 @@ TEST_F(TestSaslRpc, TestGSSAPINegotiation) {
 
 static void RunTimeoutExpectingServer(Socket* conn) {
   SaslServer sasl_server(kSaslAppName, conn->GetFd());
-  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.EnableAnonymous());
+  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: "
       << s.ToString();
@@ -296,8 +370,8 @@ TEST_F(TestSaslRpc, TestClientTimeout) {
 
 static void RunTimeoutNegotiationServer(Socket* sock) {
   SaslServer sasl_server(kSaslAppName, sock->GetFd());
-  CHECK_OK(sasl_server.Init(kSaslAppName));
   CHECK_OK(sasl_server.EnableAnonymous());
+  CHECK_OK(sasl_server.Init(kSaslAppName));
   MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
   sasl_server.set_deadline(deadline);
   Status s = sasl_server.Negotiate();

http://git-wip-us.apache.org/repos/asf/kudu/blob/fd89792a/src/kudu/rpc/sasl_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.cc b/src/kudu/rpc/sasl_server.cc
index bc1b94b..c8e5fc0 100644
--- a/src/kudu/rpc/sasl_server.cc
+++ b/src/kudu/rpc/sasl_server.cc
@@ -70,18 +70,18 @@ SaslServer::~SaslServer() {
 }
 
 Status SaslServer::EnableAnonymous() {
-  DCHECK_EQ(server_state_, SaslNegotiationState::INITIALIZED);
+  DCHECK_EQ(server_state_, SaslNegotiationState::NEW);
   return helper_.EnableAnonymous();
 }
 
 Status SaslServer::EnablePlain() {
-  DCHECK_EQ(server_state_, SaslNegotiationState::INITIALIZED);
+  DCHECK_EQ(server_state_, SaslNegotiationState::NEW);
   RETURN_NOT_OK(helper_.EnablePlain());
   return Status::OK();
 }
 
 Status SaslServer::EnableGSSAPI() {
-  DCHECK_EQ(server_state_, SaslNegotiationState::INITIALIZED);
+  DCHECK_EQ(server_state_, SaslNegotiationState::NEW);
   return helper_.EnableGSSAPI();
 }