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 23:27:07 UTC

[1/2] kudu git commit: TLS-negotiation [3/n]: rename negotiation protobuf messages

Repository: kudu
Updated Branches:
  refs/heads/master ef57bda2c -> 9d0421e80


TLS-negotiation [3/n]: rename negotiation protobuf messages

SASL negotiation in the RPC layer has evolved to encompass more than
just SASL. Today it includes RpcFeatureFlag negotiation, and in the
future it may include TLS and authn token negotiation. Consequently,
this commit renames the message to reflect its role.

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

Branch: refs/heads/master
Commit: 2960e6927137fc8ccd175001810d292066c4dac6
Parents: ef57bda
Author: Dan Burkert <da...@apache.org>
Authored: Wed Dec 14 14:03:48 2016 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Mon Jan 23 23:14:55 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/SecureRpcHelper.java | 32 +++----
 src/kudu/rpc/blocking_ops.cc                    |  2 +-
 src/kudu/rpc/blocking_ops.h                     |  4 +-
 src/kudu/rpc/constants.h                        |  2 +-
 src/kudu/rpc/rpc_header.proto                   | 10 +--
 src/kudu/rpc/sasl_client.cc                     | 67 +++++++-------
 src/kudu/rpc/sasl_client.h                      | 23 ++---
 src/kudu/rpc/sasl_helper.cc                     | 18 ++--
 src/kudu/rpc/sasl_helper.h                      | 14 +--
 src/kudu/rpc/sasl_server.cc                     | 91 ++++++++++----------
 src/kudu/rpc/sasl_server.h                      | 15 ++--
 11 files changed, 143 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java b/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
index c0bfe46..90cd167 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
@@ -106,18 +106,18 @@ public class SecureRpcHelper {
   }
 
   private void sendNegotiateMessage(Channel channel) {
-    RpcHeader.SaslMessagePB.Builder builder = RpcHeader.SaslMessagePB.newBuilder();
+    RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder();
 
     // Advertise our supported features
     for (RpcHeader.RpcFeatureFlag flag : SUPPORTED_RPC_FEATURES) {
       builder.addSupportedFeatures(flag);
     }
 
-    builder.setState(RpcHeader.SaslMessagePB.SaslState.NEGOTIATE);
+    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.NEGOTIATE);
     sendSaslMessage(channel, builder.build());
   }
 
-  private void sendSaslMessage(Channel channel, RpcHeader.SaslMessagePB msg) {
+  private void sendSaslMessage(Channel channel, RpcHeader.NegotiatePB msg) {
     RpcHeader.RequestHeader.Builder builder = RpcHeader.RequestHeader.newBuilder();
     builder.setCallId(SASL_CALL_ID);
     RpcHeader.RequestHeader header = builder.build();
@@ -128,8 +128,8 @@ public class SecureRpcHelper {
 
   public ChannelBuffer handleResponse(ChannelBuffer buf, Channel chan) throws SaslException {
     if (!saslClient.isComplete() || negoUnderway) {
-      RpcHeader.SaslMessagePB response = parseSaslMsgResponse(buf);
-      switch (response.getState()) {
+      RpcHeader.NegotiatePB response = parseSaslMsgResponse(buf);
+      switch (response.getStep()) {
         case NEGOTIATE:
           handleNegotiateResponse(chan, response);
           break;
@@ -140,7 +140,7 @@ public class SecureRpcHelper {
           handleSuccessResponse(chan);
           break;
         default:
-          LOG.error(String.format("Wrong SASL state: %s", response.getState()));
+          LOG.error(String.format("Wrong negotiation step: %s", response.getStep()));
       }
       return null;
     }
@@ -189,7 +189,7 @@ public class SecureRpcHelper {
     }
   }
 
-  private RpcHeader.SaslMessagePB parseSaslMsgResponse(ChannelBuffer buf) {
+  private RpcHeader.NegotiatePB parseSaslMsgResponse(ChannelBuffer buf) {
     CallResponse response = new CallResponse(buf);
     RpcHeader.ResponseHeader responseHeader = response.getHeader();
     int id = responseHeader.getCallId();
@@ -197,16 +197,16 @@ public class SecureRpcHelper {
       throw new IllegalStateException("Received a call that wasn't for SASL");
     }
 
-    RpcHeader.SaslMessagePB.Builder saslBuilder =  RpcHeader.SaslMessagePB.newBuilder();
+    RpcHeader.NegotiatePB.Builder saslBuilder = RpcHeader.NegotiatePB.newBuilder();
     KuduRpc.readProtobuf(response.getPBMessage(), saslBuilder);
     return saslBuilder.build();
   }
 
 
-  private void handleNegotiateResponse(Channel chan, RpcHeader.SaslMessagePB response) throws
+  private void handleNegotiateResponse(Channel chan, RpcHeader.NegotiatePB response) throws
       SaslException {
-    RpcHeader.SaslMessagePB.SaslAuth negotiatedAuth = null;
-    for (RpcHeader.SaslMessagePB.SaslAuth auth : response.getAuthsList()) {
+    RpcHeader.NegotiatePB.SaslAuth negotiatedAuth = null;
+    for (RpcHeader.NegotiatePB.SaslAuth auth : response.getAuthsList()) {
       negotiatedAuth = auth;
     }
 
@@ -223,24 +223,24 @@ public class SecureRpcHelper {
       saslToken = saslClient.evaluateChallenge(saslToken);
     }
 
-    RpcHeader.SaslMessagePB.Builder builder = RpcHeader.SaslMessagePB.newBuilder();
+    RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder();
     if (saslToken != null) {
       builder.setToken(ZeroCopyLiteralByteString.wrap(saslToken));
     }
-    builder.setState(RpcHeader.SaslMessagePB.SaslState.INITIATE);
+    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.INITIATE);
     builder.addAuths(negotiatedAuth);
     sendSaslMessage(chan, builder.build());
   }
 
-  private void handleChallengeResponse(Channel chan, RpcHeader.SaslMessagePB response) throws
+  private void handleChallengeResponse(Channel chan, RpcHeader.NegotiatePB response) throws
       SaslException {
     byte[] saslToken = saslClient.evaluateChallenge(response.getToken().toByteArray());
     if (saslToken == null) {
       throw new IllegalStateException("Not expecting an empty token");
     }
-    RpcHeader.SaslMessagePB.Builder builder = RpcHeader.SaslMessagePB.newBuilder();
+    RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder();
     builder.setToken(ZeroCopyLiteralByteString.wrap(saslToken));
-    builder.setState(RpcHeader.SaslMessagePB.SaslState.RESPONSE);
+    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.RESPONSE);
     sendSaslMessage(chan, builder.build());
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/blocking_ops.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/blocking_ops.cc b/src/kudu/rpc/blocking_ops.cc
index 4d2b201..64ae2c0 100644
--- a/src/kudu/rpc/blocking_ops.cc
+++ b/src/kudu/rpc/blocking_ops.cc
@@ -41,7 +41,7 @@ using google::protobuf::MessageLite;
 
 const char kHTTPHeader[] = "HTTP";
 
-Status EnsureBlockingMode(const Socket* const sock) {
+Status EnsureBlockingMode(const Socket* sock) {
   bool is_nonblocking;
   RETURN_NOT_OK(sock->IsNonBlocking(&is_nonblocking));
   if (is_nonblocking) {

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/blocking_ops.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/blocking_ops.h b/src/kudu/rpc/blocking_ops.h
index 9ff7bd3..01bb7a6 100644
--- a/src/kudu/rpc/blocking_ops.h
+++ b/src/kudu/rpc/blocking_ops.h
@@ -38,10 +38,8 @@ class Status;
 
 namespace rpc {
 
-class SaslMessagePB;
-
 // Returns OK if socket is in blocking mode. Otherwise, returns an error.
-Status EnsureBlockingMode(const Socket* const sock);
+Status EnsureBlockingMode(const Socket* sock);
 
 // Encode and send a message over a socket.
 // header: Request or Response header protobuf.

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/constants.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/constants.h b/src/kudu/rpc/constants.h
index d050a34..e179ddc 100644
--- a/src/kudu/rpc/constants.h
+++ b/src/kudu/rpc/constants.h
@@ -41,7 +41,7 @@ static const uint32_t kCurrentRpcVersion = 9;
 // From Hadoop.
 static const int32_t kInvalidCallId = -2;
 static const int32_t kConnectionContextCallId = -3;
-static const int32_t kSaslCallId = -33;
+static const int32_t kNegotiateCallId = -33;
 
 static const uint8_t kMagicNumberLength = 4;
 static const uint8_t kHeaderFlagsLength = 3;

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_header.proto b/src/kudu/rpc/rpc_header.proto
index c7adbcb..f2e4c8c 100644
--- a/src/kudu/rpc/rpc_header.proto
+++ b/src/kudu/rpc/rpc_header.proto
@@ -74,8 +74,8 @@ enum RpcFeatureFlag {
 };
 
 // Message type passed back & forth for the SASL negotiation.
-message SaslMessagePB {
-  enum SaslState {
+message NegotiatePB {
+  enum NegotiateStep {
     UNKNOWN   = 999;
     SUCCESS   = 0;
     NEGOTIATE = 1;
@@ -101,9 +101,9 @@ message SaslMessagePB {
   // to simplify backporting of features out-of-order.
   repeated RpcFeatureFlag supported_features = 1;
 
-  required SaslState state = 2;  // RPC system SASL state.
-  optional bytes token     = 3  [(REDACT) = true];
-  repeated SaslAuth auths  = 4;
+  required NegotiateStep step = 2;
+  optional bytes token        = 3 [(REDACT) = true];
+  repeated SaslAuth auths     = 4;
 }
 
 message RemoteMethodPB {

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_client.cc b/src/kudu/rpc/sasl_client.cc
index 379aec8..6e0cbab 100644
--- a/src/kudu/rpc/sasl_client.cc
+++ b/src/kudu/rpc/sasl_client.cc
@@ -208,30 +208,30 @@ Status SaslClient::Negotiate() {
     RETURN_NOT_OK(ReceiveFramedMessageBlocking(sock_, &recv_buf, &header, &param_buf, deadline_));
     nego_response_expected_ = false;
 
-    SaslMessagePB response;
-    RETURN_NOT_OK(ParseSaslMsgResponse(header, param_buf, &response));
+    NegotiatePB response;
+    RETURN_NOT_OK(ParseNegotiatePB(header, param_buf, &response));
 
-    switch (response.state()) {
+    switch (response.step()) {
       // NEGOTIATE: Server has sent us its list of supported SASL mechanisms.
-      case SaslMessagePB::NEGOTIATE:
+      case NegotiatePB::NEGOTIATE:
         RETURN_NOT_OK(HandleNegotiateResponse(response));
         break;
 
       // CHALLENGE: Server sent us a follow-up to an INITIATE or RESPONSE request.
-      case SaslMessagePB::CHALLENGE:
+      case NegotiatePB::CHALLENGE:
         RETURN_NOT_OK(HandleChallengeResponse(response));
         break;
 
       // SUCCESS: Server has accepted our authentication request. Negotiation successful.
-      case SaslMessagePB::SUCCESS:
+      case NegotiatePB::SUCCESS:
         RETURN_NOT_OK(HandleSuccessResponse(response));
         break;
 
       // Client sent us some unsupported SASL response.
       default:
         LOG(ERROR) << "SASL Client: Received unsupported response from server";
-        return Status::InvalidArgument("RPC client doesn't support SASL state in response",
-            SaslMessagePB::SaslState_Name(response.state()));
+        return Status::InvalidArgument("RPC client doesn't support Negotiate step",
+                                       NegotiatePB::NegotiateStep_Name(response.step()));
     }
   }
 
@@ -240,32 +240,33 @@ Status SaslClient::Negotiate() {
   return Status::OK();
 }
 
-Status SaslClient::SendSaslMessage(const SaslMessagePB& msg) {
+Status SaslClient::SendNegotiatePB(const NegotiatePB& msg) {
   DCHECK_NE(client_state_, SaslNegotiationState::NEW)
-      << "Must not send SASL messages before calling Init()";
+      << "Must not send Negotiate messages before calling Init()";
   DCHECK_NE(client_state_, SaslNegotiationState::NEGOTIATED)
-      << "Must not send SASL messages after Negotiate() succeeds";
+      << "Must not send Negotiate messages after negotiation succeeds";
 
   // Create header with SASL-specific callId
   RequestHeader header;
-  header.set_call_id(kSaslCallId);
-  return helper_.SendSaslMessage(sock_, header, msg, deadline_);
+  header.set_call_id(kNegotiateCallId);
+  return helper_.SendNegotiatePB(sock_, header, msg, deadline_);
 }
 
-Status SaslClient::ParseSaslMsgResponse(const ResponseHeader& header, const Slice& param_buf,
-    SaslMessagePB* response) {
-  RETURN_NOT_OK(helper_.SanityCheckSaslCallId(header.call_id()));
+Status SaslClient::ParseNegotiatePB(const ResponseHeader& header,
+                                    const Slice& param_buf,
+                                    NegotiatePB* response) {
+  RETURN_NOT_OK(helper_.SanityCheckNegotiateCallId(header.call_id()));
 
   if (header.is_error()) {
     return ParseError(param_buf);
   }
 
-  return helper_.ParseSaslMessage(param_buf, response);
+  return helper_.ParseNegotiatePB(param_buf, response);
 }
 
 Status SaslClient::SendNegotiateMessage() {
-  SaslMessagePB msg;
-  msg.set_state(SaslMessagePB::NEGOTIATE);
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::NEGOTIATE);
 
   // Advertise our supported features.
   for (RpcFeatureFlag feature : kSupportedClientRpcFeatureFlags) {
@@ -273,29 +274,29 @@ Status SaslClient::SendNegotiateMessage() {
   }
 
   TRACE("SASL Client: Sending NEGOTIATE request to server.");
-  RETURN_NOT_OK(SendSaslMessage(msg));
+  RETURN_NOT_OK(SendNegotiatePB(msg));
   nego_response_expected_ = true;
   return Status::OK();
 }
 
-Status SaslClient::SendInitiateMessage(const SaslMessagePB_SaslAuth& auth,
+Status SaslClient::SendInitiateMessage(const NegotiatePB_SaslAuth& auth,
     const char* init_msg, unsigned init_msg_len) {
-  SaslMessagePB msg;
-  msg.set_state(SaslMessagePB::INITIATE);
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::INITIATE);
   msg.mutable_token()->assign(init_msg, init_msg_len);
   msg.add_auths()->CopyFrom(auth);
   TRACE("SASL Client: Sending INITIATE request to server.");
-  RETURN_NOT_OK(SendSaslMessage(msg));
+  RETURN_NOT_OK(SendNegotiatePB(msg));
   nego_response_expected_ = true;
   return Status::OK();
 }
 
 Status SaslClient::SendResponseMessage(const char* resp_msg, unsigned resp_msg_len) {
-  SaslMessagePB reply;
-  reply.set_state(SaslMessagePB::RESPONSE);
+  NegotiatePB reply;
+  reply.set_step(NegotiatePB::RESPONSE);
   reply.mutable_token()->assign(resp_msg, resp_msg_len);
   TRACE("SASL Client: Sending RESPONSE request to server.");
-  RETURN_NOT_OK(SendSaslMessage(reply));
+  RETURN_NOT_OK(SendNegotiatePB(reply));
   nego_response_expected_ = true;
   return Status::OK();
 }
@@ -311,7 +312,7 @@ Status SaslClient::DoSaslStep(const string& in, const char** out, unsigned* out_
   return s;
 }
 
-Status SaslClient::HandleNegotiateResponse(const SaslMessagePB& response) {
+Status SaslClient::HandleNegotiateResponse(const NegotiatePB& response) {
   TRACE("SASL Client: Received NEGOTIATE response from server");
   // Fill in the set of features supported by the server.
   for (int flag : response.supported_features()) {
@@ -326,8 +327,8 @@ Status SaslClient::HandleNegotiateResponse(const SaslMessagePB& response) {
   // Build a map of the mechanisms offered by the server.
   const set<string>& local_mechs = helper_.LocalMechs();
   set<string> server_mechs;
-  map<string, SaslMessagePB::SaslAuth> server_mech_map;
-  for (const SaslMessagePB::SaslAuth& auth : response.auths()) {
+  map<string, NegotiatePB::SaslAuth> server_mech_map;
+  for (const NegotiatePB::SaslAuth& auth : response.auths()) {
     const auto& mech = auth.mechanism();
     server_mech_map[mech] = auth;
     server_mechs.insert(mech);
@@ -384,7 +385,7 @@ Status SaslClient::HandleNegotiateResponse(const SaslMessagePB& response) {
   }
 
   // The server matched one of our mechanisms.
-  SaslMessagePB::SaslAuth* auth = FindOrNull(server_mech_map, negotiated_mech);
+  NegotiatePB::SaslAuth* auth = FindOrNull(server_mech_map, negotiated_mech);
   if (PREDICT_FALSE(auth == nullptr)) {
     return Status::IllegalState("Unable to find auth in map, unexpected error", negotiated_mech);
   }
@@ -394,7 +395,7 @@ Status SaslClient::HandleNegotiateResponse(const SaslMessagePB& response) {
   return Status::OK();
 }
 
-Status SaslClient::HandleChallengeResponse(const SaslMessagePB& response) {
+Status SaslClient::HandleChallengeResponse(const NegotiatePB& response) {
   TRACE("SASL Client: Received CHALLENGE response from server");
   if (PREDICT_FALSE(nego_ok_)) {
     LOG(DFATAL) << "Server sent CHALLENGE response after client library returned SASL_OK";
@@ -414,7 +415,7 @@ Status SaslClient::HandleChallengeResponse(const SaslMessagePB& response) {
   return Status::OK();
 }
 
-Status SaslClient::HandleSuccessResponse(const SaslMessagePB& response) {
+Status SaslClient::HandleSuccessResponse(const NegotiatePB& response) {
   TRACE("SASL Client: Received SUCCESS response from server");
   if (!nego_ok_) {
     const char* out = nullptr;

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_client.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_client.h b/src/kudu/rpc/sasl_client.h
index e35ff2a..746a6f8 100644
--- a/src/kudu/rpc/sasl_client.h
+++ b/src/kudu/rpc/sasl_client.h
@@ -37,9 +37,9 @@ namespace rpc {
 
 using std::string;
 
+class NegotiatePB;
+class NegotiatePB_SaslAuth;
 class ResponseHeader;
-class SaslMessagePB;
-class SaslMessagePB_SaslAuth;
 
 // Class for doing SASL negotiation with a SaslServer over a bidirectional socket.
 // Operations on this class are NOT thread-safe.
@@ -106,18 +106,19 @@ class SaslClient {
   int SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret);
 
  private:
-  // Encode and send the specified SASL message to the server.
-  Status SendSaslMessage(const SaslMessagePB& msg);
+  // Encode and send the specified negotiate message to the server.
+  Status SendNegotiatePB(const NegotiatePB& msg);
 
   // Validate that header does not indicate an error, parse param_buf into response.
-  Status ParseSaslMsgResponse(const ResponseHeader& header, const Slice& param_buf,
-      SaslMessagePB* response);
+  Status ParseNegotiatePB(const ResponseHeader& header,
+                          const Slice& param_buf,
+                          NegotiatePB* response);
 
   // Send an NEGOTIATE message to the server.
   Status SendNegotiateMessage();
 
-  // Send an INITIATE message to the server.
-  Status SendInitiateMessage(const SaslMessagePB_SaslAuth& auth,
+  // Send an SASL_INITIATE message to the server.
+  Status SendInitiateMessage(const NegotiatePB_SaslAuth& auth,
                              const char* init_msg, unsigned init_msg_len);
 
   // Send a RESPONSE message to the server.
@@ -132,13 +133,13 @@ class SaslClient {
   Status DoSaslStep(const string& in, const char** out, unsigned* out_len);
 
   // Handle case when server sends NEGOTIATE response.
-  Status HandleNegotiateResponse(const SaslMessagePB& response);
+  Status HandleNegotiateResponse(const NegotiatePB& response);
 
   // Handle case when server sends CHALLENGE response.
-  Status HandleChallengeResponse(const SaslMessagePB& response);
+  Status HandleChallengeResponse(const NegotiatePB& response);
 
   // Handle case when server sends SUCCESS response.
-  Status HandleSuccessResponse(const SaslMessagePB& response);
+  Status HandleSuccessResponse(const NegotiatePB& response);
 
   // Parse error status message from raw bytes of an ErrorStatusPB.
   Status ParseError(const Slice& err_data);

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_helper.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_helper.cc b/src/kudu/rpc/sasl_helper.cc
index 60be7a6..4e408f7 100644
--- a/src/kudu/rpc/sasl_helper.cc
+++ b/src/kudu/rpc/sasl_helper.cc
@@ -29,6 +29,7 @@
 #include "kudu/gutil/map-util.h"
 #include "kudu/gutil/port.h"
 #include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
 #include "kudu/rpc/blocking_ops.h"
 #include "kudu/rpc/constants.h"
 #include "kudu/rpc/rpc_header.pb.h"
@@ -147,17 +148,18 @@ bool SaslHelper::IsPlainEnabled() const {
   return plain_enabled_;
 }
 
-Status SaslHelper::SanityCheckSaslCallId(int32_t call_id) const {
-  if (call_id != kSaslCallId) {
-    Status s = Status::IllegalState(StringPrintf("Non-SASL request during negotiation. "
-          "Expected callId: %d, received callId: %d", kSaslCallId, call_id));
+Status SaslHelper::SanityCheckNegotiateCallId(int32_t call_id) const {
+  if (call_id != kNegotiateCallId) {
+    Status s = Status::IllegalState(strings::Substitute(
+          "Non-Negotiate request during negotiation. Expected callId: $0, received callId: $1",
+          kNegotiateCallId, call_id));
     LOG(DFATAL) << tag_ << ": " << s.ToString();
     return s;
   }
   return Status::OK();
 }
 
-Status SaslHelper::ParseSaslMessage(const Slice& param_buf, SaslMessagePB* msg) {
+Status SaslHelper::ParseNegotiatePB(const Slice& param_buf, NegotiatePB* msg) {
   if (!msg->ParseFromArray(param_buf.data(), param_buf.size())) {
     return Status::IOError(tag_ + ": Invalid SASL message, missing fields",
         msg->InitializationErrorString());
@@ -165,8 +167,10 @@ Status SaslHelper::ParseSaslMessage(const Slice& param_buf, SaslMessagePB* msg)
   return Status::OK();
 }
 
-Status SaslHelper::SendSaslMessage(Socket* sock, const MessageLite& header, const MessageLite& msg,
-      const MonoTime& deadline) {
+Status SaslHelper::SendNegotiatePB(Socket* sock,
+                                   const MessageLite& header,
+                                   const MessageLite& msg,
+                                   const MonoTime& deadline) {
   DCHECK(sock != nullptr);
   DCHECK(header.IsInitialized()) << tag_ << ": Header must be initialized";
   DCHECK(msg.IsInitialized()) << tag_ << ": Message must be initialized";

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_helper.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_helper.h b/src/kudu/rpc/sasl_helper.h
index 45adbe3..f0a676e 100644
--- a/src/kudu/rpc/sasl_helper.h
+++ b/src/kudu/rpc/sasl_helper.h
@@ -43,7 +43,7 @@ namespace rpc {
 
 using std::string;
 
-class SaslMessagePB;
+class NegotiatePB;
 
 // Helper class which contains functionality that is common to SaslClient & SaslServer.
 // Most of these methods are convenience methods for interacting with the libsasl2 library.
@@ -93,16 +93,18 @@ class SaslHelper {
   // Check for the PLAIN SASL mechanism.
   bool IsPlainEnabled() const;
 
-  // Sanity check that the call ID is the SASL call ID.
+  // Sanity check that the call ID is the negotiation call ID.
   // Logs DFATAL if call_id does not match.
-  Status SanityCheckSaslCallId(int32_t call_id) const;
+  Status SanityCheckNegotiateCallId(int32_t call_id) const;
 
   // Parse msg from the given Slice.
-  Status ParseSaslMessage(const Slice& param_buf, SaslMessagePB* msg);
+  Status ParseNegotiatePB(const Slice& param_buf, NegotiatePB* msg);
 
   // Encode and send a message over a socket, sending the connection header if necessary.
-  Status SendSaslMessage(Socket* sock, const google::protobuf::MessageLite& header,
-      const google::protobuf::MessageLite& msg, const MonoTime& deadline);
+  Status SendNegotiatePB(Socket* sock,
+                         const google::protobuf::MessageLite& header,
+                         const google::protobuf::MessageLite& msg,
+                         const MonoTime& deadline);
 
  private:
   Status EnableMechanism(const std::string& mech);

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.cc b/src/kudu/rpc/sasl_server.cc
index 6dae445..f8d4040 100644
--- a/src/kudu/rpc/sasl_server.cc
+++ b/src/kudu/rpc/sasl_server.cc
@@ -173,36 +173,36 @@ Status SaslServer::Negotiate() {
 
   nego_ok_ = false;
   while (!nego_ok_) {
-    TRACE("Waiting for next SASL message...");
+    TRACE("Waiting for next Negotiation message...");
     RequestHeader header;
     Slice param_buf;
     RETURN_NOT_OK(ReceiveFramedMessageBlocking(sock_, &recv_buf, &header, &param_buf, deadline_));
 
-    SaslMessagePB request;
-    RETURN_NOT_OK(ParseSaslMsgRequest(header, param_buf, &request));
+    NegotiatePB request;
+    RETURN_NOT_OK(ParseNegotiatePB(header, param_buf, &request));
 
-    switch (request.state()) {
+    switch (request.step()) {
       // NEGOTIATE: They want a list of available mechanisms.
-      case SaslMessagePB::NEGOTIATE:
+      case NegotiatePB::NEGOTIATE:
         RETURN_NOT_OK(HandleNegotiateRequest(request));
         break;
 
       // INITIATE: They want to initiate negotiation based on their specified mechanism.
-      case SaslMessagePB::INITIATE:
+      case NegotiatePB::INITIATE:
         RETURN_NOT_OK(HandleInitiateRequest(request));
         break;
 
       // RESPONSE: Client sent a new request as a follow-up to a CHALLENGE response.
-      case SaslMessagePB::RESPONSE:
+      case NegotiatePB::RESPONSE:
         RETURN_NOT_OK(HandleResponseRequest(request));
         break;
 
-      // Client sent us some unsupported SASL request.
+      // Client sent us an unsupported Negotiation request.
       default: {
         TRACE("SASL Server: Received unsupported request from client");
-        Status s = Status::InvalidArgument("RPC server doesn't support SASL state in request",
-            SaslMessagePB::SaslState_Name(request.state()));
-        RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+        Status s = Status::InvalidArgument("RPC server doesn't support negotiation step in request",
+                                           NegotiatePB::NegotiateStep_Name(request.step()));
+        RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
         return s;
       }
     }
@@ -234,35 +234,36 @@ Status SaslServer::ValidateConnectionHeader(faststring* recv_buf) {
   return Status::OK();
 }
 
-Status SaslServer::ParseSaslMsgRequest(const RequestHeader& header, const Slice& param_buf,
-    SaslMessagePB* request) {
-  Status s = helper_.SanityCheckSaslCallId(header.call_id());
+Status SaslServer::ParseNegotiatePB(const RequestHeader& header,
+                                    const Slice& param_buf,
+                                    NegotiatePB* request) {
+  Status s = helper_.SanityCheckNegotiateCallId(header.call_id());
   if (!s.ok()) {
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_INVALID_RPC_HEADER, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_INVALID_RPC_HEADER, s));
   }
 
-  s = helper_.ParseSaslMessage(param_buf, request);
+  s = helper_.ParseNegotiatePB(param_buf, request);
   if (!s.ok()) {
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_DESERIALIZING_REQUEST, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_DESERIALIZING_REQUEST, s));
     return s;
   }
 
   return Status::OK();
 }
 
-Status SaslServer::SendSaslMessage(const SaslMessagePB& msg) {
+Status SaslServer::SendNegotiatePB(const NegotiatePB& msg) {
   DCHECK_NE(server_state_, SaslNegotiationState::NEW)
-      << "Must not send SASL messages before calling Init()";
+      << "Must not send Negotiate messages before calling Init()";
   DCHECK_NE(server_state_, SaslNegotiationState::NEGOTIATED)
-      << "Must not send SASL messages after Negotiate() succeeds";
+      << "Must not send Negotiate messages after Negotiate() succeeds";
 
-  // Create header with SASL-specific callId
+  // Create header with negotiation-specific callId
   ResponseHeader header;
-  header.set_call_id(kSaslCallId);
-  return helper_.SendSaslMessage(sock_, header, msg, deadline_);
+  header.set_call_id(kNegotiateCallId);
+  return helper_.SendNegotiatePB(sock_, header, msg, deadline_);
 }
 
-Status SaslServer::SendSaslError(ErrorStatusPB::RpcErrorCodePB code, const Status& err) {
+Status SaslServer::SendRpcError(ErrorStatusPB::RpcErrorCodePB code, const Status& err) {
   DCHECK_NE(server_state_, SaslNegotiationState::NEW)
       << "Must not send SASL messages before calling Init()";
   DCHECK_NE(server_state_, SaslNegotiationState::NEGOTIATED)
@@ -271,9 +272,9 @@ Status SaslServer::SendSaslError(ErrorStatusPB::RpcErrorCodePB code, const Statu
     return Status::InvalidArgument("Cannot send error message using OK status");
   }
 
-  // Create header with SASL-specific callId
+  // Create header with negotiation-specific callId
   ResponseHeader header;
-  header.set_call_id(kSaslCallId);
+  header.set_call_id(kNegotiateCallId);
   header.set_is_error(true);
 
   // Get RPC error code from Status object
@@ -281,12 +282,12 @@ Status SaslServer::SendSaslError(ErrorStatusPB::RpcErrorCodePB code, const Statu
   msg.set_code(code);
   msg.set_message(err.ToString());
 
-  RETURN_NOT_OK(helper_.SendSaslMessage(sock_, header, msg, deadline_));
+  RETURN_NOT_OK(helper_.SendNegotiatePB(sock_, header, msg, deadline_));
   TRACE("Sent SASL error: $0", ErrorStatusPB::RpcErrorCodePB_Name(code));
   return Status::OK();
 }
 
-Status SaslServer::HandleNegotiateRequest(const SaslMessagePB& request) {
+Status SaslServer::HandleNegotiateRequest(const NegotiatePB& request) {
   TRACE("SASL Server: Received NEGOTIATE request from client");
 
   // Fill in the set of features supported by the client.
@@ -305,7 +306,7 @@ Status SaslServer::HandleNegotiateRequest(const SaslMessagePB& request) {
     Status s = Status::IllegalState("SASL server mechanism list is empty!");
     LOG(ERROR) << s.ToString();
     TRACE("SASL Server: Sending FATAL_UNAUTHORIZED response to client");
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }
 
@@ -314,8 +315,8 @@ Status SaslServer::HandleNegotiateRequest(const SaslMessagePB& request) {
 }
 
 Status SaslServer::SendNegotiateResponse(const set<string>& server_mechs) {
-  SaslMessagePB response;
-  response.set_state(SaslMessagePB::NEGOTIATE);
+  NegotiatePB response;
+  response.set_step(NegotiatePB::NEGOTIATE);
 
   for (const string& mech : server_mechs) {
     response.add_auths()->set_mechanism(mech);
@@ -326,24 +327,24 @@ Status SaslServer::SendNegotiateResponse(const set<string>& server_mechs) {
     response.add_supported_features(feature);
   }
 
-  RETURN_NOT_OK(SendSaslMessage(response));
+  RETURN_NOT_OK(SendNegotiatePB(response));
   TRACE("Sent NEGOTIATE response");
   return Status::OK();
 }
 
 
-Status SaslServer::HandleInitiateRequest(const SaslMessagePB& request) {
+Status SaslServer::HandleInitiateRequest(const NegotiatePB& request) {
   TRACE("SASL Server: Received INITIATE request from client");
 
   if (request.auths_size() != 1) {
     Status s = Status::NotAuthorized(StringPrintf(
           "SASL INITIATE request must include exactly one SaslAuth section, found: %d",
           request.auths_size()));
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }
 
-  const SaslMessagePB::SaslAuth& auth = request.auths(0);
+  const NegotiatePB::SaslAuth& auth = request.auths(0);
   TRACE("SASL Server: Client requested to use mechanism: $0", auth.mechanism());
 
   // Security issue to display this. Commented out but left for debugging purposes.
@@ -363,7 +364,7 @@ Status SaslServer::HandleInitiateRequest(const SaslMessagePB& request) {
           &server_out_len);         // Output len.
     });
   if (PREDICT_FALSE(!s.ok() && !s.IsIncomplete())) {
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }
   negotiated_mech_ = SaslMechanism::value_of(auth.mechanism());
@@ -379,32 +380,32 @@ Status SaslServer::HandleInitiateRequest(const SaslMessagePB& request) {
 }
 
 Status SaslServer::SendChallengeResponse(const char* challenge, unsigned clen) {
-  SaslMessagePB response;
-  response.set_state(SaslMessagePB::CHALLENGE);
+  NegotiatePB response;
+  response.set_step(NegotiatePB::CHALLENGE);
   response.mutable_token()->assign(challenge, clen);
   TRACE("SASL Server: Sending CHALLENGE response to client");
-  RETURN_NOT_OK(SendSaslMessage(response));
+  RETURN_NOT_OK(SendNegotiatePB(response));
   return Status::OK();
 }
 
 Status SaslServer::SendSuccessResponse(const char* token, unsigned tlen) {
-  SaslMessagePB response;
-  response.set_state(SaslMessagePB::SUCCESS);
+  NegotiatePB response;
+  response.set_step(NegotiatePB::SUCCESS);
   if (PREDICT_FALSE(tlen > 0)) {
     response.mutable_token()->assign(token, tlen);
   }
   TRACE("SASL Server: Sending SUCCESS response to client");
-  RETURN_NOT_OK(SendSaslMessage(response));
+  RETURN_NOT_OK(SendNegotiatePB(response));
   return Status::OK();
 }
 
 
-Status SaslServer::HandleResponseRequest(const SaslMessagePB& request) {
+Status SaslServer::HandleResponseRequest(const NegotiatePB& request) {
   TRACE("SASL Server: Received RESPONSE request from client");
 
   if (!request.has_token()) {
     Status s = Status::InvalidArgument("No token in CHALLENGE RESPONSE from client");
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }
 
@@ -420,7 +421,7 @@ Status SaslServer::HandleResponseRequest(const SaslMessagePB& request) {
           &server_out_len);         // Output len
     });
   if (!s.ok() && !s.IsIncomplete()) {
-    RETURN_NOT_OK(SendSaslError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
+    RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/2960e692/src/kudu/rpc/sasl_server.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.h b/src/kudu/rpc/sasl_server.h
index 7cbf6fd..679a7c4 100644
--- a/src/kudu/rpc/sasl_server.h
+++ b/src/kudu/rpc/sasl_server.h
@@ -116,24 +116,25 @@ class SaslServer {
   Status ValidateConnectionHeader(faststring* recv_buf);
 
   // Parse request body. If malformed, sends an error message to the client.
-  Status ParseSaslMsgRequest(const RequestHeader& header, const Slice& param_buf,
-    SaslMessagePB* request);
+  Status ParseNegotiatePB(const RequestHeader& header,
+                          const Slice& param_buf,
+                          NegotiatePB* request);
 
   // Encode and send the specified SASL message to the client.
-  Status SendSaslMessage(const SaslMessagePB& msg);
+  Status SendNegotiatePB(const NegotiatePB& msg);
 
   // Encode and send the specified RPC error message to the client.
   // Calls Status.ToString() for the embedded error message.
-  Status SendSaslError(ErrorStatusPB::RpcErrorCodePB code, const Status& err);
+  Status SendRpcError(ErrorStatusPB::RpcErrorCodePB code, const Status& err);
 
   // Handle case when client sends NEGOTIATE request.
-  Status HandleNegotiateRequest(const SaslMessagePB& request);
+  Status HandleNegotiateRequest(const NegotiatePB& request);
 
   // Send a NEGOTIATE response to the client with the list of available mechanisms.
   Status SendNegotiateResponse(const std::set<string>& server_mechs);
 
   // Handle case when client sends INITIATE request.
-  Status HandleInitiateRequest(const SaslMessagePB& request);
+  Status HandleInitiateRequest(const NegotiatePB& request);
 
   // Send a CHALLENGE response to the client with a challenge token.
   Status SendChallengeResponse(const char* challenge, unsigned clen);
@@ -142,7 +143,7 @@ class SaslServer {
   Status SendSuccessResponse(const char* token, unsigned tlen);
 
   // Handle case when client sends RESPONSE request.
-  Status HandleResponseRequest(const SaslMessagePB& request);
+  Status HandleResponseRequest(const NegotiatePB& request);
 
   string app_name_;
   Socket* sock_;


[2/2] kudu git commit: TLS-negotiation [4/n]: rename Negotiation steps

Posted by da...@apache.org.
TLS-negotiation [4/n]: rename Negotiation steps

This makes it more clear which steps are SASL-specific.

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

Branch: refs/heads/master
Commit: 9d0421e807887a83f011e9cdc265901593b2fdf8
Parents: 2960e69
Author: Dan Burkert <da...@apache.org>
Authored: Fri Dec 16 16:36:15 2016 -0800
Committer: Dan Burkert <da...@apache.org>
Committed: Mon Jan 23 23:25:36 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kudu/client/SecureRpcHelper.java |  9 +++--
 src/kudu/rpc/rpc_header.proto                   | 37 ++++++++++++--------
 src/kudu/rpc/sasl_client.cc                     | 26 +++++++-------
 src/kudu/rpc/sasl_server.cc                     | 16 ++++-----
 4 files changed, 48 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/9d0421e8/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
----------------------------------------------------------------------
diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java b/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
index 90cd167..0ec6547 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/SecureRpcHelper.java
@@ -26,7 +26,6 @@
 
 package org.apache.kudu.client;
 
-import java.security.PrivilegedExceptionAction;
 import java.util.Map;
 import java.util.Set;
 import javax.security.auth.Subject;
@@ -133,10 +132,10 @@ public class SecureRpcHelper {
         case NEGOTIATE:
           handleNegotiateResponse(chan, response);
           break;
-        case CHALLENGE:
+        case SASL_CHALLENGE:
           handleChallengeResponse(chan, response);
           break;
-        case SUCCESS:
+        case SASL_SUCCESS:
           handleSuccessResponse(chan);
           break;
         default:
@@ -227,7 +226,7 @@ public class SecureRpcHelper {
     if (saslToken != null) {
       builder.setToken(ZeroCopyLiteralByteString.wrap(saslToken));
     }
-    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.INITIATE);
+    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.SASL_INITIATE);
     builder.addAuths(negotiatedAuth);
     sendSaslMessage(chan, builder.build());
   }
@@ -240,7 +239,7 @@ public class SecureRpcHelper {
     }
     RpcHeader.NegotiatePB.Builder builder = RpcHeader.NegotiatePB.newBuilder();
     builder.setToken(ZeroCopyLiteralByteString.wrap(saslToken));
-    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.RESPONSE);
+    builder.setStep(RpcHeader.NegotiatePB.NegotiateStep.SASL_RESPONSE);
     sendSaslMessage(chan, builder.build());
   }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9d0421e8/src/kudu/rpc/rpc_header.proto
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/rpc_header.proto b/src/kudu/rpc/rpc_header.proto
index f2e4c8c..26e56ed 100644
--- a/src/kudu/rpc/rpc_header.proto
+++ b/src/kudu/rpc/rpc_header.proto
@@ -76,33 +76,42 @@ enum RpcFeatureFlag {
 // Message type passed back & forth for the SASL negotiation.
 message NegotiatePB {
   enum NegotiateStep {
-    UNKNOWN   = 999;
-    SUCCESS   = 0;
-    NEGOTIATE = 1;
-    INITIATE  = 2;
-    CHALLENGE = 3;
-    RESPONSE  = 4;
+    UNKNOWN        = 999;
+    NEGOTIATE      = 1;
+    SASL_SUCCESS   = 0;
+    SASL_INITIATE  = 2;
+    SASL_CHALLENGE = 3;
+    SASL_RESPONSE  = 4;
   }
 
   message SaslAuth {
-    required string mechanism = 2;  // Standard SASL mechanism, i.e. ANONYMOUS, PLAIN, GSSAPI.
+    // The SASL mechanism, i.e. 'PLAIN' or 'GSSAPI'.
+    required string mechanism = 2;
 
     // Deprecated: no longer used.
     optional string DEPRECATED_method = 1;
     optional bytes DEPRECATED_challenge = 5 [(REDACT) = true];
   }
 
-  // When the client sends its first NEGOTIATE message, it sends its set of supported
-  // RPC system features. In the response to this message, the server sends back its own.
-  // This allows the two peers to agree on whether newer extensions of the
-  // RPC system may be used on this connection. We use a list of features rather than
-  // a simple version number to make it easier for the Java and C++ clients to implement
-  // features in different orders while still maintaining compatibility, as well as
-  // to simplify backporting of features out-of-order.
+  // When the client sends its NEGOTIATE step message, it sends its set of
+  // supported RPC system features. In the response to this message, the server
+  // sends back its own.  This allows the two peers to agree on whether newer
+  // extensions of the RPC system may be used on this connection. We use a list
+  // of features rather than a simple version number to make it easier for the
+  // Java and C++ clients to implement features in different orders while still
+  // maintaining compatibility, as well as to simplify backporting of features
+  // out-of-order.
   repeated RpcFeatureFlag supported_features = 1;
 
+  // The current negotiation step.
   required NegotiateStep step = 2;
+
+  // The SASL token, containing either the challenge during the SASL_CHALLENGE
+  // step, or the response during the SASL_RESPONSE step.
   optional bytes token        = 3 [(REDACT) = true];
+
+  // During the NEGOTIATE step, contains the supported SASL mechanisms.
+  // During the SASL_INITIATE step, contains the single chosen SASL mechanism.
   repeated SaslAuth auths     = 4;
 }
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/9d0421e8/src/kudu/rpc/sasl_client.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_client.cc b/src/kudu/rpc/sasl_client.cc
index 6e0cbab..c88e3cc 100644
--- a/src/kudu/rpc/sasl_client.cc
+++ b/src/kudu/rpc/sasl_client.cc
@@ -217,13 +217,13 @@ Status SaslClient::Negotiate() {
         RETURN_NOT_OK(HandleNegotiateResponse(response));
         break;
 
-      // CHALLENGE: Server sent us a follow-up to an INITIATE or RESPONSE request.
-      case NegotiatePB::CHALLENGE:
+      // SASL_CHALLENGE: Server sent us a follow-up to an SASL_INITIATE or SASL_RESPONSE request.
+      case NegotiatePB::SASL_CHALLENGE:
         RETURN_NOT_OK(HandleChallengeResponse(response));
         break;
 
-      // SUCCESS: Server has accepted our authentication request. Negotiation successful.
-      case NegotiatePB::SUCCESS:
+      // SASL_SUCCESS: Server has accepted our authentication request. Negotiation successful.
+      case NegotiatePB::SASL_SUCCESS:
         RETURN_NOT_OK(HandleSuccessResponse(response));
         break;
 
@@ -282,10 +282,10 @@ Status SaslClient::SendNegotiateMessage() {
 Status SaslClient::SendInitiateMessage(const NegotiatePB_SaslAuth& auth,
     const char* init_msg, unsigned init_msg_len) {
   NegotiatePB msg;
-  msg.set_step(NegotiatePB::INITIATE);
+  msg.set_step(NegotiatePB::SASL_INITIATE);
   msg.mutable_token()->assign(init_msg, init_msg_len);
   msg.add_auths()->CopyFrom(auth);
-  TRACE("SASL Client: Sending INITIATE request to server.");
+  TRACE("SASL Client: Sending SASL_INITIATE request to server.");
   RETURN_NOT_OK(SendNegotiatePB(msg));
   nego_response_expected_ = true;
   return Status::OK();
@@ -293,9 +293,9 @@ Status SaslClient::SendInitiateMessage(const NegotiatePB_SaslAuth& auth,
 
 Status SaslClient::SendResponseMessage(const char* resp_msg, unsigned resp_msg_len) {
   NegotiatePB reply;
-  reply.set_step(NegotiatePB::RESPONSE);
+  reply.set_step(NegotiatePB::SASL_RESPONSE);
   reply.mutable_token()->assign(resp_msg, resp_msg_len);
-  TRACE("SASL Client: Sending RESPONSE request to server.");
+  TRACE("SASL Client: Sending SASL_RESPONSE request to server.");
   RETURN_NOT_OK(SendNegotiatePB(reply));
   nego_response_expected_ = true;
   return Status::OK();
@@ -396,13 +396,13 @@ Status SaslClient::HandleNegotiateResponse(const NegotiatePB& response) {
 }
 
 Status SaslClient::HandleChallengeResponse(const NegotiatePB& response) {
-  TRACE("SASL Client: Received CHALLENGE response from server");
+  TRACE("SASL Client: Received SASL_CHALLENGE response from server");
   if (PREDICT_FALSE(nego_ok_)) {
-    LOG(DFATAL) << "Server sent CHALLENGE response after client library returned SASL_OK";
+    LOG(DFATAL) << "Server sent SASL_CHALLENGE response after client library returned SASL_OK";
   }
 
   if (PREDICT_FALSE(!response.has_token())) {
-    return Status::InvalidArgument("No token in CHALLENGE response from server");
+    return Status::InvalidArgument("No token in SASL_CHALLENGE response from server");
   }
 
   const char* out = nullptr;
@@ -416,7 +416,7 @@ Status SaslClient::HandleChallengeResponse(const NegotiatePB& response) {
 }
 
 Status SaslClient::HandleSuccessResponse(const NegotiatePB& response) {
-  TRACE("SASL Client: Received SUCCESS response from server");
+  TRACE("SASL Client: Received SASL_SUCCESS response from server");
   if (!nego_ok_) {
     const char* out = nullptr;
     unsigned out_len = 0;
@@ -427,7 +427,7 @@ Status SaslClient::HandleSuccessResponse(const NegotiatePB& response) {
     }
     RETURN_NOT_OK(s);
     if (out_len > 0) {
-      return Status::IllegalState("SASL client library generated spurious token after SUCCESS",
+      return Status::IllegalState("SASL client library generated spurious token after SASL_SUCCESS",
           string(out, out_len));
     }
     CHECK(nego_ok_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/9d0421e8/src/kudu/rpc/sasl_server.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/sasl_server.cc b/src/kudu/rpc/sasl_server.cc
index f8d4040..d8c627d 100644
--- a/src/kudu/rpc/sasl_server.cc
+++ b/src/kudu/rpc/sasl_server.cc
@@ -188,12 +188,12 @@ Status SaslServer::Negotiate() {
         break;
 
       // INITIATE: They want to initiate negotiation based on their specified mechanism.
-      case NegotiatePB::INITIATE:
+      case NegotiatePB::SASL_INITIATE:
         RETURN_NOT_OK(HandleInitiateRequest(request));
         break;
 
-      // RESPONSE: Client sent a new request as a follow-up to a CHALLENGE response.
-      case NegotiatePB::RESPONSE:
+      // RESPONSE: Client sent a new request as a follow-up to a SASL_CHALLENGE response.
+      case NegotiatePB::SASL_RESPONSE:
         RETURN_NOT_OK(HandleResponseRequest(request));
         break;
 
@@ -381,20 +381,20 @@ Status SaslServer::HandleInitiateRequest(const NegotiatePB& request) {
 
 Status SaslServer::SendChallengeResponse(const char* challenge, unsigned clen) {
   NegotiatePB response;
-  response.set_step(NegotiatePB::CHALLENGE);
+  response.set_step(NegotiatePB::SASL_CHALLENGE);
   response.mutable_token()->assign(challenge, clen);
-  TRACE("SASL Server: Sending CHALLENGE response to client");
+  TRACE("SASL Server: Sending SASL_CHALLENGE response to client");
   RETURN_NOT_OK(SendNegotiatePB(response));
   return Status::OK();
 }
 
 Status SaslServer::SendSuccessResponse(const char* token, unsigned tlen) {
   NegotiatePB response;
-  response.set_step(NegotiatePB::SUCCESS);
+  response.set_step(NegotiatePB::SASL_SUCCESS);
   if (PREDICT_FALSE(tlen > 0)) {
     response.mutable_token()->assign(token, tlen);
   }
-  TRACE("SASL Server: Sending SUCCESS response to client");
+  TRACE("SASL Server: Sending SASL_SUCCESS response to client");
   RETURN_NOT_OK(SendNegotiatePB(response));
   return Status::OK();
 }
@@ -404,7 +404,7 @@ Status SaslServer::HandleResponseRequest(const NegotiatePB& request) {
   TRACE("SASL Server: Received RESPONSE request from client");
 
   if (!request.has_token()) {
-    Status s = Status::InvalidArgument("No token in CHALLENGE RESPONSE from client");
+    Status s = Status::InvalidArgument("No token in SASL_RESPONSE from client");
     RETURN_NOT_OK(SendRpcError(ErrorStatusPB::FATAL_UNAUTHORIZED, s));
     return s;
   }