You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/05/19 19:13:11 UTC

[GitHub] [kafka] nizhikov opened a new pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

nizhikov opened a new pull request #8695:
URL: https://github.com/apache/kafka/pull/8695


   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r428114569



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +622,34 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       Hello. Sorry, I don't understand your concern :)
   
   1. DEFAULT_SSL_ENABLED_PROTOCOLS = TLSv1.2,TLSv1.3 for java11+
   2. DEFAULT_SSL_ENABLED_PROTOCOLS = TLSv1.2 for others jdk.
   
   This property modified inside this test so I forcefully set it as default value.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433920350



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -117,24 +123,51 @@ public void testTlsDefaults() throws Exception {
             server.waitForMetric("response", 1);
         } else {
             NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
         }
     }
 
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     * @return {@code True} if client should be able to connect to the server.
+     */
+    private boolean isCompatible(List<String> serverProtocols, List<String> clientProtocols) {
+        return serverProtocols.contains(clientProtocols.get(0)) ||
+            (clientProtocols.get(0).equals("TLSv1.3") && clientProtocols.contains("TLSv1.2"));

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433339180



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);

Review comment:
       Moved.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431668846



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final String tlsServerProtocol;
+    private final String tlsClientProtocol;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        values.add(new Object[] {"TLSv1.2", "TLSv1.2"});
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {"TLSv1.2", "TLSv1.3"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.2"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.3"});
+        }
+        return values;
+    }
+
+    public SslVersionsTransportLayerTest(String tlsServerProtocol, String tlsClientProtocol) {
+        this.tlsServerProtocol = tlsServerProtocol;
+        this.tlsClientProtocol = tlsClientProtocol;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTLSDefaults() throws Exception {

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637647620


   @nizhikov I think we're good to merge this after the non code suggestions above are addressed (assuming we can get a Jenkins build, I merged your other PR fixing the build issue).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433253088



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -54,21 +54,16 @@
         List<Object[]> values = new ArrayList<>();
         values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.2")});
         if (Java.IS_JAVA11_COMPATIBLE) {
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+

Review comment:
       Why did we make these changes? I think what we had was good.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637774614


   @ijuma Thanks for your time! Appreciate it.
   Try my best to waste fewer rounds of review next time :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-636705289


   @ijuma 
   I ran the following tests:
   
   * tests/kafkatest/tests/tools/log4j_appender_test.py
   * tests/kafkatest/tests/core/upgrade_test.py
   * tests/kafkatest/tests/core/mirror_maker_test.py
   * tests/kafkatest/tests/core/consumer_group_command_test.py
   * tests/kafkatest/sanity_checks/test_console_consumer.py
   * tests/kafkatest/benchmarks/core/benchmark_test.py
   ```
   ====================================================================================================
   SESSION REPORT (ALL TESTS)
   ducktape version: 0.7.7
   session_id:       2020-05-29--003
   run time:         183 minutes 43.112 seconds
   tests run:        121
   passed:           120
   failed:           1
   ignored:          0
   ====================================================================================================
   ```
   
   Infor about single failure can be found in [ticket comments](https://issues.apache.org/jira/browse/KAFKA-9320?focusedCommentId=17120838&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17120838)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r432987394



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -354,15 +358,16 @@ def start_cmd(self, node):
 
     def start_node(self, node, timeout_sec=60):
         node.account.mkdirs(KafkaService.PERSISTENT_ROOT)
+
+        self.security_config.setup_node(node)
+        self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=True)

Review comment:
       Why do we need this change?

##########
File path: tests/kafkatest/services/security/security_config.py
##########
@@ -259,6 +267,9 @@ def setup_node(self, node):
         if self.has_sasl:
             self.setup_sasl(node)
 
+        if java_version(node) <= 9 and self.properties['tls.version'] == 'TLSv1.3':

Review comment:
       For consistency, shall we use `11` here?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       What is the reason for this?

##########
File path: tests/kafkatest/tests/core/replication_test.py
##########
@@ -126,9 +126,11 @@ def min_cluster_size(self):
             security_protocol="SASL_SSL", client_sasl_mechanism="SCRAM-SHA-256", interbroker_sasl_mechanism="SCRAM-SHA-512")
     @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
             security_protocol=["PLAINTEXT"], broker_type=["leader"], compression_type=["gzip"])
+    @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
+            security_protocol=["SSL"], broker_type=["leader"], compression_type=["gzip"], tls_version=["TLSv1.2", "TLSv1.3"])

Review comment:
       I think we can change one of the `SASL_SSL` entries to include multiple TLS versions. This way, we also verify that it works for SASL_SSL.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12_FailsForTls13() throws Exception {

Review comment:
       We normally don't have `_` in method names. Can we remove it from here and other test methods?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-632073022


   @nizhikov Thanks. Can you update the KIP and start the voting on it?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433270767



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can't be made if server uses TLSv1.2 with custom cipher suite and client uses TLSv1.3.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTls12ClientTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String tls12CipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+        String tls13CipherSuite = "TLS_AES_128_GCM_SHA256";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls12CipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls13CipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.3 cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTls13() throws Exception {

Review comment:
       Fixed.
   The cause was the line 
   `NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.READY);`
   changed to - `NetworkTestUtils.waitForChannelReady(selector, node);`
   
   We shouldn't wait channel to close.
   Now, locally test run less than second.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431857886



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431678155



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -352,17 +357,18 @@ def start_cmd(self, node):
                 KafkaService.STDOUT_STDERR_CAPTURE)
         return cmd
 
-    def start_node(self, node, timeout_sec=60):
+    def start_node(self, node, timeout_sec=180):

Review comment:
       No, sorry. Reverted.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433987384



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
##########
@@ -64,7 +66,17 @@
 
     public static final String SSL_ENABLED_PROTOCOLS_CONFIG = "ssl.enabled.protocols";
     public static final String SSL_ENABLED_PROTOCOLS_DOC = "The list of protocols enabled for SSL connections.";

Review comment:
       How about:
   
   ```
   The list of protocols enabled for SSL connections. The default is 'TLSv1.2,TLSv1.3' when running with Java 11 or newer, 'TLSv1.2' otherwise. With the default value for Java 11, clients and servers will prefer TLSv1.3 if both support it and fallback to TLSv1.2 otherwise (assuming both support at least TLSv1.2). This default should be fine for most cases.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-632308200


   Since the vote passed, can we flesh out the PR to include more tests that exercise TLS 1.3? A few things to think about:
   
   1. Unit tests like the ones included in the PR currently. Can we go through the various possible combinations of client and server configuration and check that they all work or fail in the way we expect.
   
   2. Make sure the integration tests use the same TLS configuration we use by default (if they don't already). Since Java 8 sticks to TLS 1.2 for now, we will get coverage of the old and new approach this way.
   
   3. Adjust system tests to use TLS 1.3 by default, but also include variants where client uses TLS 1.2 and broker uses 1.3, the reverse and finally where TLS 1.2 is used for both.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637777879


   @nizhikov These changes are pretty tricky to validate. Thanks for being thorough on the tests. :)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r428173640



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +622,34 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       Tests added.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433274613



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       Also, note that we also have `testUnsupportedCiphers`. I wonder if we can delete this test.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433351684



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);

Review comment:
       Done. See `SslTransportTls12Tls13Test`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov edited a comment on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov edited a comment on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-631328373


   @ijuma Looks like tests are OK.
   Please, take a look at this preliminary PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433990762



##########
File path: docs/upgrade.html
##########
@@ -18,6 +18,10 @@
 <script><!--#include virtual="js/templateData.js" --></script>
 
 <script id="upgrade-template" type="text/x-handlebars-template">
+<h5><a id="upgrade_270_notable" href="#upgrade_270_notable">Notable changes in 2.7.0</a></h5>
+<ul>
+    <li>TLSv1.3 have been enabled by default for JDK11+. See <a href="https://cwiki.apache.org/confluence/display/KAFKA/KIP-573%3A+Enable+TLSv1.3+by+default">KIP-573</a> for full details.</li>

Review comment:
       One more nit: "TLSv1.3 has been enabled by default for Java 11 or newer. The client and server will negotiate TLSv1.3 if both support it and fallback to TLSv1.2 otherwise. See...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433097896



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -354,15 +358,16 @@ def start_cmd(self, node):
 
     def start_node(self, node, timeout_sec=60):
         node.account.mkdirs(KafkaService.PERSISTENT_ROOT)
+
+        self.security_config.setup_node(node)
+        self.security_config.setup_credentials(node, self.path, self.zk_connect_setting(), broker=True)

Review comment:
       In the constructor of `SecurityConfig` we don't know node java version.
   Therefore we should fix `tls.version` after node version known.
   ```
       def setup_node(self, node):
   ...
           if java_version(node) <= 11 and self.properties['tls.version'] == 'TLSv1.3':
               self.properties.update({'tls.version': 'TLSv1.2'})
   
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433279649



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       Maybe we add `checkAuthentiationFailed` to `testUnsupportedCiphers` and delete this test.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);

Review comment:
       These two lines are unused.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r432632266



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +623,79 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433987384



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
##########
@@ -64,7 +66,17 @@
 
     public static final String SSL_ENABLED_PROTOCOLS_CONFIG = "ssl.enabled.protocols";
     public static final String SSL_ENABLED_PROTOCOLS_DOC = "The list of protocols enabled for SSL connections.";

Review comment:
       How about:
   
   ```
   The list of protocols enabled for SSL connections. The default is 'TLSv1.2,TLSv1.3' when running with Java 11 or newer, 'TLSv1.2' otherwise. With the default value for Java 11, clients and servers will prefer TLSv1.3 if both support it and fallback to TLSv1.2 otherwise (assuming both support at least TLSv1.2). This default should be fine for most cases. Also see the `ssl.protocol` config documentation.
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433283367



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r428020072



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +622,34 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       I think you don't want to leave this as the default and see if it works correctly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431675317



##########
File path: tests/kafkatest/benchmarks/core/benchmark_test.py
##########
@@ -236,9 +238,9 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol=
         return data
 
     @cluster(num_nodes=6)
-    @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"])
-    def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT",
+    @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT'], tls_version=['TLSv1.2', 'TLSv1.3'], compression_type=["none", "snappy"])
+    @matrix(security_protocol=['PLAINTEXT'], compression_type=["none", "snappy"])

Review comment:
       OK for me. Will revert other changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433283096



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can't be made if server uses TLSv1.2 with custom cipher suite and client uses TLSv1.3.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTls12ClientTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String tls12CipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+        String tls13CipherSuite = "TLS_AES_128_GCM_SHA256";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls12CipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls13CipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.3 cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTls13() throws Exception {

Review comment:
       Great!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433981892



##########
File path: clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java
##########
@@ -49,11 +50,12 @@
 
     public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol";
     public static final String SSL_PROTOCOL_DOC = "The SSL protocol used to generate the SSLContext. "
-            + "Default setting is TLSv1.2, which is fine for most cases. "
+            + "Default setting is TLSv1.2(TLSv1.3 for modern JVM), which is fine for most cases. "
             + "Allowed values in recent JVMs are TLSv1.2 and TLSv1.3. TLS, TLSv1.1, SSL, SSLv2 and SSLv3 "
-            + "may be supported in older JVMs, but their usage is discouraged due to known security vulnerabilities.";
+            + "may be supported in older JVMs, but their usage is discouraged due to known security vulnerabilities."
+            + "Please, note, TLSv1.2 clients can't connect to the servers with TLSv1.3 only even if ssl.enabled.protocols contains TLSv1.3";

Review comment:
       How about:
   
   ```java
   "The SSL protocol used to generate the SSLContext. "
               + "The default is TLSv1.3 when running with Java 11 or newer, TLSv1.2 otherwise. "
               + "This value should be fine for most use cases. "
               + "Allowed values in recent JVMs are TLSv1.2 and TLSv1.3. TLS, TLSv1.1, SSL, SSLv2 and SSLv3 "
               + "may be supported in older JVMs, but their usage is discouraged due to known security vulnerabilities. ";
               + "With the default value for this config and ssl.enabled.protocols, clients will downgrade to TLSv1.2 if "
               + "the server does not support TLSv1.3. If this config is set to TLSv1.2, clients will not use TLSv1.3 even "
               + "if it is one of the values in ssl.enabled.protocols and the server only supports TLSv1.3."
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637569220


   Oh, one more thing, let's please add an entry to `upgrade.html`.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma merged pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma merged pull request #8695:
URL: https://github.com/apache/kafka/pull/8695


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433352122



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433413250



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -732,11 +614,8 @@ public void testUnsupportedCiphers() throws Exception {
 
         sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[1]));
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
-        selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
-        server.verifyAuthenticationMetrics(0, 1);

Review comment:
       Shall we keep the authentication metrics check?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-631921191


   retest this, please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r428020072



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +622,34 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       I think you want to leave this as the default and see if it works correctly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433892269



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -117,24 +123,51 @@ public void testTlsDefaults() throws Exception {
             server.waitForMetric("response", 1);
         } else {
             NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
         }
     }
 
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     * @return {@code True} if client should be able to connect to the server.
+     */
+    private boolean isCompatible(List<String> serverProtocols, List<String> clientProtocols) {
+        return serverProtocols.contains(clientProtocols.get(0)) ||
+            (clientProtocols.get(0).equals("TLSv1.3") && clientProtocols.contains("TLSv1.2"));
+    }
+
     private static Map<String, Object> getTrustingConfig(CertStores certStores, CertStores peerCertStores, List<String> tlsProtocols) {
         Map<String, Object> configs = certStores.getTrustingConfig(peerCertStores);
         configs.putAll(sslConfig(tlsProtocols));
         return configs;
     }
 
-    private static Map<String, Object> sslConfig(List<String> tlsServerProtocols) {
+    private static Map<String, Object> sslConfig(List<String> tlsProtocols) {
         Map<String, Object> sslConfig = new HashMap<>();
-        sslConfig.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsServerProtocols.get(0));
-        sslConfig.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, tlsServerProtocols);
+        sslConfig.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocols.get(0));
+        sslConfig.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, tlsProtocols);
         return sslConfig;
     }
 
     private Selector createSelector(Map<String, Object> sslClientConfigs) {

Review comment:
       Should this be called `createClientSelector`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433271076



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       The previous logic is weird, I agree. I think the idea here is to simply pick a different supported cipher in the server vs the client. I think we can drop `getSupportedCipherSuites` altogether and just pick two ciphers explicitly. The cipher names would be different for TLS 1.2 versus TLS 1.3.
   
   Also, we should change the following to simply use the `tlsVersion` field.
   
   ```
               checkAuthentiationFailed("1", "TLSv1.1");
               server.verifyAuthenticationMetrics(0, 1);
               checkAuthentiationFailed("2", "TLSv1");
               server.verifyAuthenticationMetrics(0, 2);
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma edited a comment on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma edited a comment on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637842859


   The failures are unrelated:
   
   ```
   org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.StickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.StickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   ```
   
   Also failing in master:
   
   ```
   org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.StickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-631614084


   @ijuma I can't see downside in forcing usage of the latest TLS version.
   Added this change to PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433636236



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -732,11 +614,8 @@ public void testUnsupportedCiphers() throws Exception {
 
         sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[1]));
         createSelector(sslClientConfigs);
-        InetSocketAddress addr = new InetSocketAddress("localhost", server.port());
-        selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE);
 
-        NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
-        server.verifyAuthenticationMetrics(0, 1);

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433091643



##########
File path: tests/kafkatest/services/security/security_config.py
##########
@@ -259,6 +267,9 @@ def setup_node(self, node):
         if self.has_sasl:
             self.setup_sasl(node)
 
+        if java_version(node) <= 9 and self.properties['tls.version'] == 'TLSv1.3':

Review comment:
       OK. Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433102140



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -580,7 +581,16 @@ public void testTLSDefaults() throws Exception {
 
     @Test
     public void testUnsupportedCipher() throws Exception {
-        String[] cipherSuites = ((SSLServerSocketFactory) SSLServerSocketFactory.getDefault()).getSupportedCipherSuites();
+        String[] cipherSuites;
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            cipherSuites = new String[] {
+                "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384",

Review comment:
       We should use correct cipher for the server(which uses `TLSv1.3` in case Java11) otherwise server metrics not updated because we never get `AuthenticationException` on the server-side. Instead of it, we get `IOException` from `SSLTransportLayer`:
   
   ```
       private SSLEngineResult handshakeUnwrap(boolean doRead, boolean ignoreHandshakeStatus) throws IOException {
   ...
           // Throw EOF exception for failed read after processing already received data
           // so that handshake failures are reported correctly
           if (read == -1)
               throw new EOFException("EOF during handshake, handshake status is " + handshakeStatus);
   }
   ```
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-631601762


   One question: any downside to setting `ssl.protocol=TLSv1.3` instead of `ssl.protocol=TLSv1.2`?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-636536255


   @nizhikov Have you had a chance to run the system tests with Java 11?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-631328373


   @ijuma Looks like tests is OK.
   Please, take a look at this preliminary PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-633687352


   A few failures seem related to the changes in this PR:
   
   > kafka.network.SocketServerTest.testConnectionIdReuse
   > kafka.network.SocketServerTest.remoteCloseWithBufferedReceivesFailedSend
   > kafka.network.SocketServerTest.remoteCloseSendFailure
   > kafka.network.SocketServerTest.remoteCloseWithoutBufferedReceives
   > kafka.network.SocketServerTest.remoteCloseWithCompleteAndIncompleteBufferedReceives
   > kafka.network.SocketServerTest.remoteCloseWithIncompleteBufferedReceive
   > kafka.network.SocketServerTest.closingChannelWithBufferedReceives
   > kafka.network.SocketServerTest.closingChannelSendFailure
   > kafka.network.SocketServerTest.idleExpiryWithBufferedReceives
   > kafka.network.SocketServerTest.closingChannelWithBufferedReceivesFailedSend
   > kafka.network.SocketServerTest.remoteCloseWithBufferedReceives
   > kafka.network.SocketServerTest.closingChannelWithCompleteAndIncompleteBufferedReceives


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r432987691



##########
File path: tests/kafkatest/tests/core/replication_test.py
##########
@@ -126,9 +126,11 @@ def min_cluster_size(self):
             security_protocol="SASL_SSL", client_sasl_mechanism="SCRAM-SHA-256", interbroker_sasl_mechanism="SCRAM-SHA-512")
     @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
             security_protocol=["PLAINTEXT"], broker_type=["leader"], compression_type=["gzip"])
+    @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
+            security_protocol=["SSL"], broker_type=["leader"], compression_type=["gzip"], tls_version=["TLSv1.2", "TLSv1.3"])

Review comment:
       I think we can change one of the `SASL_SSL` entries to include multiple TLS versions. This way, we also verify that it works for SASL_SSL without increasing test time too much.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433270767



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can't be made if server uses TLSv1.2 with custom cipher suite and client uses TLSv1.3.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTls12ClientTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String tls12CipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+        String tls13CipherSuite = "TLS_AES_128_GCM_SHA256";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls12CipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls13CipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.3 cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTls13() throws Exception {

Review comment:
       Fixed.
   The cause was the line 
   `NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.READY);`
   
   We shouldn't wait channel to close.
   Now, locally test run less than second.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433283509



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -54,21 +54,16 @@
         List<Object[]> values = new ArrayList<>();
         values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.2")});
         if (Java.IS_JAVA11_COMPATIBLE) {
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+

Review comment:
       Interesting! Good that we added that test. :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433903099



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final List<String> serverProtocols;
+    private final List<String> clientProtocols;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+
+        values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.2")});
+
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+        }
+        return values;
+    }
+
+    /**
+     * Be aware that you can turn on debug mode for a javax.net.ssl library with the line {@code System.setProperty("javax.net.debug", "ssl:handshake");}
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     */
+    public SslVersionsTransportLayerTest(List<String> serverProtocols, List<String> clientProtocols) {
+        this.serverProtocols = serverProtocols;
+        this.clientProtocols = clientProtocols;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTlsDefaults() throws Exception {
+        // Create certificates for use by client and server. Add server cert to client truststore and vice versa.
+        CertStores serverCertStores = new CertStores(true, "server",  "localhost");
+        CertStores clientCertStores = new CertStores(false, "client", "localhost");
+
+        Map<String, Object> sslClientConfigs = getTrustingConfig(clientCertStores, serverCertStores, clientProtocols);
+        Map<String, Object> sslServerConfigs = getTrustingConfig(serverCertStores, clientCertStores, serverProtocols);
+
+        NioEchoServer server = NetworkTestUtils.createEchoServer(ListenerName.forSecurityProtocol(SecurityProtocol.SSL),
+            SecurityProtocol.SSL,
+            new TestSecurityConfig(sslServerConfigs),
+            null,
+            TIME);
+        Selector selector = createSelector(sslClientConfigs);
+
+        String node = "0";
+        selector.connect(node, new InetSocketAddress("localhost", server.port()), BUFFER_SIZE, BUFFER_SIZE);
+
+        if (isCompatible(serverProtocols, clientProtocols)) {
+            NetworkTestUtils.waitForChannelReady(selector, node);
+
+            int msgSz = 1024 * 1024;
+            String message = TestUtils.randomString(msgSz);
+            selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
+            while (selector.completedReceives().isEmpty()) {
+                selector.poll(100L);
+            }
+            int totalBytes = msgSz + 4; // including 4-byte size
+            server.waitForMetric("incoming-byte", totalBytes);
+            server.waitForMetric("outgoing-byte", totalBytes);
+            server.waitForMetric("request", 1);
+            server.waitForMetric("response", 1);
+        } else {
+            NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
+        }
+    }
+
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.

Review comment:
       Maybe we should document that `serverProtocols` and `clientProtocols` are expected to be non empty as the code below assumes it.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final List<String> serverProtocols;
+    private final List<String> clientProtocols;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+
+        values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.2")});
+
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+        }
+        return values;
+    }
+
+    /**
+     * Be aware that you can turn on debug mode for a javax.net.ssl library with the line {@code System.setProperty("javax.net.debug", "ssl:handshake");}
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     */
+    public SslVersionsTransportLayerTest(List<String> serverProtocols, List<String> clientProtocols) {
+        this.serverProtocols = serverProtocols;
+        this.clientProtocols = clientProtocols;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTlsDefaults() throws Exception {
+        // Create certificates for use by client and server. Add server cert to client truststore and vice versa.
+        CertStores serverCertStores = new CertStores(true, "server",  "localhost");
+        CertStores clientCertStores = new CertStores(false, "client", "localhost");
+
+        Map<String, Object> sslClientConfigs = getTrustingConfig(clientCertStores, serverCertStores, clientProtocols);
+        Map<String, Object> sslServerConfigs = getTrustingConfig(serverCertStores, clientCertStores, serverProtocols);
+
+        NioEchoServer server = NetworkTestUtils.createEchoServer(ListenerName.forSecurityProtocol(SecurityProtocol.SSL),
+            SecurityProtocol.SSL,
+            new TestSecurityConfig(sslServerConfigs),
+            null,
+            TIME);
+        Selector selector = createSelector(sslClientConfigs);
+
+        String node = "0";
+        selector.connect(node, new InetSocketAddress("localhost", server.port()), BUFFER_SIZE, BUFFER_SIZE);
+
+        if (isCompatible(serverProtocols, clientProtocols)) {
+            NetworkTestUtils.waitForChannelReady(selector, node);
+
+            int msgSz = 1024 * 1024;
+            String message = TestUtils.randomString(msgSz);
+            selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
+            while (selector.completedReceives().isEmpty()) {
+                selector.poll(100L);
+            }
+            int totalBytes = msgSz + 4; // including 4-byte size
+            server.waitForMetric("incoming-byte", totalBytes);
+            server.waitForMetric("outgoing-byte", totalBytes);
+            server.waitForMetric("request", 1);
+            server.waitForMetric("response", 1);
+        } else {
+            NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
+        }
+    }
+
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     * @return {@code True} if client should be able to connect to the server.

Review comment:
       Nit: `True` should be lower case.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -117,24 +123,51 @@ public void testTlsDefaults() throws Exception {
             server.waitForMetric("response", 1);
         } else {
             NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
         }
     }
 
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     * @return {@code True} if client should be able to connect to the server.
+     */
+    private boolean isCompatible(List<String> serverProtocols, List<String> clientProtocols) {
+        return serverProtocols.contains(clientProtocols.get(0)) ||
+            (clientProtocols.get(0).equals("TLSv1.3") && clientProtocols.contains("TLSv1.2"));

Review comment:
       Would the second check if clearer if it was like:
   
   `clientProtocols.get(0).equals("TLSv1.3") && !Collections.disjoint(tlsServerProtocols, tlsClientProtocols)`
   
   We're basically saying that we check all the client protocols against the broker protocols if the client uses TLS 1.3. The first condition checks the case where the first client protocol is supported by the server (TLS 1.2).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433279701



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);

Review comment:
       These two lines are unused. Similar for other tests.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433272524



##########
File path: tests/kafkatest/tests/core/replication_test.py
##########
@@ -126,9 +126,11 @@ def min_cluster_size(self):
             security_protocol="SASL_SSL", client_sasl_mechanism="SCRAM-SHA-256", interbroker_sasl_mechanism="SCRAM-SHA-512")
     @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
             security_protocol=["PLAINTEXT"], broker_type=["leader"], compression_type=["gzip"])
+    @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"],
+            security_protocol=["SSL"], broker_type=["leader"], compression_type=["gzip"], tls_version=["TLSv1.2", "TLSv1.3"])

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433265356



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -54,21 +54,16 @@
         List<Object[]> values = new ArrayList<>();
         values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.2")});
         if (Java.IS_JAVA11_COMPATIBLE) {
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
-            values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Collections.singletonList("TLSv1.2")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
-            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Collections.singletonList("TLSv1.2")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
             values.add(new Object[] {Arrays.asList("TLSv1.3", "TLSv1.2"), Arrays.asList("TLSv1.3", "TLSv1.2")});
+

Review comment:
       Actually, test fails for a single case:
   For now, I have no idea why this happen :)
   I need some time to dig into details and explain results.
   Anyway, I revert the test with all possible combinations(that will fail).
   
   ```
   values.add(new Object[] {Collections.singletonList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
   ```
   This means:
   ```
   #server config
   ssl.protocol=TLSv1.3
   ssl.enabled.protocols=TLSv1.3
   
   #client config
   ssl.protocol=TLSv1.2
   ssl.enabled.protocols=TLSv1.2,TLSv1.3
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637842859


   The failures are unrelated:
   
   ```
   org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.CooperativeStickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.StickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   org.apache.kafka.clients.consumer.StickyAssignorTest.testReassignmentWithRandomSubscriptionsAndChanges
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433899761



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -117,24 +123,51 @@ public void testTlsDefaults() throws Exception {
             server.waitForMetric("response", 1);
         } else {
             NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+            server.verifyAuthenticationMetrics(0, 1);
         }
     }
 
+    /**
+     * <p>
+     * The explanation of this check in the structure of the ClientHello SSL message.
+     * Please, take a look at the <a href="https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6">Guide</a>,
+     * "Send ClientHello Message" section.
+     * <p>
+     * > Client version: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
+     * ...
+     * > supported_versions: Lists which versions of TLS the client supports. In particular, if the client
+     * > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension
+     * > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
+     * > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
+     * <p>
+     *
+     * This mean that TLSv1.3 client can fallback to TLSv1.2 but TLSv1.2 client can't change protocol to TLSv1.3.
+     *
+     * @param serverProtocols Server protocols.
+     * @param clientProtocols Client protocols.
+     * @return {@code True} if client should be able to connect to the server.
+     */
+    private boolean isCompatible(List<String> serverProtocols, List<String> clientProtocols) {
+        return serverProtocols.contains(clientProtocols.get(0)) ||
+            (clientProtocols.get(0).equals("TLSv1.3") && clientProtocols.contains("TLSv1.2"));
+    }
+
     private static Map<String, Object> getTrustingConfig(CertStores certStores, CertStores peerCertStores, List<String> tlsProtocols) {
         Map<String, Object> configs = certStores.getTrustingConfig(peerCertStores);
         configs.putAll(sslConfig(tlsProtocols));
         return configs;
     }
 
-    private static Map<String, Object> sslConfig(List<String> tlsServerProtocols) {
+    private static Map<String, Object> sslConfig(List<String> tlsProtocols) {
         Map<String, Object> sslConfig = new HashMap<>();
-        sslConfig.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsServerProtocols.get(0));
-        sslConfig.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, tlsServerProtocols);
+        sslConfig.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocols.get(0));
+        sslConfig.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, tlsProtocols);
         return sslConfig;
     }
 
     private Selector createSelector(Map<String, Object> sslClientConfigs) {

Review comment:
       Done.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431674950



##########
File path: tests/docker/run_tests.sh
##########
@@ -30,6 +30,6 @@ if [ "$REBUILD" == "t" ]; then
 fi
 
 if ${SCRIPT_DIR}/ducker-ak ssh | grep -q '(none)'; then
-    ${SCRIPT_DIR}/ducker-ak up -n "${KAFKA_NUM_CONTAINERS}" || die "ducker-ak up failed"
+    ${SCRIPT_DIR}/ducker-ak up -j 'openjdk:11' -n "${KAFKA_NUM_CONTAINERS}" || die "ducker-ak up failed"

Review comment:
       I think this should be reverted prior merge. 
   Changed only to test PR on external server(my local machine too slow to run all changed tests)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-634055561


   retest this, please
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637515040


   @ijuma I found explanation of the test behavior.
   
   Full information can be found in the [guide](https://docs.oracle.com/en/java/javase/11/security/java-secure-socket-extension-jsse-reference-guide.html#GUID-4D421910-C36D-40A2-8BA2-7D42CCBED3C6) Please, navigate to the "Send ClientHello Message". You may want to take a look at the "client version" and "supported_versions (43)" fields.
   
   The root of the "strange" behavior is the structure of the SSL ClientHello message(quote from tutorial):
   > **Client version**: For TLS 1.3, this has a fixed value, TLSv1.2; TLS 1.3 uses the extension supported_versions and not this field to negotiate protocol version
   > ...
   > **supported_versions**: Lists which versions of TLS the client supports. In particular, if the client 
   > requests TLS 1.3, then the client version field has the value TLSv1.2 and this extension 
   > contains the value TLSv1.3; if the client requests TLS 1.2, then the client version field has the
   > value TLSv1.2 and this extension either doesn’t exist or contains the value TLSv1.2 but not the value TLSv1.3.
   
   This means we can't connect with the following configuration:
   client: 
   ```
   ssl.protocol=TLSv1.2 #this will be used for ClientHello
   ssl.enabled.protocols=TLSv1.2,TLSv1.3 #TLS v1.3 will be ignored in ClientHello message.
   ```
   Server:
   ```
   ssl.protocol=TLSv1.3
   ssl.enabled.protocols=TLSv1.3 # Accept only TLSv1.3 
   ```
   
   You can see all details of the SSL connection process in the javax.net log. 
   It can be enabled like the following:
   ```
       public SslVersionsTransportLayerTest(List<String> serverProtocols, List<String> clientProtocols) {
           System.setProperty("javax.net.debug", "ssl:handshake"); //This will turn on the log from jdk SSL system classes.
           this.serverProtocols = serverProtocols;
           this.clientProtocols = clientProtocols;
       }
   ```
   
   So correct check should be:
   
   ```
       private boolean isCompatible(List<String> serverProtocols, List<String> clientProtocols) {
           return serverProtocols.contains(clientProtocols.get(0)) ||
               (clientProtocols.get(0).equals("TLSv1.3") && clientProtocols.contains("TLSv1.2"));
       }
   ```
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433266555



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can't be made if server uses TLSv1.2 with custom cipher suite and client uses TLSv1.3.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTls12ClientTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String tls12CipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+        String tls13CipherSuite = "TLS_AES_128_GCM_SHA256";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls12CipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls13CipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.3 cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTls13() throws Exception {

Review comment:
       Hmm, could it be related to that fact that we don't call `getTrustingConfig` like it's done in `testUnsupportedCipher`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637573626


   retest this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-635989928


   @ijuma
   
   I fixed java test failures.
   It seems I resolved all your comments.
   I need some time to check system tests.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433260716



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can't be made if server uses TLSv1.2 with custom cipher suite and client uses TLSv1.3.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTls12ClientTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        String tls12CipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+        String tls13CipherSuite = "TLS_AES_128_GCM_SHA256";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls12CipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Collections.singletonList(tls13CipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.3 cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTls13() throws Exception {

Review comment:
       Hmm, these tests seem to take a long time. Do you know why? See:
   
   ![image](https://user-images.githubusercontent.com/24747/83418057-d11f3180-a3d7-11ea-873d-9f6aedf4e567.png)
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-632087277


   @ijuma [KIP-573](https://cwiki.apache.org/confluence/display/KAFKA/KIP-573%3A+Enable+TLSv1.3+by+default) updated.
   Actually, I'm started one ([link](https://mail-archives.apache.org/mod_mbox/kafka-dev/202003.mbox/%3CC741B223-739D-4FDE-B8F9-63DD4ACC433F%40gmail.com%3E)), but didn't get any votes :)
   Should I start another?
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#issuecomment-637574517


   I think, currently, the trunk is broken with the c6633a157eec1712116d294eb3785a96cba4e331
   I prepared oneliner fix for it - #8779 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r428153104



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +622,34 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       Ah, I see, you are forcefully setting it to the default. Makes sense. OK, so this test shows that we can negotiate successfully even though we have no cipher suites that work with TLS 1.3. Can we also test that if the client sets TLS 1.3, it will fail?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431677154



##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -52,6 +51,7 @@ def advertised_listener(self, node):
     def listener_security_protocol(self):
         return "%s:%s" % (self.name, self.security_protocol)
 
+

Review comment:
       This change not necessary.
   But PyCharm and IDEA print warning without it: "PEP 8: E302 expected 2 blank lines, found 1"
   
   I will revert it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: KIP-573 - Enable TLSv1.3 by default

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r431587702



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;

Review comment:
       We can use `assume` to do this in a more idiomatic way. Same for other places where we do something similar.

##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -352,17 +357,18 @@ def start_cmd(self, node):
                 KafkaService.STDOUT_STDERR_CAPTURE)
         return cmd
 
-    def start_node(self, node, timeout_sec=60):
+    def start_node(self, node, timeout_sec=180):

Review comment:
       Why do we need this change?

##########
File path: tests/kafkatest/services/kafka/kafka.py
##########
@@ -52,6 +51,7 @@ def advertised_listener(self, node):
     def listener_security_protocol(self):
         return "%s:%s" % (self.name, self.security_protocol)
 
+

Review comment:
       Is this intentional?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");

Review comment:
       Can we set this to `DEFAULT_SSL_PROTOCOL`?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.

Review comment:
       The important detail is that TLS 1.3 is enabled too, right?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final String tlsServerProtocol;
+    private final String tlsClientProtocol;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        values.add(new Object[] {"TLSv1.2", "TLSv1.2"});
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {"TLSv1.2", "TLSv1.3"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.2"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.3"});
+        }
+        return values;
+    }
+
+    public SslVersionsTransportLayerTest(String tlsServerProtocol, String tlsClientProtocol) {
+        this.tlsServerProtocol = tlsServerProtocol;
+        this.tlsClientProtocol = tlsClientProtocol;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTLSDefaults() throws Exception {

Review comment:
       Nit: the test should be `testTlsDefaults` to match the naming convention.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -591,10 +591,7 @@ public void testUnsupportedCipher() throws Exception {
             createSelector(sslClientConfigs);
 
             checkAuthentiationFailed("1", "TLSv1.1");
-            server.verifyAuthenticationMetrics(0, 1);
-
             checkAuthentiationFailed("2", "TLSv1");
-            server.verifyAuthenticationMetrics(0, 2);

Review comment:
       Why did we remove both lines above?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -1250,7 +1322,7 @@ private NioEchoServer createEchoServer(SecurityProtocol securityProtocol) throws
         void run() throws IOException;
     }
 
-    private static class TestSslChannelBuilder extends SslChannelBuilder {
+    public static class TestSslChannelBuilder extends SslChannelBuilder {

Review comment:
       Do these have to be public or can they be package private?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");

Review comment:
       Same here, can we `SslConfigs.DEFAULT_SSL_PROTOCOL`?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.

Review comment:
       This comment seems incorrect.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {
+        String node = "0";
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(",")));

Review comment:
       Can we replace
   
   `Arrays.asList(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS.split(","))`
   
   with
   
   `SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS`
   
   Doesn't the code handle comma separated Strings?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {

Review comment:
       If we take the suggestions to use `DEFAULT_SSL_PROTOCOL`, then this test is checking that we can negotiate TLSv1.2 if the custom cipher suites are only compatible with TLSv1.2 and we use the default protocol configuration (TLSv1.3).

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final String tlsServerProtocol;
+    private final String tlsClientProtocol;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        values.add(new Object[] {"TLSv1.2", "TLSv1.2"});
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {"TLSv1.2", "TLSv1.3"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.2"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.3"});
+        }
+        return values;
+    }
+
+    public SslVersionsTransportLayerTest(String tlsServerProtocol, String tlsClientProtocol) {

Review comment:
       Could we have each parameter be a list so that we can set enabled protocols with more than one element?

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final String tlsServerProtocol;
+    private final String tlsClientProtocol;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        values.add(new Object[] {"TLSv1.2", "TLSv1.2"});
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {"TLSv1.2", "TLSv1.3"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.2"});
+            values.add(new Object[] {"TLSv1.3", "TLSv1.3"});
+        }
+        return values;
+    }
+
+    public SslVersionsTransportLayerTest(String tlsServerProtocol, String tlsClientProtocol) {
+        this.tlsServerProtocol = tlsServerProtocol;
+        this.tlsClientProtocol = tlsClientProtocol;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTLSDefaults() throws Exception {
+        // Create certificates for use by client and server. Add server cert to client truststore and vice versa.
+        CertStores serverCertStores = new CertStores(true, "server",  "localhost");
+        CertStores clientCertStores = new CertStores(false, "client", "localhost");
+
+        Map<String, Object> sslClientConfigs = getTrustingConfig(clientCertStores, serverCertStores, tlsClientProtocol);
+        Map<String, Object> sslServerConfigs = getTrustingConfig(serverCertStores, clientCertStores, tlsServerProtocol);
+
+        NioEchoServer server = NetworkTestUtils.createEchoServer(ListenerName.forSecurityProtocol(SecurityProtocol.SSL),
+             SecurityProtocol.SSL,
+             new TestSecurityConfig(sslServerConfigs),
+             null,
+            TIME);
+        Selector selector = createSelector(sslClientConfigs);
+
+        String node = "0";
+        selector.connect(node, new InetSocketAddress("localhost", server.port()), BUFFER_SIZE, BUFFER_SIZE);
+
+        if (tlsServerProtocol.equals(tlsClientProtocol)) {
+            NetworkTestUtils.waitForChannelReady(selector, node);
+
+            int msgSz = 1024 * 1024;
+            String message = TestUtils.randomString(msgSz);
+            selector.send(new NetworkSend(node, ByteBuffer.wrap(message.getBytes())));
+            while (selector.completedReceives().isEmpty()) {
+                selector.poll(100L);
+            }
+            int totalBytes = msgSz + 4; // including 4-byte size
+            server.waitForMetric("incoming-byte", totalBytes);
+            server.waitForMetric("outgoing-byte", totalBytes);
+            server.waitForMetric("request", 1);
+            server.waitForMetric("response", 1);
+        } else {
+            NetworkTestUtils.waitForChannelClose(selector, node, ChannelState.State.AUTHENTICATION_FAILED);
+        }
+    }
+
+    public static Map<String, Object> sslConfig(String tlsServerProtocol) {
+        Map<String, Object> sslConfig = new HashMap<>();
+
+        sslConfig.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsServerProtocol);
+        sslConfig.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList(tlsServerProtocol));
+

Review comment:
       We probably don't need these empty lines.

##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +619,81 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2_FailsForTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);
+        context.init(null, null, null);
+
+        //Note, that only some ciphers works out of the box. Others requires additional configuration.
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.3"));
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+        server.verifyAuthenticationMetrics(0, 1);
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteFailForServerTLSv1_2_ClientTLSv1_3() throws Exception {
+        if (!Java.IS_JAVA11_COMPATIBLE)
+            return;
+
+        String cipherSuite = "TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384";
+
+        sslServerConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2");
+        sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2"));
+        sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+        server = createEchoServer(SecurityProtocol.SSL);
+
+        sslClientConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.3");
+        sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuite));
+
+        checkAuthentiationFailed("0", "TLSv1.3");
+    }
+
+    /**
+     * Tests that connections can be made with TLSv1.2 and custom cipher suite.
+     */
+    @Test
+    public void testCiphersSuiteForTLSv1_2() throws Exception {

Review comment:
       We should also add a similar test where the custom cipher is a TLS 1.3 cipher.

##########
File path: tests/docker/run_tests.sh
##########
@@ -30,6 +30,6 @@ if [ "$REBUILD" == "t" ]; then
 fi
 
 if ${SCRIPT_DIR}/ducker-ak ssh | grep -q '(none)'; then
-    ${SCRIPT_DIR}/ducker-ak up -n "${KAFKA_NUM_CONTAINERS}" || die "ducker-ak up failed"
+    ${SCRIPT_DIR}/ducker-ak up -j 'openjdk:11' -n "${KAFKA_NUM_CONTAINERS}" || die "ducker-ak up failed"

Review comment:
       Do we intend to change this temporarily or permanently?

##########
File path: tests/kafkatest/benchmarks/core/benchmark_test.py
##########
@@ -236,9 +238,9 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol=
         return data
 
     @cluster(num_nodes=6)
-    @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
-    @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"])
-    def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT",
+    @matrix(security_protocol=['SSL'], interbroker_security_protocol=['PLAINTEXT'], tls_version=['TLSv1.2', 'TLSv1.3'], compression_type=["none", "snappy"])
+    @matrix(security_protocol=['PLAINTEXT'], compression_type=["none", "snappy"])

Review comment:
       Maybe we can configure the TLS version for `benchmark_test.py` and `replication_test.py` only. The rest can use the default, which will be TLS 1.3 for Java 11 and TLS 1.2 for Java 8. That would not inflate test times by too much. What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ijuma commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
ijuma commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r433282708



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java
##########
@@ -622,6 +632,108 @@ public void testUnsupportedTLSVersion() throws Exception {
         server.verifyAuthenticationMetrics(0, 1);
     }
 
+    /**
+     * Tests that connections fails if TLSv1.3 enabled but cipher suite suitable only for TLSv1.2 used.
+     */
+    @Test
+    public void testCiphersSuiteForTls12FailsForTls13() throws Exception {
+        assumeTrue(Java.IS_JAVA11_COMPATIBLE);
+
+        SSLContext context = SSLContext.getInstance(tlsProtocol);

Review comment:
       I had not noticed that this test is parameterized on `tlsProtocol`. Maybe we should move these tests to a separate class? It's a bit weird to run them twice and ignore the parameterized value.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] nizhikov commented on a change in pull request #8695: KAFKA-9320: Enable TLSv1.3 by default (KIP-573)

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8695:
URL: https://github.com/apache/kafka/pull/8695#discussion_r432632145



##########
File path: clients/src/test/java/org/apache/kafka/common/network/SslVersionsTransportLayerTest.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.kafka.common.network;
+
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.security.TestSecurityConfig;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Java;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Tests for the SSL transport layer.
+ * Checks different versions of the protocol usage on the server and client.
+ */
+@RunWith(value = Parameterized.class)
+public class SslVersionsTransportLayerTest {
+    private static final int BUFFER_SIZE = 4 * 1024;
+    private static final Time TIME = Time.SYSTEM;
+
+    private final List<String> tlsServerProtocols;
+    private final List<String> tlsClientProtocols;
+
+    @Parameterized.Parameters(name = "tlsServerProtocol={0},tlsClientProtocol={1}")
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<>();
+        values.add(new Object[] {Arrays.asList("TLSv1.2"), Arrays.asList("TLSv1.2")});
+        if (Java.IS_JAVA11_COMPATIBLE) {
+            values.add(new Object[] {Arrays.asList("TLSv1.2"), Arrays.asList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3"), Arrays.asList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3"), Arrays.asList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2")});
+            values.add(new Object[] {Arrays.asList("TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+            values.add(new Object[] {Arrays.asList("TLSv1.2", "TLSv1.3"), Arrays.asList("TLSv1.2", "TLSv1.3")});
+        }
+        return values;
+    }
+
+    public SslVersionsTransportLayerTest(List<String> tlsServerProtocols, List<String> tlsClientProtocols) {
+        this.tlsServerProtocols = tlsServerProtocols;
+        this.tlsClientProtocols = tlsClientProtocols;
+    }
+
+    /**
+     * Tests that connection success with the default TLS version.
+     */
+    @Test
+    public void testTlsDefaults() throws Exception {
+        // Create certificates for use by client and server. Add server cert to client truststore and vice versa.
+        CertStores serverCertStores = new CertStores(true, "server",  "localhost");
+        CertStores clientCertStores = new CertStores(false, "client", "localhost");
+
+        Map<String, Object> sslClientConfigs = getTrustingConfig(clientCertStores, serverCertStores, tlsClientProtocols);
+        Map<String, Object> sslServerConfigs = getTrustingConfig(serverCertStores, clientCertStores, tlsServerProtocols);
+
+        NioEchoServer server = NetworkTestUtils.createEchoServer(ListenerName.forSecurityProtocol(SecurityProtocol.SSL),
+             SecurityProtocol.SSL,
+             new TestSecurityConfig(sslServerConfigs),
+             null,
+            TIME);
+        Selector selector = createSelector(sslClientConfigs);
+
+        String node = "0";
+        selector.connect(node, new InetSocketAddress("localhost", server.port()), BUFFER_SIZE, BUFFER_SIZE);
+
+        if (tlsServerProtocols.contains(tlsClientProtocols.get(0))) {

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org