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 2017/02/14 18:47:00 UTC

[1/2] kudu git commit: rpc: move UserCredentials to separate .h/cc file

Repository: kudu
Updated Branches:
  refs/heads/master a09450465 -> a29871f30


rpc: move UserCredentials to separate .h/cc file

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

Branch: refs/heads/master
Commit: c4b44dc141dbd03bf332e86db9a7ea73197679dd
Parents: a094504
Author: Todd Lipcon <to...@apache.org>
Authored: Tue Feb 14 00:00:33 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 14 17:20:22 2017 +0000

----------------------------------------------------------------------
 src/kudu/master/master_service.cc |  2 +-
 src/kudu/rpc/CMakeLists.txt       |  1 +
 src/kudu/rpc/outbound_call.cc     | 35 -------------------
 src/kudu/rpc/outbound_call.h      | 31 +----------------
 src/kudu/rpc/user_credentials.cc  | 63 ++++++++++++++++++++++++++++++++++
 src/kudu/rpc/user_credentials.h   | 57 ++++++++++++++++++++++++++++++
 6 files changed, 123 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/master/master_service.cc
----------------------------------------------------------------------
diff --git a/src/kudu/master/master_service.cc b/src/kudu/master/master_service.cc
index 2ea29db..53b2503 100644
--- a/src/kudu/master/master_service.cc
+++ b/src/kudu/master/master_service.cc
@@ -30,7 +30,7 @@
 #include "kudu/master/ts_descriptor.h"
 #include "kudu/master/ts_manager.h"
 #include "kudu/rpc/rpc_context.h"
-#include "kudu/rpc/outbound_call.h" // for UserCredentials
+#include "kudu/rpc/user_credentials.h"
 #include "kudu/server/webserver.h"
 #include "kudu/util/flag_tags.h"
 #include "kudu/util/pb_util.h"

http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/rpc/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/CMakeLists.txt b/src/kudu/rpc/CMakeLists.txt
index fa21551..95b4268 100644
--- a/src/kudu/rpc/CMakeLists.txt
+++ b/src/kudu/rpc/CMakeLists.txt
@@ -66,6 +66,7 @@ set(KRPC_SRCS
     service_if.cc
     service_pool.cc
     service_queue.cc
+    user_credentials.cc
     transfer.cc
 )
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/rpc/outbound_call.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/outbound_call.cc b/src/kudu/rpc/outbound_call.cc
index 6952cc3..dafcf2f 100644
--- a/src/kudu/rpc/outbound_call.cc
+++ b/src/kudu/rpc/outbound_call.cc
@@ -354,41 +354,6 @@ void OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
 }
 
 ///
-/// UserCredentials
-///
-
-UserCredentials::UserCredentials() {}
-
-bool UserCredentials::has_real_user() const {
-  return !real_user_.empty();
-}
-
-void UserCredentials::set_real_user(const string& real_user) {
-  real_user_ = real_user;
-}
-
-void UserCredentials::CopyFrom(const UserCredentials& other) {
-  real_user_ = other.real_user_;
-}
-
-string UserCredentials::ToString() const {
-  // Does not print the password.
-  return StringPrintf("{real_user=%s}", real_user_.c_str());
-}
-
-size_t UserCredentials::HashCode() const {
-  size_t seed = 0;
-  if (has_real_user()) {
-    boost::hash_combine(seed, real_user());
-  }
-  return seed;
-}
-
-bool UserCredentials::Equals(const UserCredentials& other) const {
-  return real_user() == other.real_user();
-}
-
-///
 /// ConnectionId
 ///
 

http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/rpc/outbound_call.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/outbound_call.h b/src/kudu/rpc/outbound_call.h
index 7df4a31..fbaaa31 100644
--- a/src/kudu/rpc/outbound_call.h
+++ b/src/kudu/rpc/outbound_call.h
@@ -30,6 +30,7 @@
 #include "kudu/rpc/remote_method.h"
 #include "kudu/rpc/response_callback.h"
 #include "kudu/rpc/transfer.h"
+#include "kudu/rpc/user_credentials.h"
 #include "kudu/util/locks.h"
 #include "kudu/util/monotime.h"
 #include "kudu/util/net/sockaddr.h"
@@ -52,36 +53,6 @@ class InboundTransfer;
 class RpcCallInProgressPB;
 class RpcController;
 
-// Client-side user credentials, such as a user's username & password.
-// In the future, we will add Kerberos credentials.
-//
-// TODO(mpercy): this is actually used server side too -- should
-// we instead introduce a RemoteUser class or something?
-// TODO(todd): this should move into a standalone header.
-class UserCredentials {
- public:
-   UserCredentials();
-
-  // 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_; }
-
-  // Copy state from another object to this one.
-  void CopyFrom(const UserCredentials& other);
-
-  // Returns a string representation of the object, not including the password field.
-  std::string ToString() const;
-
-  std::size_t HashCode() const;
-  bool Equals(const UserCredentials& other) const;
-
- private:
-  // Remember to update HashCode() and Equals() when new fields are added.
-  std::string real_user_;
-
-  DISALLOW_COPY_AND_ASSIGN(UserCredentials);
-};
 
 // Used to key on Connection information.
 // For use as a key in an unordered STL collection, use ConnectionIdHash and ConnectionIdEqual.

http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/rpc/user_credentials.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/user_credentials.cc b/src/kudu/rpc/user_credentials.cc
new file mode 100644
index 0000000..5065271
--- /dev/null
+++ b/src/kudu/rpc/user_credentials.cc
@@ -0,0 +1,63 @@
+// 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/user_credentials.h"
+
+#include <string>
+
+#include <boost/functional/hash.hpp>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+UserCredentials::UserCredentials() {}
+
+bool UserCredentials::has_real_user() const {
+  return !real_user_.empty();
+}
+
+void UserCredentials::set_real_user(const string& real_user) {
+  real_user_ = real_user;
+}
+
+void UserCredentials::CopyFrom(const UserCredentials& other) {
+  real_user_ = other.real_user_;
+}
+
+string UserCredentials::ToString() const {
+  // Does not print the password.
+  return strings::Substitute("{real_user=$0}", real_user_);
+}
+
+size_t UserCredentials::HashCode() const {
+  size_t seed = 0;
+  if (has_real_user()) {
+    boost::hash_combine(seed, real_user());
+  }
+  return seed;
+}
+
+bool UserCredentials::Equals(const UserCredentials& other) const {
+  return real_user() == other.real_user();
+}
+
+} // namespace rpc
+} // namespace kudu

http://git-wip-us.apache.org/repos/asf/kudu/blob/c4b44dc1/src/kudu/rpc/user_credentials.h
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/user_credentials.h b/src/kudu/rpc/user_credentials.h
new file mode 100644
index 0000000..ebeccb2
--- /dev/null
+++ b/src/kudu/rpc/user_credentials.h
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+#pragma once
+
+#include <string>
+
+#include "kudu/gutil/macros.h"
+
+namespace kudu {
+namespace rpc {
+
+// Client-side user credentials, such as a user's username & password.
+// In the future, we will add Kerberos credentials.
+//
+// TODO(mpercy): this is actually used server side too -- should
+// we instead introduce a RemoteUser class or something?
+class UserCredentials {
+ public:
+   UserCredentials();
+
+  // 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_; }
+
+  // Copy state from another object to this one.
+  void CopyFrom(const UserCredentials& other);
+
+  // Returns a string representation of the object, not including the password field.
+  std::string ToString() const;
+
+  std::size_t HashCode() const;
+  bool Equals(const UserCredentials& other) const;
+
+ private:
+  // Remember to update HashCode() and Equals() when new fields are added.
+  std::string real_user_;
+
+  DISALLOW_COPY_AND_ASSIGN(UserCredentials);
+};
+
+} // namespace rpc
+} // namespace kudu


[2/2] kudu git commit: Fix TLS_AUTHENTICATION_ONLY detection

Posted by to...@apache.org.
Fix TLS_AUTHENTICATION_ONLY detection

The patch which added support for TLS_AUTHENTICATION_ONLY had a serious bug: it
always got enabled due to a typo in Socket::IsLoopbackConnection. This fixes the
typo and also adds some trace messages in negotiation when TLS-only auth
is negotiated.

I manually verified on an Impala cluster that tshark showed encrypted traffic
between nodes and plaintext on the loopback interface after fixing this issue
(previously I saw plaintext everywhere!)

Change-Id: I76fd3bb7c64c6b831f406912852b064f9fec3d00
Reviewed-on: http://gerrit.cloudera.org:8080/5996
Tested-by: Kudu Jenkins
Reviewed-by: Dan Burkert <da...@apache.org>


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

Branch: refs/heads/master
Commit: a29871f306464c5ef5f586431ac6f4f1bed026ae
Parents: c4b44dc
Author: Todd Lipcon <to...@apache.org>
Authored: Mon Feb 13 23:21:11 2017 -0800
Committer: Todd Lipcon <to...@apache.org>
Committed: Tue Feb 14 18:41:38 2017 +0000

----------------------------------------------------------------------
 src/kudu/rpc/client_negotiation.cc | 1 +
 src/kudu/rpc/server_negotiation.cc | 1 +
 src/kudu/util/net/socket.cc        | 2 +-
 3 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kudu/blob/a29871f3/src/kudu/rpc/client_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/client_negotiation.cc b/src/kudu/rpc/client_negotiation.cc
index 4319183..934b714 100644
--- a/src/kudu/rpc/client_negotiation.cc
+++ b/src/kudu/rpc/client_negotiation.cc
@@ -418,6 +418,7 @@ Status ClientNegotiation::HandleTlsHandshake(const NegotiatePB& response) {
 
   if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
       ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    TRACE("Negotiated auth-only TLS");
     return tls_handshake_.FinishNoWrap(*socket_);
   }
   return tls_handshake_.Finish(&socket_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/a29871f3/src/kudu/rpc/server_negotiation.cc
----------------------------------------------------------------------
diff --git a/src/kudu/rpc/server_negotiation.cc b/src/kudu/rpc/server_negotiation.cc
index 00c66e8..078acc7 100644
--- a/src/kudu/rpc/server_negotiation.cc
+++ b/src/kudu/rpc/server_negotiation.cc
@@ -408,6 +408,7 @@ Status ServerNegotiation::HandleTlsHandshake(const NegotiatePB& request) {
   // TLS handshake is finished.
   if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
       ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    TRACE("Negotiated auth-only TLS");
     return tls_handshake_.FinishNoWrap(*socket_);
   }
   return tls_handshake_.Finish(&socket_);

http://git-wip-us.apache.org/repos/asf/kudu/blob/a29871f3/src/kudu/util/net/socket.cc
----------------------------------------------------------------------
diff --git a/src/kudu/util/net/socket.cc b/src/kudu/util/net/socket.cc
index 85e858a..c0945ca 100644
--- a/src/kudu/util/net/socket.cc
+++ b/src/kudu/util/net/socket.cc
@@ -299,7 +299,7 @@ Status Socket::GetPeerAddress(Sockaddr *cur_addr) const {
 bool Socket::IsLoopbackConnection() const {
   Sockaddr local, remote;
   if (!GetSocketAddress(&local).ok()) return false;
-  if (!GetSocketAddress(&remote).ok()) return false;
+  if (!GetPeerAddress(&remote).ok()) return false;
 
   // Compare without comparing ports.
   local.set_port(0);