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/07 14:52:11 UTC

[kudu] 03/03: KUDU-2871 support TLSv1.3 in Kudu RPC (Java part)

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

commit 0e1a1545b827d9a3d5d50fd3d54f60bdea83f40c
Author: Alexey Serbin <al...@apache.org>
AuthorDate: Mon Mar 29 21:41:40 2021 -0700

    KUDU-2871 support TLSv1.3 in Kudu RPC (Java part)
    
    With this patch, Kudu Java client is able to talk TLSv1.3 to the server
    side.  Essentially, this is about adding TLSv1.3 ciphers into the list
    of preferred cipher suites in Negotiator.java and sending a chunk of
    data produced by the final TLSv1.3 handshake iteration to the server
    when the client senses that the TLS handshake is done.
    
    Prior to this patch, Kudu Java client didn't support TLSv1.3 for RPC
    because:
      * It didn't have ciphers required for TLSv1.3
      * It didn't send back the last chunk of data to the server side
        in the end of the TLS handshake process
    
    The former is crucial to allow Kudu Java clients of prior versions
    talking with newer, TLSv1.3-capable Kudu servers.  In other words,
    TLSv1.2 is the highest protocol version that Kudu Java clients of prior
    versions are able to use even if talking to newer Kudu servers which
    are now capable talking TLSv1.3.
    
    This patch also contains a new test scenario to verify the functionality
    of TLSv1.3-enabled RPC between a Kudu Java client and a Kudu cluster.
    The newly added scenario runs only if both the JVM and the node's
    OpenSSL library support TLSv1.3.
    
    Change-Id: I884170bdbded8b4017a80db34ecc0a755426c5c2
    Reviewed-on: http://gerrit.cloudera.org:8080/17245
    Tested-by: Alexey Serbin <as...@cloudera.com>
    Reviewed-by: Grant Henke <gr...@apache.org>
---
 .../java/org/apache/kudu/client/Negotiator.java    | 114 +++++++----
 .../apache/kudu/client/TestNegotiationTLSv13.java  | 210 +++++++++++++++++++++
 .../org/apache/kudu/client/TestNegotiator.java     |  13 +-
 3 files changed, 303 insertions(+), 34 deletions(-)

diff --git a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
index 75b1489..e570665 100644
--- a/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
+++ b/java/kudu-client/src/main/java/org/apache/kudu/client/Negotiator.java
@@ -115,26 +115,40 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
 
   /**
    * The cipher suites, in order of our preference.
-   * This list is based on the kDefaultTlsCiphers list in security_flags.cc,
-   * See that file for details on how it was derived.
+   * This list is based on the kDefaultTls13Ciphers and kDefaultTlsCiphers lists
+   * in security_flags.cc: see that file for details on how it was derived.
    */
   static final String[] PREFERRED_CIPHER_SUITES = new String[] {
-      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384", // Java 8
-      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",   // Java 8
-      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256", // Java 8
-      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",   // Java 8
-      "TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA384", // Java 7 (TLS 1.2+ only)
-      "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384",   // Java 7 (TLS 1.2+ only)
-      "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256", // Java 7 (TLS 1.2+ only)
-      "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256",   // Java 7 (TLS 1.2+ only)
-      "TLS_RSA_WITH_AES_256_GCM_SHA384",         // Java 8
-      "TLS_RSA_WITH_AES_128_GCM_SHA256",         // Java 8
-      "TLS_RSA_WITH_AES_256_CBC_SHA256",         // Java 7 (TLS 1.2+ only)
-      "TLS_RSA_WITH_AES_128_CBC_SHA256",         // Java 7 (TLS 1.2+ only)
+      "TLS_AES_128_GCM_SHA256",                   // Java 8 (updated), Java 11 (TLS 1.3+ only)
+      "TLS_AES_256_GCM_SHA384",                   // Java 8 (updated), Java 11 (TLS 1.3+ only)
+      "TLS_CHACHA20_POLY1305_SHA256",             // Java 8 (updated), Java 11 (TLS 1.3+ only)
+      "TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384",  // Java 8
+      "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",    // Java 8
+      "TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256",  // Java 8
+      "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256",    // Java 8
+      "TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA384",  // Java 7 (TLS 1.2+ only)
+      "TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA384",    // Java 7 (TLS 1.2+ only)
+      "TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256",  // Java 7 (TLS 1.2+ only)
+      "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256",    // Java 7 (TLS 1.2+ only)
+      "TLS_RSA_WITH_AES_256_GCM_SHA384",          // Java 8
+      "TLS_RSA_WITH_AES_128_GCM_SHA256",          // Java 8
+      "TLS_RSA_WITH_AES_256_CBC_SHA256",          // Java 7 (TLS 1.2+ only)
+      "TLS_RSA_WITH_AES_128_CBC_SHA256",          // Java 7 (TLS 1.2+ only)
       // The following two are critical to allow the client to connect to
       // servers running versions of OpenSSL that don't support TLS 1.2.
-      "TLS_RSA_WITH_AES_256_CBC_SHA",            // All Java versions
-      "TLS_RSA_WITH_AES_128_CBC_SHA"             // All Java versions
+      "TLS_RSA_WITH_AES_256_CBC_SHA",             // All Java versions
+      "TLS_RSA_WITH_AES_128_CBC_SHA"              // All Java versions
+  };
+
+  /**
+   * TLS protocols to enable among those supported by SSLEngine.
+   * This list is based on the kDefaultTlsMinVersion in security_flags.cc.
+   */
+  static final String[] PREFERRED_PROTOCOLS = new String[]{
+      "TLSv1.3",
+      "TLSv1.2",
+      "TLSv1.1",
+      "TLSv1",
   };
 
   private enum State {
@@ -545,21 +559,55 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
         throw new AssertionError("unreachable");
     }
     engine.setUseClientMode(true);
-    Set<String> supported = Sets.newHashSet(engine.getSupportedCipherSuites());
-    List<String> toEnable = Lists.newArrayList();
-    for (String cipher : PREFERRED_CIPHER_SUITES) {
-      if (supported.contains(cipher)) {
-        toEnable.add(cipher);
+
+    // Set the preferred cipher suites.
+    {
+      Set<String> supported = Sets.newHashSet(engine.getSupportedCipherSuites());
+      List<String> toEnable = Lists.newArrayList();
+      for (String c: PREFERRED_CIPHER_SUITES) {
+        if (supported.contains(c)) {
+          toEnable.add(c);
+        }
       }
+      if (toEnable.isEmpty()) {
+        // This should never be the case given the cipher suites we picked are
+        // supported by the standard JDK, but just in case, better to have a clear
+        // exception.
+        throw new RuntimeException("found no preferred cipher suite among supported: " +
+            Joiner.on(',').join(supported));
+      }
+      engine.setEnabledCipherSuites(toEnable.toArray(new String[0]));
+    }
+
+    // Enable preferred TLS protocols, if supported. This is to match the set
+    // of TLS protocols supported by Kudu servers: no other protocols need to
+    // be enabled. In addition, this is to enable TLSv1.3 in Java 8. The latest
+    // builds of OpenJDK 8 and Oracle JDK 8 support TLSv1.3, but TLSv1.3 is not
+    // enabled by default for SSLEngine.
+    // For example, see Oracle JDK 8u261 update release notes at
+    // https://www.oracle.com/java/technologies/javase/8u261-relnotes.html
+    // TLSv1.3 is enabled by default in Java 11, at least with OpenJDK.
+    {
+      Set<String> supported = Sets.newHashSet(engine.getSupportedProtocols());
+      List<String> toEnable = Lists.newArrayList();
+      for (String p : PREFERRED_PROTOCOLS) {
+        if (supported.contains(p)) {
+          toEnable.add(p);
+        }
+      }
+      if (toEnable.isEmpty()) {
+        // This should never be the case given that at least one preferred TLS
+        // protocol (TLSv1) is supported by the standard JDK. It's better to
+        // have a clear exception, just in case.
+        throw new RuntimeException("found no preferred TLS protocol among supported: " +
+            Joiner.on(',').join(supported));
+      }
+      engine.setEnabledProtocols(toEnable.toArray(new String[0]));
     }
-    if (toEnable.isEmpty()) {
-      // This should never be the case given the cipher suites we picked are
-      // supported by the standard JDK, but just in case, better to have a clear
-      // exception.
-      throw new RuntimeException("No preferred cipher suites were supported. " +
-          "Supported suites: " + Joiner.on(',').join(supported));
-    }
-    engine.setEnabledCipherSuites(toEnable.toArray(new String[0]));
+
+    // TODO(aserbin): maybe, check that at least one cipher is enabled per each
+    //                enabled protocol?
+
     SharableSslHandler handler = new SharableSslHandler(engine);
 
     sslEmbedder = new EmbeddedChannel(handler);
@@ -634,10 +682,10 @@ public class Negotiator extends SimpleChannelInboundHandler<CallResponse> {
     ByteString data = ByteString.copyFrom(bufs);
     if (sslHandshakeFuture.isDone()) {
       // TODO(todd): should check sslHandshakeFuture.isSuccess()
-      // TODO(danburkert): is this a correct assumption? would the
-      // client ever be "done" but also produce handshake data?
-      // if it did, would we want to encrypt the SSL message or no?
-      assert data.isEmpty();
+      if (!data.isEmpty()) {
+        // This is a case of TLSv1.3 protocol.
+        sendTunneledTls(ctx, data);
+      }
       return false;
     } else {
       assert data.size() > 0;
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiationTLSv13.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiationTLSv13.java
new file mode 100644
index 0000000..a822a0c
--- /dev/null
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiationTLSv13.java
@@ -0,0 +1,210 @@
+// 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.
+
+package org.apache.kudu.client;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.kudu.test.ClientTestUtil.createDefaultTable;
+import static org.apache.kudu.test.ClientTestUtil.loadDefaultTable;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.Reader;
+import java.security.KeyManagementException;
+import java.security.NoSuchAlgorithmException;
+import java.security.Security;
+import java.util.List;
+import java.util.Set;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.SSLEngine;
+
+import com.google.common.base.Joiner;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.io.CharStreams;
+import org.junit.Rule;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kudu.test.KuduTestHarness;
+import org.apache.kudu.test.KuduTestHarness.MasterServerConfig;
+import org.apache.kudu.test.KuduTestHarness.TabletServerConfig;
+import org.apache.kudu.test.TempDirUtils;
+import org.apache.kudu.test.cluster.KuduBinaryLocator;
+import org.apache.kudu.test.cluster.MiniKuduCluster;
+import org.apache.kudu.test.cluster.MiniKuduCluster.MiniKuduClusterBuilder;
+
+// This is a class for Kudu RPC connection negotiation test scenarios targeting
+// TLSv1.3. See TestNegotiator for pre-TLSv1.3 test scenarios.
+public class TestNegotiationTLSv13 {
+
+  static final String[] TLS13_CIPHERS = new String[]{
+      "TLS_AES_128_GCM_SHA256",
+      "TLS_AES_256_GCM_SHA384",
+      "TLS_CHACHA20_POLY1305_SHA256",
+  };
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestNegotiation.class);
+  private static final String TABLE_NAME = "tls_v_1_3_test_table";
+  private static final int NUM_ROWS = 10;
+
+  private final MiniKuduClusterBuilder clusterBuilder;
+
+  @Rule
+  public KuduTestHarness harness;
+
+  // Whether TLSv1.3 supported by both server and client side.
+  private boolean isTLSv13Supported = false;
+
+  // Check if TLSv1.3 is supported by the JVM.
+  private static boolean isTLSv13SupportedByJVM() {
+    // It seems some policy-related globals are initialized due to the
+    // SSLContext.getInstance("TLSv1.3") call below, so server certificates
+    // signed by 768-bit RSA keys aren't accepted later on when running test
+    // scenarios due to default security policies. To work around that, override
+    // the default security constraints the same way it's done
+    // in the MiniKuduCluster's constructor.
+    Security.setProperty("jdk.certpath.disabledAlgorithms", "MD2, RC4, MD5");
+    Security.setProperty("jdk.tls.disabledAlgorithms", "SSLv3, RC4, MD5");
+    try {
+      SSLContext ctx = SSLContext.getInstance("TLS");
+      ctx.init(null, null, null);
+      SSLEngine engine = ctx.createSSLEngine();
+      engine.setUseClientMode(true);
+      {
+        Set<String> supported = Sets.newHashSet(engine.getSupportedCipherSuites());
+        List<String> common = Lists.newArrayList();
+        for (String c : TLS13_CIPHERS) {
+          if (supported.contains(c)) {
+            common.add(c);
+          }
+        }
+        if (common.isEmpty()) {
+          LOG.info("client side doesn't support TLSv1.3: no common ciphers");
+          return false;
+        }
+      }
+      {
+        String[] enabled = engine.getEnabledProtocols();
+        LOG.debug("enabled TLS protocols: {}", Joiner.on(' ').join(enabled));
+        Set<String> supported = Sets.newHashSet(engine.getSupportedProtocols());
+        LOG.debug("supported TLS protocols: {}", Joiner.on(' ').join(supported));
+        if (!supported.contains("TLSv1.3")) {
+          LOG.info("client side doesn't support TLSv1.3: unsupported protocol");
+          return false;
+        }
+      }
+    } catch (KeyManagementException | NoSuchAlgorithmException e) {
+      LOG.info("client side doesn't support TLSv1.3", e);
+      return false;
+    }
+    return true;
+  }
+
+  // Check if TLSv1.3 is supported by the Kudu server side.
+  private static boolean isTLSv13SupportedByServerSide() {
+    // Try to start kudu-master requiring TLSv1.3.  It will fail to start if
+    // TLSv1.3 isn't supported either by the node's OpenSSL library or
+    // by the build environment where the kudu-master binary was built.
+    MiniKuduClusterBuilder b = new MiniKuduClusterBuilder()
+        .numMasterServers(1)
+        .numTabletServers(0)
+        .addMasterServerFlag("--time_source=system_unsync")
+        .addMasterServerFlag("--rpc_tls_min_protocol=TLSv1.3");
+    try (MiniKuduCluster c = b.build()) {
+      try {
+        // A sanity check: make sure the started processes haven't crashed.
+        // MiniKuduCluster does neither detect nor report properly on such
+        // events otherwise.
+        c.killAllMasterServers();
+      } catch (IOException e) {
+        LOG.error("unexpected exception:", e);
+        fail("kudu-master didn't actually start");
+        return false; // unreachable
+      }
+    } catch (IOException e) {
+      LOG.info("server side doesn't support TLSv1.3", e);
+      return false;
+    }
+    return true;
+  }
+
+  public TestNegotiationTLSv13() {
+    clusterBuilder = new MiniKuduClusterBuilder()
+        .numMasterServers(1)
+        .numTabletServers(3)
+        .enableKerberos();
+
+    isTLSv13Supported = isTLSv13SupportedByJVM() && isTLSv13SupportedByServerSide();
+    if (isTLSv13Supported) {
+      // By the virtue of excluding all other protocols but TLSv1.3
+      // from the list of available TLS protocols at the server side,
+      // client and server will use TLSv1.3 to negotiate a connection.
+      clusterBuilder.addMasterServerFlag("--rpc_tls_min_protocol=TLSv1.3");
+      clusterBuilder.addTabletServerFlag("--rpc_tls_min_protocol=TLSv1.3");
+    }
+
+    harness = new KuduTestHarness(clusterBuilder);
+  }
+
+  /**
+   * Make sure that Kudu Java client is able to negotiate RPC connections
+   * protected by TLSv1.3 with Kudu servers. By the virtue of excluding all
+   * other protocols but TLSv1.3 from the list of available TLS protocols
+   * at the server side, this scenario verifies that Kudu Java client is able to
+   * work with a secure Kudu cluster using TLSv1.3.
+   *
+   * Using the JUnit's terminology, this test scenario is conditionally run only
+   * if both the client and the server sides support TLSv1.3.
+   */
+  @Test
+  @MasterServerConfig(flags = {
+      "--rpc-encryption=required",
+      "--rpc_encrypt_loopback_connections",
+      "--rpc-trace-negotiation",
+  })
+  @TabletServerConfig(flags = {
+      "--rpc-encryption=required",
+      "--rpc_encrypt_loopback_connections",
+      "--rpc-trace-negotiation",
+  })
+  public void connectionNegotiation() throws Exception {
+    assumeTrue("TLSv1.3 isn't supported by both sides", isTLSv13Supported);
+
+    // Make sure Java client is able to communicate with Kudu masters and tablet
+    // servers: create a table and write several rows into the table.
+    {
+      KuduClient c = harness.getClient();
+      createDefaultTable(c, TABLE_NAME);
+      loadDefaultTable(c, TABLE_NAME, NUM_ROWS);
+    }
+
+    // An extra sanity check: on successful negotiation the connection should be
+    // considered 'private' once it's protected by TLS, so Kudu master must send
+    // the client an authn token.
+    {
+      AsyncKuduClient c = harness.getAsyncClient();
+      SecurityContext ctx = c.securityContext;
+      assertNotNull(ctx.getAuthenticationToken());
+    }
+  }
+}
diff --git a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
index f1ae46d..4d4dce6 100644
--- a/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
+++ b/java/kudu-client/src/test/java/org/apache/kudu/client/TestNegotiator.java
@@ -72,6 +72,15 @@ public class TestNegotiator {
 
   private static final char[] KEYSTORE_PASSWORD = "password".toCharArray();
 
+  // This test handles pre-TLSv1.3 protocols only. See TestNegotiationTLSv13
+  // for TLSv1.3-specific test scenarios.
+  // TODO(aserbin): update corresponding test scenarios to work with TLSv1.3
+  static final String[] ENABLED_PROTOCOLS = new String[]{
+      "TLSv1.2",
+      "TLSv1.1",
+      "TLSv1",
+  };
+
   /**
    * The cert stored in the keystore, in base64ed DER format.
    * The real certs we'll get from the server will not be in Base64,
@@ -136,7 +145,9 @@ public class TestNegotiator {
       kmf.init(loadTestKeystore(), KEYSTORE_PASSWORD);
       SSLContext ctx = SSLContext.getInstance("TLS");
       ctx.init(kmf.getKeyManagers(), null, null);
-      return ctx.createSSLEngine();
+      SSLEngine engine = ctx.createSSLEngine();
+      engine.setEnabledProtocols(ENABLED_PROTOCOLS);
+      return engine;
     } catch (Exception e) {
       throw new RuntimeException(e);
     }