You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kudu.apache.org by al...@apache.org on 2021/04/30 21:09:30 UTC

[kudu] branch master updated: [rpc] add transport-specific info for RPC connections

This is an automated email from the ASF dual-hosted git repository.

alexey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kudu.git


The following commit(s) were added to refs/heads/master by this push:
     new fb4017c  [rpc] add transport-specific info for RPC connections
fb4017c is described below

commit fb4017ca876c20b7a04b4cb1756479daf8874db7
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Thu Apr 29 18:41:31 2021 -0700

    [rpc] add transport-specific info for RPC connections
    
    This patch transport details to the info on RPC connections produced by
    the /rpcz endpoint of the kudu-master/kudu-tserver embedded webserver:
      * TCP maximum segment size
      * if applicable, TLS protocol version and cipher suite description
        used to negotiate the connection
    
    I didn't add any tests, but I manually verified that with this patch
    kudu-master and kudu-tserver output elements similar to the one below
    in the "inbound_connections" section for their '/rpcz' HTTP endpoint:
    
    {
        "remote_ip": "10.17.240.17:49038",
        "state": "OPEN",
        "remote_user_credentials": "{username='joe'}",
        "socket_stats": {
            "rtt": 10875,
            "rttvar": 16500,
            "snd_cwnd": 10,
            "total_retrans": 0,
            "send_queue_bytes": 0,
            "receive_queue_bytes": 0,
            "send_bytes_per_sec": 1331494
        },
        "transport_details": {
            "tcp": {
                "max_segment_size": 1448
            },
            "tls": {
                "protocol": "TLSv1.3",
                "cipher_suite": "TLS_AES_128_GCM_SHA256 TLSv1.3 Kx=any Au=any Enc=AESGCM(128) Mac=AEAD"
            }
        }
    }
    
    Change-Id: Ib712f98b93dd5ee12c2083467261af02863c18e8
    Reviewed-on: http://gerrit.cloudera.org:8080/17369
    Reviewed-by: Andrew Wong <aw...@cloudera.com>
    Tested-by: Alexey Serbin <as...@cloudera.com>
---
 src/kudu/rpc/connection.cc           | 44 +++++++++++++++++++++++++++++++++---
 src/kudu/rpc/connection.h            |  4 ++++
 src/kudu/rpc/rpc_introspection.proto | 34 ++++++++++++++++++++++++++--
 src/kudu/security/openssl_util.cc    | 22 ++++++++++++++++++
 src/kudu/security/openssl_util.h     |  8 +++++++
 src/kudu/security/tls_socket.cc      |  9 ++++++++
 src/kudu/security/tls_socket.h       |  7 ++++++
 7 files changed, 123 insertions(+), 5 deletions(-)

diff --git a/src/kudu/rpc/connection.cc b/src/kudu/rpc/connection.cc
index ddd715a..e211d53 100644
--- a/src/kudu/rpc/connection.cc
+++ b/src/kudu/rpc/connection.cc
@@ -18,6 +18,7 @@
 #include "kudu/rpc/connection.h"
 
 #include <netinet/in.h>
+#include <netinet/tcp.h>
 #include <string.h>
 
 #include <algorithm>
@@ -44,18 +45,20 @@
 #include "kudu/rpc/rpc_header.pb.h"
 #include "kudu/rpc/rpc_introspection.pb.h"
 #include "kudu/rpc/transfer.h"
+#include "kudu/security/tls_socket.h"
+#include "kudu/util/errno.h"
 #include "kudu/util/faststring.h"
 #include "kudu/util/net/sockaddr.h"
 #include "kudu/util/net/socket.h"
 #include "kudu/util/slice.h"
 #include "kudu/util/status.h"
 
+#include <sys/socket.h>
 #ifdef __linux__
 #include <sys/ioctl.h>
-#include <sys/socket.h>
-#include <linux/tcp.h>
 #endif
 
+using kudu::security::TlsSocket;
 using std::includes;
 using std::set;
 using std::shared_ptr;
@@ -887,7 +890,7 @@ Status Connection::DumpPB(const DumpConnectionsRequestPB& req,
 
   if (direction_ == CLIENT) {
     for (const car_map_t::value_type& entry : awaiting_response_) {
-      CallAwaitingResponse *c = entry.second;
+      CallAwaitingResponse* c = entry.second;
       if (c->call) {
         c->call->DumpPB(req, resp->add_calls_in_flight());
       }
@@ -916,6 +919,11 @@ Status Connection::DumpPB(const DumpConnectionsRequestPB& req,
                 "could not fill in TCP info for RPC connection");
   }
 #endif // __linux__
+
+  if (negotiation_complete_ && remote_.is_ip()) {
+    WARN_NOT_OK(GetTransportDetailsPB(resp->mutable_transport_details()),
+                "could not fill in transport info for RPC connection");
+  }
   return Status::OK();
 }
 
@@ -997,5 +1005,35 @@ Status Connection::GetSocketStatsPB(SocketStatsPB* pb) const {
 }
 #endif // __linux__
 
+Status Connection::GetTransportDetailsPB(TransportDetailsPB* pb) const {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  DCHECK(pb);
+
+  // As for the dynamic_cast below: this is not very elegant or performant code,
+  // but introducing a generic virtual method with vague semantics into the base
+  // Socket class doesn't look like a good choice either. Also, the
+  // GetTransportDetailsPB() method isn't supposed to be a part of any hot path.
+  const TlsSocket* tls_socket = dynamic_cast<TlsSocket*>(socket_.get());
+  if (tls_socket) {
+    auto* tls = pb->mutable_tls();
+    tls->set_protocol(tls_socket->GetProtocolName());
+    tls->set_cipher_suite(tls_socket->GetCipherDescription());
+  }
+
+  int fd = socket_->GetFd();
+  CHECK_GE(fd, 0);
+  int32_t max_seg_size = 0;
+  socklen_t optlen = sizeof(max_seg_size);
+  int ret = ::getsockopt(fd, IPPROTO_TCP, TCP_MAXSEG, &max_seg_size, &optlen);
+  if (ret) {
+    int err = errno;
+    return Status::NetworkError(
+        "getsockopt(TCP_MAXSEG) failed", ErrnoToString(err), err);
+  }
+  pb->mutable_tcp()->set_max_segment_size(max_seg_size);
+
+  return Status::OK();
+}
+
 } // namespace rpc
 } // namespace kudu
diff --git a/src/kudu/rpc/connection.h b/src/kudu/rpc/connection.h
index 5ec15e9..5b3638e 100644
--- a/src/kudu/rpc/connection.h
+++ b/src/kudu/rpc/connection.h
@@ -22,6 +22,7 @@
 #include <memory>
 #include <set>
 #include <string>
+#include <type_traits>
 #include <unordered_map>
 #include <utility>
 
@@ -53,6 +54,7 @@ class ReactorThread;
 class RpcConnectionPB;
 class RpczStore;
 class SocketStatsPB;
+class TransportDetailsPB;
 
 enum class CredentialsPolicy;
 
@@ -320,6 +322,8 @@ class Connection : public RefCountedThreadSafe<Connection> {
 
   Status GetSocketStatsPB(SocketStatsPB* pb) const;
 
+  Status GetTransportDetailsPB(TransportDetailsPB* pb) const;
+
   // The reactor thread that created this connection.
   ReactorThread* const reactor_thread_;
 
diff --git a/src/kudu/rpc/rpc_introspection.proto b/src/kudu/rpc/rpc_introspection.proto
index 3e4facd..9a81372 100644
--- a/src/kudu/rpc/rpc_introspection.proto
+++ b/src/kudu/rpc/rpc_introspection.proto
@@ -74,7 +74,34 @@ message SocketStatsPB {
 
   // Calculated sender throughput.
   optional int64 send_bytes_per_sec = 13;
-};
+}
+
+// Transport-related information for an RPC connection.
+message TransportDetailsPB {
+
+  // TCP-specific details.
+  message TcpDetails {
+    // Maximum segment size for the packets: this directly maps into the
+    // TCP_MAXSEG socket option.
+    optional int32 max_segment_size = 1;
+  }
+
+  // TLS-specific details.
+  //
+  // NOTE: TLS/SSL doesn't map nicely into a single layer of the TCP/IP or
+  // the OSI model, but intuitively that's something related to the transport.
+  message TlsDetails {
+    // The name of the TLS protocol negotiated to protect the connection
+    // (e.g. TLSv1.3).
+    optional string protocol = 1;
+
+    // Description of the TLS cipher suite used.
+    optional string cipher_suite = 2;
+  }
+
+  optional TcpDetails tcp = 1;
+  optional TlsDetails tls = 2;
+}
 
 // Debugging information about a currently-open RPC connection.
 message RpcConnectionPB {
@@ -82,7 +109,7 @@ message RpcConnectionPB {
     UNKNOWN = 999;
     NEGOTIATING = 0;  // Connection is still being negotiated.
     OPEN = 1;         // Connection is active.
-  };
+  }
 
   required string remote_ip = 1;
   required StateType state = 2;
@@ -93,6 +120,9 @@ message RpcConnectionPB {
 
   // Information on the actual TCP connection as reported by the kernel.
   optional SocketStatsPB socket_stats = 6;
+
+  // Transport-specific details.
+  optional TransportDetailsPB transport_details = 7;
 }
 
 message DumpConnectionsRequestPB {
diff --git a/src/kudu/security/openssl_util.cc b/src/kudu/security/openssl_util.cc
index 3d9544c..024c7ed 100644
--- a/src/kudu/security/openssl_util.cc
+++ b/src/kudu/security/openssl_util.cc
@@ -367,5 +367,27 @@ Status GetPasswordFromShellCommand(const string& cmd, string* password) {
   return Status::OK();
 }
 
+string GetProtocolName(const SSL* ssl) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  return SSL_get_version(ssl);
+}
+
+string GetCipherDescription(const SSL* ssl) {
+  SCOPED_OPENSSL_NO_PENDING_ERRORS;
+  const SSL_CIPHER* cipher = SSL_get_current_cipher(ssl);
+  if (!cipher) {
+    return "NONE";
+  }
+  char buf[512];
+  const char* description = SSL_CIPHER_description(cipher, buf, sizeof(buf));
+  if (!description) {
+    return "NONE";
+  }
+  string ret(description);
+  StripTrailingNewline(&ret);
+  StripDupCharacters(&ret, ' ', 0);
+  return ret;
+}
+
 } // namespace security
 } // namespace kudu
diff --git a/src/kudu/security/openssl_util.h b/src/kudu/security/openssl_util.h
index b2f936d..03769bd 100644
--- a/src/kudu/security/openssl_util.h
+++ b/src/kudu/security/openssl_util.h
@@ -116,6 +116,14 @@ std::string GetSSLErrorDescription(int error_code);
 // An error Status object is returned otherwise.
 Status GetPasswordFromShellCommand(const std::string& cmd, std::string* password);
 
+// Retrieve the negotiated TLS protocol version. Only valid after the
+// TLS handshake is complete.
+std::string GetProtocolName(const SSL* ssl);
+
+// Retrive the description of the negotiated TLS cipher.
+// Only valid to call after the handshake is complete.
+std::string GetCipherDescription(const SSL* ssl);
+
 // A generic wrapper for OpenSSL structures.
 template <typename T>
 using c_unique_ptr = std::unique_ptr<T, std::function<void(T*)>>;
diff --git a/src/kudu/security/tls_socket.cc b/src/kudu/security/tls_socket.cc
index f1c6c39..3e63c97 100644
--- a/src/kudu/security/tls_socket.cc
+++ b/src/kudu/security/tls_socket.cc
@@ -257,5 +257,14 @@ Status TlsSocket::Close() {
   return ssl_shutdown;
 }
 
+string TlsSocket::GetProtocolName() const {
+  return ::kudu::security::GetProtocolName(ssl_.get());
+}
+
+string TlsSocket::GetCipherDescription() const {
+  return ::kudu::security::GetCipherDescription(ssl_.get());
+}
+
+
 } // namespace security
 } // namespace kudu
diff --git a/src/kudu/security/tls_socket.h b/src/kudu/security/tls_socket.h
index 94d1bf6..209eb08 100644
--- a/src/kudu/security/tls_socket.h
+++ b/src/kudu/security/tls_socket.h
@@ -21,6 +21,7 @@
 
 #include <cstdint>
 #include <memory>
+#include <string>
 
 #include "kudu/gutil/port.h"
 #include "kudu/security/openssl_util.h" // IWYU pragma: keep
@@ -50,6 +51,12 @@ class TlsSocket : public Socket {
 
   Status Close() override WARN_UNUSED_RESULT;
 
+  // Get the name of the negotiated TLS protocol for the connection.
+  std::string GetProtocolName() const;
+
+  // Get the description of the negotiated TLS cipher suite for the connection.
+  std::string GetCipherDescription() const;
+
  private:
 
   friend class TlsHandshake;