From 4c92e689089d33618c59261922db13be7bd9a114 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 28 Apr 2015 00:29:53 -0700 Subject: [PATCH 1/6] KAFKA-1690. new java producer needs ssl support as a client. --- checkstyle/checkstyle.xml | 1 - checkstyle/import-control.xml | 14 +- .../java/org/apache/kafka/clients/ClientUtils.java | 14 + .../apache/kafka/clients/CommonClientConfigs.java | 2 + .../kafka/clients/consumer/ConsumerConfig.java | 11 +- .../kafka/clients/consumer/KafkaConsumer.java | 5 +- .../kafka/clients/producer/KafkaProducer.java | 5 +- .../kafka/clients/producer/ProducerConfig.java | 8 +- .../apache/kafka/common/config/SecurityConfig.java | 113 ++++++ .../apache/kafka/common/network/Authenticator.java | 60 +++ .../org/apache/kafka/common/network/Channel.java | 133 +++++++ .../kafka/common/network/DefaultAuthenticator.java | 46 +++ .../common/network/PlainTextTransportLayer.java | 148 +++++++ .../apache/kafka/common/network/SSLFactory.java | 188 +++++++++ .../kafka/common/network/SSLTransportLayer.java | 430 +++++++++++++++++++++ .../apache/kafka/common/network/Selectable.java | 1 + .../org/apache/kafka/common/network/Selector.java | 129 +++++-- .../kafka/common/network/TransportLayer.java | 86 +++++ .../kafka/common/protocol/SecurityProtocol.java | 2 + .../java/org/apache/kafka/common/utils/Utils.java | 11 + .../apache/kafka/common/network/EchoServer.java | 108 ++++++ .../kafka/common/network/SSLSelectorTest.java | 97 +++++ .../apache/kafka/common/network/SelectorTest.java | 79 +--- .../org/apache/kafka/common/utils/UtilsTest.java | 2 + .../java/org/apache/kafka/test/TestSSLUtils.java | 208 ++++++++++ 25 files changed, 1780 insertions(+), 121 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/Authenticator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/Channel.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java create mode 100644 clients/src/test/java/org/apache/kafka/common/network/EchoServer.java create mode 100644 clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java create mode 100644 clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index a215ff3..5fbf562 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -33,7 +33,6 @@ - diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index f2e6cec..e649189 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -26,6 +26,9 @@ + + + @@ -33,14 +36,18 @@ - + + + + + @@ -51,6 +58,9 @@ + + + @@ -73,6 +83,7 @@ + @@ -80,6 +91,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 0d68bf1..54a554f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -13,17 +13,22 @@ package org.apache.kafka.clients; import java.io.Closeable; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; +import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.SecurityConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; +import static org.apache.kafka.common.utils.Utils.loadProps; + public class ClientUtils { private static final Logger log = LoggerFactory.getLogger(ClientUtils.class); @@ -61,4 +66,13 @@ public class ClientUtils { } } } + + public static SecurityConfig parseSecurityConfig(String securityConfigFile) throws IOException { + Properties securityProps = new Properties(); + if (securityConfigFile == null || securityConfigFile == "") { + return new SecurityConfig(securityProps); + } + securityProps = loadProps(securityConfigFile); + return new SecurityConfig(securityProps); + } } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index cf32e4e..0b23875 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -55,4 +55,6 @@ public class CommonClientConfigs { public static final String METRIC_REPORTER_CLASSES_CONFIG = "metric.reporters"; public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the MetricReporter interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; + public static final String SECURITY_CONFIG_FILE_CONFIG = "security.config.file"; + public static final String SECURITY_CONFIG_FILE_DOC = "Kafka client security related config file."; } \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index bdff518..190fe63 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -156,6 +156,10 @@ public class ConsumerConfig extends AbstractConfig { public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + /** security.config.file */ + public static final String SECURITY_CONFIG_FILE_CONFIG = CommonClientConfigs.SECURITY_CONFIG_FILE_CONFIG; + private static final String SECURITY_CONFIG_FILE_DOC = CommonClientConfigs.SECURITY_CONFIG_FILE_DOC; + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, @@ -277,7 +281,12 @@ public class ConsumerConfig extends AbstractConfig { .define(VALUE_DESERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, - VALUE_DESERIALIZER_CLASS_DOC); + VALUE_DESERIALIZER_CLASS_DOC) + .define(SECURITY_CONFIG_FILE_CONFIG, + Type.STRING, + "", + Importance.MEDIUM, + SECURITY_CONFIG_FILE_DOC); } public static Map addDeserializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d301be4..032fd4b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -37,6 +37,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.MetricName; @@ -361,6 +362,7 @@ public class KafkaConsumer implements Consumer { private final boolean autoCommit; private final long autoCommitIntervalMs; private final ConsumerRebalanceCallback rebalanceCallback; + private final SecurityConfig securityConfig; private long lastCommitAttemptMs; private boolean closed = false; @@ -472,7 +474,8 @@ public class KafkaConsumer implements Consumer { String metricGrpPrefix = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags), + this.securityConfig = ClientUtils.parseSecurityConfig(config.getString(ConsumerConfig.SECURITY_CONFIG_FILE_CONFIG)); + this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags, securityConfig), this.metadata, clientId, 100, // a fixed large enough value will suffice diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 8e336a3..2c21dee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -32,6 +32,7 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.RecordTooLargeException; @@ -138,6 +139,7 @@ public class KafkaProducer implements Producer { private final Serializer keySerializer; private final Serializer valueSerializer; private final ProducerConfig producerConfig; + private final SecurityConfig securityConfig; /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -214,6 +216,7 @@ public class KafkaProducer implements Producer { this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); + this.securityConfig = ClientUtils.parseSecurityConfig(config.getString(ProducerConfig.SECURITY_CONFIG_FILE_CONFIG)); Map metricTags = new LinkedHashMap(); metricTags.put("client-id", clientId); this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), @@ -228,7 +231,7 @@ public class KafkaProducer implements Producer { List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); - NetworkClient client = new NetworkClient(new Selector(this.metrics, time, "producer", metricTags), + NetworkClient client = new NetworkClient(new Selector(this.metrics, time, "producer", metricTags, securityConfig), this.metadata, clientId, config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 187d000..83506e7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -169,6 +169,11 @@ public class ProducerConfig extends AbstractConfig { public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + /** security.config.file */ + public static final String SECURITY_CONFIG_FILE_CONFIG = CommonClientConfigs.SECURITY_CONFIG_FILE_CONFIG; + private static final String SECURITY_CONFIG_FILE_DOC = CommonClientConfigs.SECURITY_CONFIG_FILE_DOC; + + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Importance.HIGH, CommonClientConfigs.BOOSTRAP_SERVERS_DOC) .define(BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), Importance.HIGH, BUFFER_MEMORY_DOC) @@ -217,7 +222,8 @@ public class ProducerConfig extends AbstractConfig { Importance.LOW, MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC); + .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) + .define(SECURITY_CONFIG_FILE_CONFIG, Type.STRING, "", Importance.MEDIUM, SECURITY_CONFIG_FILE_DOC); } public static Map addSerializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java b/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java new file mode 100644 index 0000000..7954a7e --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java @@ -0,0 +1,113 @@ +/** + * 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.config; + + +import java.util.Map; + +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.protocol.SecurityProtocol; + + +/** + * Security Related config for clients and server. + */ + +public class SecurityConfig extends AbstractConfig { + /* + * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS THESE ARE PART OF THE PUBLIC API AND + * CHANGE WILL BREAK USER CODE. + */ + + private static final ConfigDef CONFIG; + + public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol"; + public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT and SSL are supported."; + + public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol"; + public static final String SSL_PROTOCOL_DOC = "The TLS protocol used for broker connections if security protocol is SSL. " + + "Any version of TLS is accepted by default."; + + public static final String SSL_CIPHER_SUITES_CONFIG = "ssl.cipher.suites"; + public static final String SSL_CIPHER_SUITES_DOC = "The list of cipher suites enabled for SSL connections. " + + "Default value is the list of cipher suites enabled for the Java Virtual Machine."; + + 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. " + + "Default value is the list of protocols enabled for the Java Virtual Machine."; + + + public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; + public static final String SSL_KEYSTORE_TYPE_DOC = "The file format of the key store file. " + + "Default value is the default key store format of the Java Virtual Machine."; + + public static final String SSL_KEYSTORE_LOCATION_CONFIG = "ssl.keystore.location"; + public static final String SSL_KEYSTORE_LOCATION_DOC = "The location of the key store file. " + + "This is optional for Client and can be used for two-way authentication for client."; + + public static final String SSL_KEYSTORE_PASSWORD_CONFIG = "ssl.keystore.password"; + public static final String SSL_KEYSTORE_PASSWORD_DOC = "The store password for the key store file. "; + + + public static final String SSL_KEY_PASSWORD_CONFIG = "ssl.key.password"; + public static final String SSL_KEY_PASSWORD_DOC = "The password of the private key in the key store file. " + + "This is optional for client."; + + public static final String SSL_TRUSTSTORE_TYPE_CONFIG = "ssl.truststore.type"; + public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file. " + + "Default value is JKS."; + + public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location"; + public static final String SSL_TRUSTSTORE_LOCATION_DOC = "The location of the trust store file. "; + + public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "ssl.truststore.password"; + public static final String SSL_TRUSTSTORE_PASSWORD_DOC = "The password for the trust store file. "; + + public static final String SSL_CLIENT_REQUIRE_CERT_CONFIG = "ssl.client.require.cert"; + public static final String SSL_CLIENT_REQUIRE_CERT_DOC = "This is to enforce two-way authentication between client and server." + + "Default value is false. If set to true client need to prover Keystrore releated config"; + + public static final String SSL_KEYMANAGER_ALGORITHM_CONFIG = "ssl.keymanager.algorithm"; + public static final String SSL_KEYMANAGER_ALGORITHM_DOC = "The algorithm used by key manager factory for SSL connections. " + + "Default value is the key manager factory algorithm configured for the Java Virtual Machine."; + + public static final String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = "ssl.trustmanager.algorithm"; + public static final String SSL_TRUSTMANAGER_ALGORITHM_DOC = "The algorithm used by trust manager factory for SSL connections. " + + "Default value is the trust manager factory algorithm configured for the Java Virtual Machine."; + + + static { + CONFIG = new ConfigDef().define(SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, SECURITY_PROTOCOL_DOC) + .define(SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, SSL_PROTOCOL_DOC) + .define(SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.MEDIUM, SSL_CIPHER_SUITES_DOC, false) + .define(SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, SSL_ENABLED_PROTOCOLS_DOC) + .define(SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_TYPE_DOC, false) + .define(SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_LOCATION_DOC, false) + .define(SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_PASSWORD_DOC, false) + .define(SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEY_PASSWORD_DOC, false) + .define(SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_TYPE_DOC, false) + .define(SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_LOCATION_DOC, false) + .define(SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_PASSWORD_DOC, false) + .define(SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYMANAGER_ALGORITHM_DOC, false) + .define(SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTMANAGER_ALGORITHM_DOC, false) + .define(SSL_CLIENT_REQUIRE_CERT_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, SSL_CLIENT_REQUIRE_CERT_DOC); + } + + public SecurityConfig(Map props) { + super(CONFIG, props); + } + + +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java new file mode 100644 index 0000000..ee8516f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java @@ -0,0 +1,60 @@ +/** + * 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; + +/* + * Authentication for Channel + */ + +import java.io.IOException; +import com.sun.security.auth.UserPrincipal; + + +public interface Authenticator { + + /** + * Closes this channel + * + * @throws IOException if any I/O error occurs + */ + void close() throws IOException; + + /** + * + * @throws IOException + */ + void init() throws IOException; + + /** + * Returns UserPrincipal after authentication is established + */ + UserPrincipal userPrincipal(); + + + /** + * Does authentication and returns SelectionKey.OP if further communication needed + */ + int authenticate(boolean read, boolean write) throws IOException; + + /** + * returns true if authentication is complete otherwise returns false; + */ + + boolean isComplete(); + +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/Channel.java b/clients/src/main/java/org/apache/kafka/common/network/Channel.java new file mode 100644 index 0000000..3526ba3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/Channel.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.io.IOException; +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import java.nio.ByteBuffer; +import java.nio.channels.ScatteringByteChannel; +import java.nio.channels.GatheringByteChannel; +import java.nio.channels.SocketChannel; + +import com.sun.security.auth.UserPrincipal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ + +public class Channel implements ScatteringByteChannel, GatheringByteChannel { + private static final Logger log = LoggerFactory.getLogger(Channel.class); + private TransportLayer transportLayer; + private Authenticator authenticator; + + + public Channel(TransportLayer transportLayer, Authenticator authenticator) throws IOException { + this.transportLayer = transportLayer; + this.authenticator = authenticator; + this.authenticator.init(); + } + + public void close() throws IOException { + transportLayer.close(); + authenticator.close(); + } + + /** + * returns user principal for the session + * Incase of PLAINTEXT and No Authentication returns ANONYMOUS as the userPrincipal + * If SSL used without any SASL Authentication returns SSLSession.peerPrincipal + */ + public UserPrincipal userPrincipal() { + return authenticator.userPrincipal(); + } + + public int connect(boolean read, boolean write) throws IOException { + if (transportLayer.isReady() && authenticator.isComplete()) + return 0; + int status = 0; + if (!transportLayer.isReady()) + status = transportLayer.handshake(read, write); + if (status == 0 && !authenticator.isComplete()) + status = authenticator.authenticate(read, write); + return status; + } + + + public boolean isOpen() { + return transportLayer.isOpen(); + } + + public SocketChannel socketChannel() { + return transportLayer.socketChannel(); + } + + /** + * Writes a sequence of bytes to this channel from the given buffer. + */ + @Override + public int write(ByteBuffer src) throws IOException { + return transportLayer.write(src); + } + + @Override + public long write(ByteBuffer[] srcs) throws IOException { + return transportLayer.write(srcs); + } + + @Override + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + return transportLayer.write(srcs, offset, length); + } + + @Override + public int read(ByteBuffer dst) throws IOException { + return transportLayer.read(dst); + } + + @Override + public long read(ByteBuffer[] dsts) throws IOException { + return transportLayer.read(dsts); + } + + @Override + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + return transportLayer.read(dsts, offset, length); + } + + public boolean finishConnect() throws IOException { + return transportLayer.finishConnect(); + } + + public boolean isReady() { + return transportLayer.isReady() && authenticator.isComplete(); + } + + public DataInputStream getInputStream() throws IOException { + return transportLayer.inStream(); + } + + public DataOutputStream getOutputStream() throws IOException { + return transportLayer.outStream(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java new file mode 100644 index 0000000..c1ec794 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java @@ -0,0 +1,46 @@ +/** + * 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 com.sun.security.auth.UserPrincipal; +import java.io.IOException; + +public class DefaultAuthenticator implements Authenticator { + + TransportLayer transportLayer; + + public DefaultAuthenticator(TransportLayer transportLayer) { + this.transportLayer = transportLayer; + } + + public void init() {} + + public int authenticate(boolean read, boolean write) throws IOException { + return 0; + } + + public UserPrincipal userPrincipal() { + return new UserPrincipal(transportLayer.getPeerPrincipal().toString()); + } + + public void close() throws IOException {} + + public boolean isComplete() { + return true; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java new file mode 100644 index 0000000..11cd80c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java @@ -0,0 +1,148 @@ +/** + * 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; + +/* + * Transport layer for PLAINTEXT communication + */ + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; + +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import java.security.Principal; +import com.sun.security.auth.UserPrincipal; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PlainTextTransportLayer implements TransportLayer { + private static final Logger log = LoggerFactory.getLogger(PlainTextTransportLayer.class); + SocketChannel socketChannel = null; + DataInputStream inStream = null; + DataOutputStream outStream = null; + + public PlainTextTransportLayer(SocketChannel socketChannel) throws IOException { + this.socketChannel = socketChannel; + + } + + + /** + * Closes this channel + * + * @throws IOException If and I/O error occurs + */ + public void close() throws IOException { + socketChannel.socket().close(); + socketChannel.close(); + } + + /** + * Flushes the buffer to the network, non blocking + * @param buf ByteBuffer + * @return boolean true if the buffer has been emptied out, false otherwise + * @throws IOException + */ + public boolean flush(ByteBuffer buf) throws IOException { + int remaining = buf.remaining(); + if (remaining > 0) { + int written = socketChannel.write(buf); + return written >= remaining; + } + return true; + } + + /** + * Tells wheter or not this channel is open. + */ + public boolean isOpen() { + return socketChannel.isOpen(); + } + + /** + * Writes a sequence of bytes to this channel from the given buffer. + */ + public int write(ByteBuffer src) throws IOException { + return socketChannel.write(src); + } + + public long write(ByteBuffer[] srcs) throws IOException { + return socketChannel.write(srcs); + } + + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + return socketChannel.write(srcs, offset, length); + } + + public int read(ByteBuffer dst) throws IOException { + return socketChannel.read(dst); + } + + public long read(ByteBuffer[] dsts) throws IOException { + return socketChannel.read(dsts); + } + + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + return socketChannel.read(dsts, offset, length); + } + + public boolean isReady() { + return true; + } + + public SocketChannel socketChannel() { + return socketChannel; + } + + public boolean finishConnect() throws IOException { + return socketChannel.finishConnect(); + } + + /** + * Performs SSL handshake hence is a no-op for the non-secure + * implementation + * @param read Unused in non-secure implementation + * @param write Unused in non-secure implementation + * @return Always return 0 + * @throws IOException + */ + public int handshake(boolean read, boolean write) throws IOException { + return 0; + } + + public DataInputStream inStream() throws IOException { + if (inStream == null) + this.inStream = new DataInputStream(socketChannel.socket().getInputStream()); + return inStream; + } + + public DataOutputStream outStream() throws IOException { + if (outStream == null) + this.outStream = new DataOutputStream(socketChannel.socket().getOutputStream()); + return outStream; + } + + public Principal getPeerPrincipal() { + return new UserPrincipal("ANONYMOUS"); + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java new file mode 100644 index 0000000..9cf9051 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java @@ -0,0 +1,188 @@ +/** + * 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.io.FileInputStream; +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; + +import javax.net.ssl.*; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SecurityConfig; + + +public class SSLFactory { + + public enum Mode { CLIENT, SERVER }; + private String protocol; + private String provider; + private String kmfAlgorithm; + private String tmfAlgorithm; + private SecurityStore keystore = null; + private String keyPassword; + private SecurityStore truststore; + private String[] cipherSuites; + private String[] enabledProtocols; + private SSLContext sslContext; + private boolean requireClientCert; + private Mode mode; + + + public SSLFactory(Mode mode) { + this.mode = mode; + } + + + public void init(SecurityConfig securityConfig) throws IOException, GeneralSecurityException { + this.protocol = securityConfig.getString(SecurityConfig.SSL_PROTOCOL_CONFIG); + if (securityConfig.getList(SecurityConfig.SSL_CIPHER_SUITES_CONFIG) != null) + this.cipherSuites = (String[]) securityConfig.getList(SecurityConfig.SSL_CIPHER_SUITES_CONFIG).toArray(); + if (securityConfig.getList(SecurityConfig.SSL_ENABLED_PROTOCOLS_CONFIG) != null) + this.enabledProtocols = (String[]) securityConfig.getList(SecurityConfig.SSL_ENABLED_PROTOCOLS_CONFIG).toArray(); + this.requireClientCert = securityConfig.getBoolean(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG); + this.kmfAlgorithm = securityConfig.getString(SecurityConfig.SSL_KEYMANAGER_ALGORITHM_CONFIG); + this.tmfAlgorithm = securityConfig.getString(SecurityConfig.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); + if ((mode == Mode.CLIENT && requireClientCert) || (mode == Mode.SERVER)) + createKeystore(securityConfig.getString(SecurityConfig.SSL_KEYSTORE_TYPE_CONFIG), + securityConfig.getString(SecurityConfig.SSL_KEYSTORE_LOCATION_CONFIG), + securityConfig.getString(SecurityConfig.SSL_KEYSTORE_PASSWORD_CONFIG), + securityConfig.getString(SecurityConfig.SSL_KEY_PASSWORD_CONFIG)); + createTruststore(securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG), + securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG), + securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + + this.sslContext = createSSLContext(); + + } + + + private SSLContext createSSLContext() throws GeneralSecurityException, IOException { + SSLContext sslContext; + if (provider != null) + sslContext = SSLContext.getInstance(protocol, provider); + else + sslContext = SSLContext.getInstance(protocol); + + KeyManager[] keyManagers = null; + if (keystore != null) { + String kmfAlgorithm = this.kmfAlgorithm != null ? this.kmfAlgorithm : KeyManagerFactory.getDefaultAlgorithm(); + KeyManagerFactory kmf = KeyManagerFactory.getInstance(kmfAlgorithm); + KeyStore ks = keystore.load(); + String keyPassword = this.keyPassword != null ? this.keyPassword : keystore.password; + kmf.init(ks, keyPassword.toCharArray()); + keyManagers = kmf.getKeyManagers(); + } + + String tmfAlgorithm = this.tmfAlgorithm != null ? this.tmfAlgorithm : TrustManagerFactory.getDefaultAlgorithm(); + TrustManagerFactory tmf = TrustManagerFactory.getInstance(tmfAlgorithm); + KeyStore ts = truststore == null ? null : truststore.load(); + tmf.init(ts); + + sslContext.init(keyManagers, tmf.getTrustManagers(), null); + return sslContext; + } + + public SSLEngine createSSLEngine(String peerHost, int peerPort) { + SSLEngine sslEngine = sslContext.createSSLEngine(peerHost, peerPort); + if (cipherSuites != null) sslEngine.setEnabledCipherSuites(cipherSuites); + if (mode == Mode.SERVER) { + sslEngine.setUseClientMode(false); + } else { + sslEngine.setUseClientMode(true); + sslEngine.setNeedClientAuth(requireClientCert); + } + if (enabledProtocols != null) sslEngine.setEnabledProtocols(enabledProtocols); + return sslEngine; + } + + /** + * Returns a configured SSLServerSocketFactory. + * + * @return the configured SSLSocketFactory. + * @throws GeneralSecurityException thrown if the SSLSocketFactory could not + * be initialized. + * @throws IOException thrown if and IO error occurred while loading + * the server keystore. + */ + public SSLServerSocketFactory createSSLServerSocketFactory() throws GeneralSecurityException, IOException { + if (mode != Mode.SERVER) { + throw new IllegalStateException("Factory is in CLIENT mode"); + } + return sslContext.getServerSocketFactory(); + } + + /** + * Returns if client certificates are required or not. + * + * @return if client certificates are required or not. + */ + public boolean isClientCertRequired() { + return requireClientCert; + } + + + private void createKeystore(String type, String path, String password, String keyPassword) { + if (path == null && password != null) { + throw new KafkaException("SSL key store password is not specified."); + } else if (path != null && password == null) { + throw new KafkaException("SSL key store is not specified, but key store password is specified."); + } else if (path != null && password != null) { + this.keystore = new SecurityStore(type, path, password); + this.keyPassword = keyPassword; + } + } + + private void createTruststore(String type, String path, String password) { + if (path == null && password != null) { + throw new KafkaException("SSL key store password is not specified."); + } else if (path != null && password == null) { + throw new KafkaException("SSL key store is not specified, but key store password is specified."); + } else if (path != null && password != null) { + this.truststore = new SecurityStore(type, path, password); + } + } + + + private class SecurityStore { + private final String type; + private final String path; + private final String password; + + private SecurityStore(String type, String path, String password) { + this.type = type == null ? KeyStore.getDefaultType() : type; + this.path = path; + this.password = password; + } + + private KeyStore load() throws GeneralSecurityException, IOException { + FileInputStream in = null; + try { + KeyStore ks = KeyStore.getInstance(type); + in = new FileInputStream(path); + ks.load(in, password.toCharArray()); + return ks; + } finally { + if (in != null) in.close(); + } + } + } + + public void close() { + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java new file mode 100644 index 0000000..dc84975 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java @@ -0,0 +1,430 @@ +/** + * 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.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; +import java.nio.channels.SelectionKey; + +import java.security.Principal; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLEngineResult; +import javax.net.ssl.SSLEngineResult.HandshakeStatus; +import javax.net.ssl.SSLEngineResult.Status; + +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/* + * Transport layer for SSL communication + */ + +public class SSLTransportLayer implements TransportLayer { + private static final Logger log = LoggerFactory.getLogger(SSLTransportLayer.class); + SocketChannel socketChannel; + SSLEngine sslEngine; + HandshakeStatus handshakeStatus = null; + SSLEngineResult handshakeResult = null; + boolean handshakeComplete = false; + boolean closed = false; + boolean closing = false; + ByteBuffer netInBuffer = null; + ByteBuffer netOutBuffer = null; + ByteBuffer appReadBuffer = null; + ByteBuffer appWriteBuffer = null; + ByteBuffer emptyBuf = ByteBuffer.allocate(0); + DataInputStream inStream = null; + DataOutputStream outStream = null; + + + public SSLTransportLayer(SocketChannel socketChannel, SSLEngine sslEngine) throws IOException { + this.socketChannel = socketChannel; + this.sslEngine = sslEngine; + this.netInBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getPacketBufferSize()); + this.netOutBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getPacketBufferSize()); + this.appWriteBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getApplicationBufferSize()); + this.appReadBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getApplicationBufferSize()); + startHandshake(); + } + + public void startHandshake() throws IOException { + netOutBuffer.position(0); + netOutBuffer.limit(0); + netInBuffer.position(0); + netInBuffer.limit(0); + handshakeComplete = false; + closed = false; + closing = false; + //initiate handshake + sslEngine.beginHandshake(); + handshakeStatus = sslEngine.getHandshakeStatus(); + } + + public SocketChannel socketChannel() { + return socketChannel; + } + + public boolean finishConnect() throws IOException { + return socketChannel.finishConnect(); + } + + /** + * Flushes the buffer to the network, non blocking + * @param buf ByteBuffer + * @return boolean true if the buffer has been emptied out, false otherwise + * @throws IOException + */ + public boolean flush(ByteBuffer buf) throws IOException { + int remaining = buf.remaining(); + if (remaining > 0) { + int written = socketChannel.write(buf); + return written >= remaining; + } + return true; + } + + /** + * Performs SSL handshake, non blocking. + * The return for this operation is 0 if the handshake is complete and a positive value if it is not complete. + * In the event of a positive value coming back, re-register the selection key for the return values interestOps. + * @param read boolean - true if the underlying channel is readable + * @param write boolean - true if the underlying channel is writable + * @return int - 0 if hand shake is complete, otherwise it returns a SelectionKey interestOps value + * @throws IOException + */ + public int handshake(boolean read, boolean write) throws IOException { + if (handshakeComplete) return 0; //we have done our initial handshake + + if (!flush(netOutBuffer)) return SelectionKey.OP_WRITE; + + switch(handshakeStatus) { + case NOT_HANDSHAKING: + // SSLEnginge.getHandshakeStatus is transient and it doesn't record FINISHED status properly + if (handshakeResult.getHandshakeStatus() == HandshakeStatus.FINISHED) { + handshakeComplete = !netOutBuffer.hasRemaining(); + if (handshakeComplete) + return 0; + else + return SelectionKey.OP_WRITE; + } else { + //should never happen + throw new IOException("NOT_HANDSHAKING during handshake"); + } + case FINISHED: + //we are complete if we have delivered the last package + handshakeComplete = !netOutBuffer.hasRemaining(); + //return 0 if we are complete, otherwise we still have data to write + if (handshakeComplete) return 0; + else return SelectionKey.OP_WRITE; + case NEED_WRAP: + handshakeResult = handshakeWrap(write); + if (handshakeResult.getStatus() == Status.OK) { + if (handshakeStatus == HandshakeStatus.NEED_TASK) + handshakeStatus = tasks(); + } else { + //wrap should always work with our buffers + throw new IOException("Unexpected status [" + handshakeResult.getStatus() + "] during handshake WRAP."); + } + if (handshakeStatus != HandshakeStatus.NEED_UNWRAP || (!flush(netOutBuffer))) + return SelectionKey.OP_WRITE; + //fall down to NEED_UNWRAP on the same call, will result in a + //BUFFER_UNDERFLOW if it needs data + case NEED_UNWRAP: + handshakeResult = handshakeUnwrap(read); + if (handshakeResult.getStatus() == Status.OK) { + if (handshakeStatus == HandshakeStatus.NEED_TASK) + handshakeStatus = tasks(); + } else if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { + return SelectionKey.OP_READ; + } else { + throw new IOException(String.format("Unexpected status [%s] during handshake UNWRAP", handshakeStatus)); + } + break; + case NEED_TASK: + handshakeStatus = tasks(); + break; + default: + throw new IllegalStateException(String.format("Unexpected status [%s]", handshakeStatus)); + } + //return 0 if we are complete, otherwise re-register for any activity that + //would cause this method to be called again. + if (handshakeComplete) return 0; + else return SelectionKey.OP_WRITE | SelectionKey.OP_READ; + } + + /** + * Executes all the tasks needed on the same thread. + * @return HandshakeStatus + */ + private HandshakeStatus tasks() { + Runnable r = null; + while ((r = sslEngine.getDelegatedTask()) != null) r.run(); + return sslEngine.getHandshakeStatus(); + } + + /** + * Performs the WRAP function + * @param doWrite boolean + * @return SSLEngineResult + * @throws IOException + */ + private SSLEngineResult handshakeWrap(Boolean doWrite) throws IOException { + //this should never be called with a network buffer that contains data + //so we can clear it here. + netOutBuffer.clear(); + SSLEngineResult result = sslEngine.wrap(appWriteBuffer, netOutBuffer); + //prepare the results to be written + netOutBuffer.flip(); + handshakeStatus = result.getHandshakeStatus(); + //optimization, if we do have a writable channel, write it now + if (doWrite) flush(netOutBuffer); + return result; + } + + /** + * Perform handshake unwrap + * @param doRead boolean + * @return SSLEngineResult + * @throws IOException + */ + private SSLEngineResult handshakeUnwrap(Boolean doRead) throws IOException { + if (netInBuffer.position() == netInBuffer.limit()) { + //clear the buffer if we have emptied it out on data + netInBuffer.clear(); + } + + if (doRead) { + int read = socketChannel.read(netInBuffer); + if (read == -1) throw new IOException("EOF during handshake."); + } + + SSLEngineResult result; + boolean cont = false; + do { + //prepare the buffer with the incoming data + netInBuffer.flip(); + result = sslEngine.unwrap(netInBuffer, appWriteBuffer); + netInBuffer.compact(); + handshakeStatus = result.getHandshakeStatus(); + if (result.getStatus() == SSLEngineResult.Status.OK && + result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) { + handshakeStatus = tasks(); + } + cont = result.getStatus() == SSLEngineResult.Status.OK && + handshakeStatus == HandshakeStatus.NEED_UNWRAP; + } while(cont); + return result; + } + + + public int getOutboundRemaining() { + return netOutBuffer.remaining(); + } + + /** + * Sends a SSL close message, will not physically close the connection here.
+ * @throws IOException if an I/O error occurs + * @throws IOException if there is data on the outgoing network buffer and we are unable to flush it + */ + public void close() throws IOException { + if (closing) return; + closing = true; + sslEngine.closeOutbound(); + + if (!flush(netOutBuffer)) { + throw new IOException("Remaining data in the network buffer, can't send SSL close message, force a close with close(true) instead"); + } + //prep the buffer for the close message + netOutBuffer.clear(); + //perform the close, since we called sslEngine.closeOutbound + SSLEngineResult handshake = sslEngine.wrap(emptyBuf, netOutBuffer); + //we should be in a close state + if (handshake.getStatus() != SSLEngineResult.Status.CLOSED) { + throw new IOException("Invalid close state, will not send network data."); + } + netOutBuffer.flip(); + flush(netOutBuffer); + socketChannel.socket().close(); + socketChannel.close(); + closed = !netOutBuffer.hasRemaining() && (handshake.getHandshakeStatus() != HandshakeStatus.NEED_WRAP); + } + + public boolean isOpen() { + return socketChannel.isOpen(); + } + + public boolean isReady() { + return handshakeComplete; + } + + /** + * Reads a sequence of bytes from this channel into the given buffer. + * + * @param dst The buffer into which bytes are to be transferred + * @return The number of bytes read, possible zero or -1 if the channel has reached end-of-stream + * @throws IOException if some other I/O error occurs + * @throws IllegalStateException if the destination buffer is different than appBufHandler.getReadBuffer() + */ + public int read(ByteBuffer dst) throws IOException { + if (closing || closed) return -1; + if (!handshakeComplete) throw new IllegalStateException("Handshake incomplete."); + netInBuffer = Utils.ensureCapacity(netInBuffer, packetBufferSize()); + int netread = socketChannel.read(netInBuffer); + if (netread == -1) return -1; + int read = 0; + SSLEngineResult unwrap = null; + + do { + netInBuffer.flip(); + unwrap = sslEngine.unwrap(netInBuffer, appReadBuffer); + //compact the buffer + netInBuffer.compact(); + if (unwrap.getStatus() == Status.OK || unwrap.getStatus() == Status.BUFFER_UNDERFLOW) { + read += unwrap.bytesProduced(); + // perform any task if needed + if (unwrap.getHandshakeStatus() == HandshakeStatus.NEED_TASK) tasks(); + //if we need more network data, than return for now. + if (unwrap.getStatus() == Status.BUFFER_UNDERFLOW) return readFromAppBuffer(dst); + } else if (unwrap.getStatus() == Status.BUFFER_OVERFLOW && read > 0) { + appReadBuffer = Utils.ensureCapacity(appReadBuffer, applicationBufferSize()); + //buffer overflow can happen, if we have read data, then + //empty out the dst buffer before we do another read + return readFromAppBuffer(dst); + } else { + //here we should trap BUFFER_OVERFLOW and call expand on the buffer + // for now, throw an exception, as we initialized the buffers + // in constructor + throw new IOException(String.format("Unable to unwrap data, invalid status [%s]", unwrap.getStatus())); + } + } while(netInBuffer.position() != 0); + return readFromAppBuffer(dst); + } + + public long read(ByteBuffer[] dsts) throws IOException { + return read(dsts, 0, dsts.length); + } + + public long read(ByteBuffer[] dsts, int offset, int length) throws IOException { + int totalRead = 0; + for (int i = offset; i < length; i++) { + int read = read(dsts[i]); + if (read > 0) { + totalRead += read; + } + } + return totalRead; + } + + + /** + * Writes a sequence of bytes to this channel from the given buffer. + * + * @param src The buffer from which bytes are to be retrieved + * @return The number of bytes written, possibly zero + * @throws IOException If some other I/O error occurs + */ + + public int write(ByteBuffer src) throws IOException { + int written = 0; + if (src == this.netOutBuffer) + written = socketChannel.write(src); + else { + if (closing || closed) throw new IOException("Channel is in closing state"); + if (!flush(netOutBuffer)) + return written; + netOutBuffer.clear(); + SSLEngineResult result = sslEngine.wrap(src, netOutBuffer); + written = result.bytesConsumed(); + netOutBuffer.flip(); + if (result.getStatus() == Status.OK) { + if (result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) + tasks(); + } else { + throw new IOException(String.format("Unable to wrap data, invalid status %s", result.getStatus())); + } + flush(netOutBuffer); + } + return written; + } + + public long write(ByteBuffer[] srcs, int offset, int length) throws IOException { + int totalWritten = 0; + for (int i = offset; i < length; i++) { + if (srcs[i].hasRemaining()) { + int written = write(srcs[i]); + if (written > 0) { + totalWritten += written; + } + } + } + return totalWritten; + } + + public long write(ByteBuffer[] srcs) throws IOException { + return write(srcs, 0, srcs.length); + } + + public DataInputStream inStream() throws IOException { + if (inStream == null) + this.inStream = new DataInputStream(socketChannel.socket().getInputStream()); + return inStream; + } + + public DataOutputStream outStream() throws IOException { + if (outStream == null) + this.outStream = new DataOutputStream(socketChannel.socket().getOutputStream()); + return outStream; + } + + public Principal getPeerPrincipal() { + //return sslEngine.getSession().getPeerPrincipal(); + return null; + } + + private int readFromAppBuffer(ByteBuffer dst) { + appReadBuffer.flip(); + try { + int remaining = appReadBuffer.remaining(); + if (remaining > 0) { + if (remaining > dst.remaining()) + remaining = dst.remaining(); + int i = 0; + while (i < remaining) { + dst.put(appReadBuffer.get()); + i++; + } + } + return remaining; + } finally { + appReadBuffer.compact(); + } + } + + private int packetBufferSize() { + return sslEngine.getSession().getPacketBufferSize(); + } + + private int applicationBufferSize() { + return sslEngine.getSession().getApplicationBufferSize(); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java index b5f8d83..4a0c2bd 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selectable.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.network; + import java.io.IOException; import java.net.InetSocketAddress; import java.util.List; diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 57de058..0068143 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -3,9 +3,9 @@ * 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. @@ -31,6 +31,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.MetricName; @@ -40,6 +41,7 @@ import org.apache.kafka.common.metrics.stats.Avg; import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,25 +53,25 @@ import org.slf4j.LoggerFactory; * responses. *

* A connection can be added to the selector associated with an integer id by doing - * + * *

  * selector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
  * 
- * + * * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating * the connection. The successful invocation of this method does not mean a valid connection has been established. - * + * * Sending requests, receiving responses, processing connection completions, and disconnections on the existing * connections are all done using the poll() call. - * + * *
  * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
  * selector.poll(TIMEOUT_MS, requestsToSend);
  * 
- * + * * The selector maintains several lists that are reset by each call to poll() which are available via * various getters. These are reset by each call to poll(). - * + * * This class is not thread safe! */ public class Selector implements Selectable { @@ -78,6 +80,7 @@ public class Selector implements Selectable { private final java.nio.channels.Selector selector; private final Map keys; + private final Map channels; private final List completedSends; private final List completedReceives; private final List disconnected; @@ -87,11 +90,14 @@ public class Selector implements Selectable { private final SelectorMetrics sensors; private final String metricGrpPrefix; private final Map metricTags; + private final SecurityConfig securityConfig; + private final SecurityProtocol securityProtocol; + private SSLFactory sslFactory = null; /** * Create a new selector */ - public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map metricTags) { + public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, SecurityConfig securityConfig) { try { this.selector = java.nio.channels.Selector.open(); } catch (IOException e) { @@ -101,12 +107,24 @@ public class Selector implements Selectable { this.metricGrpPrefix = metricGrpPrefix; this.metricTags = metricTags; this.keys = new HashMap(); + this.channels = new HashMap(); this.completedSends = new ArrayList(); this.completedReceives = new ArrayList(); this.connected = new ArrayList(); this.disconnected = new ArrayList(); this.failedSends = new ArrayList(); this.sensors = new SelectorMetrics(metrics); + this.securityConfig = securityConfig; + this.securityProtocol = SecurityProtocol.valueOf(securityConfig.getString(SecurityConfig.SECURITY_PROTOCOL_CONFIG)); + try { + if (securityProtocol == SecurityProtocol.SSL) { + this.sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT); + this.sslFactory.init(this.securityConfig); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } /** @@ -127,25 +145,37 @@ public class Selector implements Selectable { if (this.keys.containsKey(id)) throw new IllegalStateException("There is already a connection for id " + id); - SocketChannel channel = SocketChannel.open(); - channel.configureBlocking(false); - Socket socket = channel.socket(); + SocketChannel socketChannel = SocketChannel.open(); + socketChannel.configureBlocking(false); + Socket socket = socketChannel.socket(); socket.setKeepAlive(true); socket.setSendBufferSize(sendBufferSize); socket.setReceiveBufferSize(receiveBufferSize); socket.setTcpNoDelay(true); try { - channel.connect(address); + socketChannel.connect(address); } catch (UnresolvedAddressException e) { - channel.close(); + socketChannel.close(); throw new IOException("Can't resolve address: " + address, e); } catch (IOException e) { - channel.close(); + socketChannel.close(); throw e; } - SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT); + + TransportLayer transportLayer; + if (securityProtocol == SecurityProtocol.SSL) { + transportLayer = new SSLTransportLayer(socketChannel, + sslFactory.createSSLEngine(socket.getInetAddress().getHostName(), + socket.getPort())); + } else { + transportLayer = new PlainTextTransportLayer(socketChannel); + } + Authenticator authenticator = new DefaultAuthenticator(transportLayer); + Channel channel = new Channel(transportLayer, authenticator); + SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT); key.attach(new Transmissions(id)); this.keys.put(id, key); + this.channels.put(key, channel); } /** @@ -202,12 +232,12 @@ public class Selector implements Selectable { /** * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing * disconnections, initiating new sends, or making progress on in-progress sends or receives. - * + * * When this call is completed the user can check for completed sends, receives, connections or disconnects using * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any * completed I/O. - * + * * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely. * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is * already an in-progress send @@ -230,7 +260,7 @@ public class Selector implements Selectable { iter.remove(); Transmissions transmissions = transmissions(key); - SocketChannel channel = channel(key); + Channel channel = channel(key); // register all per-broker metrics at once sensors.maybeRegisterNodeMetrics(transmissions.id); @@ -242,29 +272,46 @@ public class Selector implements Selectable { key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); this.connected.add(transmissions.id); this.sensors.connectionCreated.record(); + } /* read from any connections that have readable data */ if (key.isReadable()) { - if (!transmissions.hasReceive()) - transmissions.receive = new NetworkReceive(transmissions.id); - transmissions.receive.readFrom(channel); - if (transmissions.receive.complete()) { - transmissions.receive.payload().rewind(); - this.completedReceives.add(transmissions.receive); - this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit()); - transmissions.clearReceive(); + if (!channel.isReady()) { + int status = channel.connect(key.isReadable(), key.isWritable()); + if (status == 0) + key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + else + key.interestOps(status); + } else { + if (!transmissions.hasReceive()) + transmissions.receive = new NetworkReceive(transmissions.id); + transmissions.receive.readFrom(channel); + if (transmissions.receive.complete()) { + transmissions.receive.payload().rewind(); + this.completedReceives.add(transmissions.receive); + this.sensors.recordBytesReceived(transmissions.id, transmissions.receive.payload().limit()); + transmissions.clearReceive(); + } } } /* write to any sockets that have space in their buffer and for which we have data */ if (key.isWritable()) { - transmissions.send.writeTo(channel); - if (transmissions.send.remaining() <= 0) { - this.completedSends.add(transmissions.send); - this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); - transmissions.clearSend(); - key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); + if (!channel.isReady()) { + int status = channel.connect(key.isReadable(), key.isWritable()); + if (status == 0) + key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + else + key.interestOps(status); + } else { + transmissions.send.writeTo(channel); + if (transmissions.send.remaining() <= 0) { + this.completedSends.add(transmissions.send); + this.sensors.recordBytesSent(transmissions.id, transmissions.send.size()); + transmissions.clearSend(); + key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE); + } } } @@ -287,9 +334,9 @@ public class Selector implements Selectable { long endIo = time.nanoseconds(); this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); } - - private String socketDescription(SocketChannel channel) { - Socket socket = channel.socket(); + + private String socketDescription(Channel channel) { + Socket socket = channel.socketChannel().socket(); if (socket == null) return "[unconnected socket]"; else if (socket.getInetAddress() != null) @@ -362,7 +409,7 @@ public class Selector implements Selectable { /** * Check for data, waiting up to the given timeout. - * + * * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely. * @return The number of keys ready * @throws IOException @@ -380,7 +427,8 @@ public class Selector implements Selectable { * Begin closing this connection */ private void close(SelectionKey key) { - SocketChannel channel = channel(key); + Channel channel = channel(key); + this.channels.remove(key); Transmissions trans = transmissions(key); if (trans != null) { this.keys.remove(trans.id); @@ -390,7 +438,6 @@ public class Selector implements Selectable { key.attach(null); key.cancel(); try { - channel.socket().close(); channel.close(); } catch (IOException e) { log.error("Exception closing connection to node {}:", trans.id, e); @@ -418,8 +465,8 @@ public class Selector implements Selectable { /** * Get the socket channel associated with this selection key */ - private SocketChannel channel(SelectionKey key) { - return (SocketChannel) key.channel(); + private Channel channel(SelectionKey key) { + return this.channels.get(key); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java new file mode 100644 index 0000000..6ce013b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -0,0 +1,86 @@ +/** + * 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; + +/* + * Transport layer for underlying communication + */ +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; + +import java.io.DataInputStream; +import java.io.DataOutputStream; + +import java.security.Principal; + + +public interface TransportLayer { + + /** + * Closes this channel + * + * @throws IOException If and I/O error occurs + */ + void close() throws IOException; + + + /** + * Tells wheather or not this channel is open. + */ + boolean isOpen(); + + /** + * Writes a sequence of bytes to this channel from the given buffer. + */ + int write(ByteBuffer src) throws IOException; + + long write(ByteBuffer[] srcs) throws IOException; + + long write(ByteBuffer[] srcs, int offset, int length) throws IOException; + + int read(ByteBuffer dst) throws IOException; + + long read(ByteBuffer[] dsts) throws IOException; + + long read(ByteBuffer[] dsts, int offset, int length) throws IOException; + + boolean isReady(); + + boolean finishConnect() throws IOException; + + SocketChannel socketChannel(); + + /** + * Performs SSL handshake hence is a no-op for the non-secure + * implementation + * @param read Unused in non-secure implementation + * @param write Unused in non-secure implementation + * @return Always return 0 + * @throws IOException + */ + int handshake(boolean read, boolean write) throws IOException; + + DataInputStream inStream() throws IOException; + + DataOutputStream outStream() throws IOException; + + boolean flush(ByteBuffer buffer) throws IOException; + + Principal getPeerPrincipal(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java index dab1a94..d663f7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -24,6 +24,8 @@ import java.util.Map; public enum SecurityProtocol { /** Un-authenticated, non-encrypted channel */ PLAINTEXT(0, "PLAINTEXT"), + /** SSL channe */ + SSL(1, "PLAINTEXT"), /** Currently identical to PLAINTEXT and used for testing only. We may implement extra instrumentation when testing channel code. */ TRACE(Short.MAX_VALUE, "TRACE"); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index f73eedb..9382060 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -479,4 +479,15 @@ public class Utils { public static String readFileAsString(String path) throws IOException { return Utils.readFileAsString(path, Charset.defaultCharset()); } + + public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength) { + if (newLength > existingBuffer.capacity()) { + ByteBuffer newBuffer = ByteBuffer.allocate(newLength); + existingBuffer.flip(); + newBuffer.put(existingBuffer); + return newBuffer; + } + return existingBuffer; + } + } diff --git a/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java new file mode 100644 index 0000000..47dda69 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java @@ -0,0 +1,108 @@ +/** + * 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 org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.common.protocol.SecurityProtocol; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; + + +/** + * A simple server that takes size delimited byte arrays and just echos them back to the sender. + */ +class EchoServer extends Thread { + public final int port; + private final ServerSocket serverSocket; + private final List threads; + private final List sockets; + private SecurityProtocol protocol; + private SSLFactory sslFactory; + private final AtomicBoolean startHandshake = new AtomicBoolean(); + + public EchoServer(SecurityConfig securityConfig) throws Exception { + this.protocol = SecurityProtocol.valueOf(securityConfig.getString(SecurityConfig.SECURITY_PROTOCOL_CONFIG)); + if (protocol == SecurityProtocol.SSL) { + this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); + this.sslFactory.init(securityConfig); + this.serverSocket = sslFactory.createSSLServerSocketFactory().createServerSocket(0); + this.startHandshake.set(true); + } else { + this.serverSocket = new ServerSocket(0); + } + this.port = this.serverSocket.getLocalPort(); + this.threads = Collections.synchronizedList(new ArrayList()); + this.sockets = Collections.synchronizedList(new ArrayList()); + } + + + @Override + public void run() { + try { + while (true) { + final Socket socket = serverSocket.accept(); + sockets.add(socket); + Thread thread = new Thread() { + @Override + public void run() { + try { + DataInputStream input = new DataInputStream(socket.getInputStream()); + DataOutputStream output = new DataOutputStream(socket.getOutputStream()); + while (socket.isConnected() && !socket.isClosed()) { + int size = input.readInt(); + byte[] bytes = new byte[size]; + input.readFully(bytes); + output.writeInt(size); + output.write(bytes); + output.flush(); + } + } catch (IOException e) { + // ignore + } finally { + try { + socket.close(); + } catch (IOException e) { + // ignore + } + } + } + }; + thread.start(); + threads.add(thread); + } + } catch (IOException e) { + // ignore + } + } + + public void closeConnections() throws IOException { + for (Socket socket : sockets) + socket.close(); + } + + public void close() throws IOException, InterruptedException { + this.serverSocket.close(); + closeConnections(); + for (Thread t : threads) + t.join(); + join(); + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java new file mode 100644 index 0000000..f3c6153 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java @@ -0,0 +1,97 @@ +/** + * 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 static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.util.LinkedHashMap; + +import org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.test.TestSSLUtils; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * A set of tests for the selector over ssl. These use a test harness that runs a simple socket server that echos back responses. + */ + +public class SSLSelectorTest { + + private static final int BUFFER_SIZE = 4 * 1024; + + private EchoServer server; + private Selectable selector; + + @Before + public void setup() throws Exception { + SecurityConfig serverSecurityConfig = TestSSLUtils.createSSLConfigFile(SSLFactory.Mode.SERVER, null); + this.server = new EchoServer(serverSecurityConfig); + this.server.start(); + String trustStoreServer = serverSecurityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG); + SecurityConfig clientSecurityConfig = TestSSLUtils.createSSLConfigFile(SSLFactory.Mode.CLIENT, trustStoreServer); + this.selector = new Selector(new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), clientSecurityConfig); + } + + @After + public void teardown() throws Exception { + this.selector.close(); + this.server.close(); + } + + + /** + * Validate that we can send and receive a message larger than the receive and send buffer size + */ + @Test + public void testSendLargeRequest() throws Exception { + int node = 0; + blockingConnect(node); + String big = TestUtils.randomString(10 * BUFFER_SIZE); + assertEquals(big, blockingRequest(node, big)); + } + + private String blockingRequest(int node, String s) throws IOException { + selector.send(createSend(node, s)); + selector.poll(1000L); + while (true) { + selector.poll(1000L); + for (NetworkReceive receive : selector.completedReceives()) + if (receive.source() == node) + return asString(receive); + } + } + + private String asString(NetworkReceive receive) { + return new String(Utils.toArray(receive.payload())); + } + + private NetworkSend createSend(int node, String s) { + return new NetworkSend(node, ByteBuffer.wrap(s.getBytes())); + } + + /* connect and wait for the connection to complete */ + private void blockingConnect(int node) throws IOException { + selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE); + while (!selector.connected().contains(node)) + selector.poll(10000L); + } + +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index d5b306b..e4100d3 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -15,18 +15,14 @@ package org.apache.kafka.common.network; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.io.DataInputStream; -import java.io.DataOutputStream; import java.io.IOException; import java.net.InetSocketAddress; import java.net.ServerSocket; -import java.net.Socket; import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; import java.util.LinkedHashMap; -import java.util.List; +import org.apache.kafka.clients.ClientUtils; +import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -47,9 +43,10 @@ public class SelectorTest { @Before public void setup() throws Exception { - this.server = new EchoServer(); + SecurityConfig securityConfig = ClientUtils.parseSecurityConfig(""); + this.server = new EchoServer(securityConfig); this.server.start(); - this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap()); + this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap(), securityConfig); } @After @@ -264,71 +261,5 @@ public class SelectorTest { return new String(Utils.toArray(receive.payload())); } - /** - * A simple server that takes size delimited byte arrays and just echos them back to the sender. - */ - static class EchoServer extends Thread { - public final int port; - private final ServerSocket serverSocket; - private final List threads; - private final List sockets; - - public EchoServer() throws Exception { - this.serverSocket = new ServerSocket(0); - this.port = this.serverSocket.getLocalPort(); - this.threads = Collections.synchronizedList(new ArrayList()); - this.sockets = Collections.synchronizedList(new ArrayList()); - } - - public void run() { - try { - while (true) { - final Socket socket = serverSocket.accept(); - sockets.add(socket); - Thread thread = new Thread() { - public void run() { - try { - DataInputStream input = new DataInputStream(socket.getInputStream()); - DataOutputStream output = new DataOutputStream(socket.getOutputStream()); - while (socket.isConnected() && !socket.isClosed()) { - int size = input.readInt(); - byte[] bytes = new byte[size]; - input.readFully(bytes); - output.writeInt(size); - output.write(bytes); - output.flush(); - } - } catch (IOException e) { - // ignore - } finally { - try { - socket.close(); - } catch (IOException e) { - // ignore - } - } - } - }; - thread.start(); - threads.add(thread); - } - } catch (IOException e) { - // ignore - } - } - - public void closeConnections() throws IOException { - for (Socket socket : sockets) - socket.close(); - } - - public void close() throws IOException, InterruptedException { - this.serverSocket.close(); - closeConnections(); - for (Thread t : threads) - t.join(); - join(); - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 2ebe3c2..bf2b5bd 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -100,4 +100,6 @@ public class UtilsTest { buffer = ByteBuffer.wrap(myvar).asReadOnlyBuffer(); this.subTest(buffer); } + + } \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java new file mode 100644 index 0000000..c811096 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java @@ -0,0 +1,208 @@ +/** + * 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.test; + +import org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.common.network.SSLFactory; +import sun.security.x509.AlgorithmId; +import sun.security.x509.CertificateAlgorithmId; +import sun.security.x509.CertificateIssuerName; +import sun.security.x509.CertificateSerialNumber; +import sun.security.x509.CertificateSubjectName; +import sun.security.x509.CertificateValidity; +import sun.security.x509.CertificateVersion; +import sun.security.x509.CertificateX509Key; +import sun.security.x509.X500Name; +import sun.security.x509.X509CertImpl; +import sun.security.x509.X509CertInfo; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.math.BigInteger; +import java.security.GeneralSecurityException; +import java.security.Key; +import java.security.KeyPair; +import java.security.KeyPairGenerator; +import java.security.KeyStore; +import java.security.NoSuchAlgorithmException; +import java.security.PrivateKey; +import java.security.SecureRandom; +import java.security.cert.Certificate; +import java.security.cert.X509Certificate; +import java.util.Date; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + + +public class TestSSLUtils { + + /** + * Create a self-signed X.509 Certificate. + * From http://bfo.com/blog/2011/03/08/odds_and_ends_creating_a_new_x_509_certificate.html. + * + * @param dn the X.509 Distinguished Name, eg "CN=Test, L=London, C=GB" + * @param pair the KeyPair + * @param days how many days from now the Certificate is valid for + * @param algorithm the signing algorithm, eg "SHA1withRSA" + * @return the self-signed certificate + * @throws IOException thrown if an IO error ocurred. + * @throws GeneralSecurityException thrown if an Security error ocurred. + */ + public static X509Certificate generateCertificate(String dn, KeyPair pair, + int days, String algorithm) throws GeneralSecurityException, IOException { + PrivateKey privkey = pair.getPrivate(); + X509CertInfo info = new X509CertInfo(); + Date from = new Date(); + Date to = new Date(from.getTime() + days * 86400000L); + CertificateValidity interval = new CertificateValidity(from, to); + BigInteger sn = new BigInteger(64, new SecureRandom()); + X500Name owner = new X500Name(dn); + + info.set(X509CertInfo.VALIDITY, interval); + info.set(X509CertInfo.SERIAL_NUMBER, new CertificateSerialNumber(sn)); + info.set(X509CertInfo.SUBJECT, new CertificateSubjectName(owner)); + info.set(X509CertInfo.ISSUER, new CertificateIssuerName(owner)); + info.set(X509CertInfo.KEY, new CertificateX509Key(pair.getPublic())); + info + .set(X509CertInfo.VERSION, new CertificateVersion(CertificateVersion.V3)); + AlgorithmId algo = new AlgorithmId(AlgorithmId.md5WithRSAEncryption_oid); + info.set(X509CertInfo.ALGORITHM_ID, new CertificateAlgorithmId(algo)); + + // Sign the cert to identify the algorithm that's used. + X509CertImpl cert = new X509CertImpl(info); + cert.sign(privkey, algorithm); + + // Update the algorith, and resign. + algo = (AlgorithmId) cert.get(X509CertImpl.SIG_ALG); + info + .set(CertificateAlgorithmId.NAME + "." + CertificateAlgorithmId.ALGORITHM, + algo); + cert = new X509CertImpl(info); + cert.sign(privkey, algorithm); + return cert; + } + + public static KeyPair generateKeyPair(String algorithm) throws NoSuchAlgorithmException { + KeyPairGenerator keyGen = KeyPairGenerator.getInstance(algorithm); + keyGen.initialize(1024); + return keyGen.genKeyPair(); + } + + private static KeyStore createEmptyKeyStore() throws GeneralSecurityException, IOException { + KeyStore ks = KeyStore.getInstance("JKS"); + ks.load(null, null); // initialize + return ks; + } + + private static void saveKeyStore(KeyStore ks, String filename, + String password) throws GeneralSecurityException, IOException { + FileOutputStream out = new FileOutputStream(filename); + try { + ks.store(out, password.toCharArray()); + } finally { + out.close(); + } + } + + public static void createKeyStore(String filename, + String password, String alias, + Key privateKey, Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, password.toCharArray(), + new Certificate[]{cert}); + saveKeyStore(ks, filename, password); + } + + /** + * Creates a keystore with a single key and saves it to a file. + * + * @param filename String file to save + * @param password String store password to set on keystore + * @param keyPassword String key password to set on key + * @param alias String alias to use for the key + * @param privateKey Key to save in keystore + * @param cert Certificate to use as certificate chain associated to key + * @throws GeneralSecurityException for any error with the security APIs + * @throws IOException if there is an I/O error saving the file + */ + public static void createKeyStore(String filename, + String password, String keyPassword, String alias, + Key privateKey, Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setKeyEntry(alias, privateKey, keyPassword.toCharArray(), + new Certificate[]{cert}); + saveKeyStore(ks, filename, password); + } + + public static void createTrustStore(String filename, + String password, String alias, + Certificate cert) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + ks.setCertificateEntry(alias, cert); + saveKeyStore(ks, filename, password); + } + + public static void createTrustStore( + String filename, String password, Map certs) throws GeneralSecurityException, IOException { + KeyStore ks = createEmptyKeyStore(); + for (Map.Entry cert : certs.entrySet()) { + ks.setCertificateEntry(cert.getKey(), cert.getValue()); + } + saveKeyStore(ks, filename, password); + } + + public static SecurityConfig createSSLConfigFile(SSLFactory.Mode mode, String trustStoreFileClient) throws IOException, GeneralSecurityException { + Properties securityConfigProps = new Properties(); + Map certs = new HashMap(); + KeyPair keyPair = generateKeyPair("RSA"); + X509Certificate cert = generateCertificate("CN=localhost, O=localhost", keyPair, 30, "SHA1withRSA"); + String password = "test"; + + if (mode == SSLFactory.Mode.SERVER) { + File keyStoreFile = File.createTempFile("keystore", ".jks"); + createKeyStore(keyStoreFile.getPath(), password, password, "localhost", keyPair.getPrivate(), cert); + certs.put("localhost", cert); + securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_LOCATION_CONFIG, keyStoreFile.getPath()); + securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_TYPE_CONFIG, "JKS"); + securityConfigProps.put(SecurityConfig.SSL_KEYMANAGER_ALGORITHM_CONFIG, "SunX509"); + securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_PASSWORD_CONFIG, password); + securityConfigProps.put(SecurityConfig.SSL_KEY_PASSWORD_CONFIG, password); + + File trustStoreFile = File.createTempFile("truststore", ".jks"); + createTrustStore(trustStoreFile.getPath(), password, certs); + + securityConfigProps.put(SecurityConfig.SECURITY_PROTOCOL_CONFIG, "SSL"); + securityConfigProps.put(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG, "false"); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFile.getPath()); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, password); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); + } else { + securityConfigProps.put(SecurityConfig.SECURITY_PROTOCOL_CONFIG, "SSL"); + securityConfigProps.put(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG, "false"); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFileClient); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, password); + securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); + } + + securityConfigProps.put(SecurityConfig.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, "SunX509"); + return new SecurityConfig(securityConfigProps); + } + +} -- 2.3.2 (Apple Git-55) From a0d01983ec02e7db0bbbe7e0747aca174df7d35f Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 10 May 2015 23:02:01 -0700 Subject: [PATCH 2/6] KAFKA-1690. new java producer needs ssl support as a client. --- build.gradle | 13 +- checkstyle/import-control.xml | 39 ++-- .../java/org/apache/kafka/clients/ClientUtils.java | 11 +- .../apache/kafka/clients/CommonClientConfigs.java | 67 ++++++- .../kafka/clients/consumer/ConsumerConfig.java | 28 ++- .../kafka/clients/consumer/KafkaConsumer.java | 99 +++++----- .../kafka/clients/producer/KafkaProducer.java | 61 +++--- .../kafka/clients/producer/ProducerConfig.java | 25 ++- .../apache/kafka/common/config/AbstractConfig.java | 12 +- .../apache/kafka/common/config/SecurityConfig.java | 113 ----------- .../apache/kafka/common/network/Authenticator.java | 2 +- .../org/apache/kafka/common/network/Channel.java | 2 +- .../kafka/common/network/DefaultAuthenticator.java | 2 +- .../common/network/PlainTextTransportLayer.java | 3 +- .../apache/kafka/common/network/SSLFactory.java | 55 +++--- .../kafka/common/network/SSLTransportLayer.java | 217 +++++++++++---------- .../org/apache/kafka/common/network/Selector.java | 33 ++-- .../kafka/common/network/TransportLayer.java | 2 +- .../apache/kafka/common/network/EchoServer.java | 18 +- .../kafka/common/network/SSLSelectorTest.java | 72 ++++++- .../apache/kafka/common/network/SelectorTest.java | 14 +- .../java/org/apache/kafka/test/TestSSLUtils.java | 157 ++++++++------- 22 files changed, 551 insertions(+), 494 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java diff --git a/build.gradle b/build.gradle index fef515b..32faba3 100644 --- a/build.gradle +++ b/build.gradle @@ -4,9 +4,9 @@ // 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. @@ -109,7 +109,7 @@ subprojects { archives srcJar archives javadocJar } - + plugins.withType(ScalaPlugin) { //source jar should also contain scala source: srcJar.from sourceSets.main.scala @@ -250,9 +250,9 @@ project(':core') { into "$buildDir/dependant-libs-${scalaVersion}" } - tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { + tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "kafka_${baseScalaVersion}-${version}" - compression = Compression.GZIP + compression = Compression.GZIP from(project.file("../bin")) { into "bin/" } from(project.file("../config")) { into "config/" } from '../LICENSE' @@ -354,6 +354,7 @@ project(':clients') { compile "org.slf4j:slf4j-api:1.7.6" compile 'org.xerial.snappy:snappy-java:1.1.1.6' compile 'net.jpountz.lz4:lz4:1.2.0' + compile 'org.bouncycastle:bcprov-jdk16:1.46' testCompile 'com.novocode:junit-interface:0.9' testRuntime "$slf4jlog4j" @@ -382,7 +383,7 @@ project(':clients') { artifacts { archives testJar } - + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index e649189..339c620 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -8,19 +8,19 @@ // 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. ---> +--> - + - + @@ -28,11 +28,11 @@ - - + + - + @@ -43,7 +43,7 @@ - + @@ -51,35 +51,35 @@ - + - + - + - + - + - + @@ -92,15 +92,15 @@ - + - + - + @@ -109,6 +109,7 @@ + - + diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 54a554f..3657279 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -3,9 +3,9 @@ * 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. @@ -69,10 +69,9 @@ public class ClientUtils { public static SecurityConfig parseSecurityConfig(String securityConfigFile) throws IOException { Properties securityProps = new Properties(); - if (securityConfigFile == null || securityConfigFile == "") { - return new SecurityConfig(securityProps); + if (securityConfigFile != null && securityConfigFile != "") { + securityProps = loadProps(securityConfigFile); } - securityProps = loadProps(securityConfigFile); return new SecurityConfig(securityProps); } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 0b23875..ead3826 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -3,9 +3,9 @@ * 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. @@ -17,7 +17,7 @@ package org.apache.kafka.clients; * Some configurations shared by both producer and consumer */ public class CommonClientConfigs { - + /* * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE. */ @@ -27,10 +27,10 @@ public class CommonClientConfigs { + "host1:port1,host2:port2,.... Since these servers are just used for the initial connection to " + "discover the full cluster membership (which may change dynamically), this list need not contain the full set of " + "servers (you may want more than one, though, in case a server is down)."; - + public static final String METADATA_MAX_AGE_CONFIG = "metadata.max.age.ms"; public static final String METADATA_MAX_AGE_DOC = "The period of time in milliseconds after which we force a refresh of metadata even if we haven't seen any partition leadership changes to proactively discover any new brokers or partitions."; - + public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; public static final String SEND_BUFFER_DOC = "The size of the TCP send buffer (SO_SNDBUF) to use when sending data."; @@ -45,7 +45,7 @@ public class CommonClientConfigs { public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; - + public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics."; @@ -57,4 +57,57 @@ public class CommonClientConfigs { public static final String SECURITY_CONFIG_FILE_CONFIG = "security.config.file"; public static final String SECURITY_CONFIG_FILE_DOC = "Kafka client security related config file."; -} \ No newline at end of file + + public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol"; + public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT and SSL are supported."; + + public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol"; + public static final String SSL_PROTOCOL_DOC = "The ssl protocol used to generate SSLContext." + + "Default setting is TLS. Allowed values are SSL, SSLv2, SSLv3, TLS, TLSv1.1, TLSv1.2"; + + public static final String SSL_CIPHER_SUITES_CONFIG = "ssl.cipher.suites"; + public static final String SSL_CIPHER_SUITES_DOC = "A cipher suite is a named combination of authentication, encryption, MAC and key exchange algorithm used to negotiate the security settings for a network connection using TLS or SSL network protocol." + + "By default all the available cipher suites are supported."; + + 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. " + + "All versions of TLS is enabled by default."; + + public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; + public static final String SSL_KEYSTORE_TYPE_DOC = "The file format of the key store file. " + + "This is optional for client. Default value is JKS"; + + public static final String SSL_KEYSTORE_LOCATION_CONFIG = "ssl.keystore.location"; + public static final String SSL_KEYSTORE_LOCATION_DOC = "The location of the key store file. " + + "This is optional for Client and can be used for two-way authentication for client."; + + public static final String SSL_KEYSTORE_PASSWORD_CONFIG = "ssl.keystore.password"; + public static final String SSL_KEYSTORE_PASSWORD_DOC = "The store password for the key store file. "; + + public static final String SSL_KEY_PASSWORD_CONFIG = "ssl.key.password"; + public static final String SSL_KEY_PASSWORD_DOC = "The password of the private key in the key store file. " + + "This is optional for client."; + + public static final String SSL_TRUSTSTORE_TYPE_CONFIG = "ssl.truststore.type"; + public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file. " + + "Default value is JKS."; + + public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location"; + public static final String SSL_TRUSTSTORE_LOCATION_DOC = "The location of the trust store file. "; + + public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "ssl.truststore.password"; + public static final String SSL_TRUSTSTORE_PASSWORD_DOC = "The password for the trust store file. "; + + public static final String SSL_CLIENT_REQUIRE_CERT_CONFIG = "ssl.client.require.cert"; + public static final String SSL_CLIENT_REQUIRE_CERT_DOC = "This is to enforce two-way authentication between client and server." + + "Default value is false. If set to true client need to prover Keystrore releated config"; + + public static final String SSL_KEYMANAGER_ALGORITHM_CONFIG = "ssl.keymanager.algorithm"; + public static final String SSL_KEYMANAGER_ALGORITHM_DOC = "The algorithm used by key manager factory for SSL connections. " + + "Default value is the key manager factory algorithm configured for the Java Virtual Machine."; + + public static final String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = "ssl.trustmanager.algorithm"; + public static final String SSL_TRUSTMANAGER_ALGORITHM_DOC = "The algorithm used by trust manager factory for SSL connections. " + + "Default value is the trust manager factory algorithm configured for the Java Virtual Machine."; + +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 190fe63..4ccd423 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -3,9 +3,9 @@ * 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. @@ -19,6 +19,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.protocol.SecurityProtocol; import java.util.HashMap; import java.util.Map; @@ -147,7 +148,7 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String CHECK_CRCS_CONFIG = "check.crcs"; private static final String CHECK_CRCS_DOC = "Automatically check the CRC32 of the records consumed. This ensures no on-the-wire or on-disk corruption to the messages occurred. This check adds some overhead, so it may be disabled in cases seeking extreme performance."; - + /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; @@ -256,7 +257,7 @@ public class ConsumerConfig extends AbstractConfig { Type.BOOLEAN, true, Importance.LOW, - CHECK_CRCS_DOC) + CHECK_CRCS_DOC) .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, Type.LONG, 30000, @@ -282,11 +283,20 @@ public class ConsumerConfig extends AbstractConfig { Type.CLASS, Importance.HIGH, VALUE_DESERIALIZER_CLASS_DOC) - .define(SECURITY_CONFIG_FILE_CONFIG, - Type.STRING, - "", - Importance.MEDIUM, - SECURITY_CONFIG_FILE_DOC); + .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(CommonClientConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, CommonClientConfigs.SSL_PROTOCOL_DOC) + .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC) + .define(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, CommonClientConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_KEYSTORE_TYPE_DOC) + .define(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_LOCATION_DOC, false) + .define(CommonClientConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEY_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_TRUSTSTORE_TYPE_DOC) + .define(CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_DOC, false) + .define(CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, "SunX509", Importance.LOW, CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) + .define(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, "SunX509", Importance.LOW, CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) + .define(CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_DOC); } public static Map addDeserializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 032fd4b..375669f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -3,9 +3,9 @@ * 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. @@ -37,7 +37,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.MetricName; @@ -63,7 +62,7 @@ import org.slf4j.LoggerFactory; *

* The consumer is thread safe but generally will be used only from within a single thread. The consumer client has no * threads of it's own, all work is done in the caller's thread when calls are made on the various methods exposed. - * + * *

Offsets and Consumer Position

* Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer @@ -81,9 +80,9 @@ import org.slf4j.LoggerFactory; *

* This distinction gives the consumer control over when a record is considered consumed. It is discussed in further * detail below. - * + * *

Consumer Groups

- * + * * Kafka uses the concept of consumer groups to allow a pool of processes to divide up the work of consuming and * processing records. These processes can either be running on the same machine or, as is more likely, they can be * distributed over many machines to provide additional scalability and fault tolerance for processing. @@ -112,14 +111,14 @@ import org.slf4j.LoggerFactory; *

* It is also possible for the consumer to manually specify the partitions it subscribes to, which disables this dynamic * partition balancing. - * + * *

Usage Examples

* The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to * demonstrate how to use them. - * + * *

Simple Processing

* This example demonstrates the simplest usage of Kafka's consumer api. - * + * *
  *     Properties props = new Properties();
  *     props.put("bootstrap.servers", "localhost:9092");
@@ -137,7 +136,7 @@ import org.slf4j.LoggerFactory;
  *             System.out.printf("offset = %d, key = %s, value = %s", record.offset(), record.key(), record.value());
  *     }
  * 
- * + * * Setting enable.auto.commit means that offsets are committed automatically with a frequency controlled by * the config auto.commit.interval.ms. *

@@ -157,9 +156,9 @@ import org.slf4j.LoggerFactory; *

* The serializers settings specify how to turn the objects the user provides into bytes. By specifying the string * serializers we are saying that our record's key and value will just be simple strings. - * + * *

Controlling When Messages Are Considered Consumed

- * + * * In this example we will consume a batch of records and batch them up in memory, when we have sufficient records * batched we will insert them into a database. If we allowed offsets to auto commit as in the previous example messages * would be considered consumed after they were given out by the consumer, and it would be possible that our process @@ -171,7 +170,7 @@ import org.slf4j.LoggerFactory; * would consume from last committed offset and would repeat the insert of the last batch of data. Used in this way * Kafka provides what is often called "at-least once delivery" guarantees, as each message will likely be delivered one * time but in failure cases could be duplicated. - * + * *
  *     Properties props = new Properties();
  *     props.put("bootstrap.servers", "localhost:9092");
@@ -197,9 +196,9 @@ import org.slf4j.LoggerFactory;
  *         }
  *     }
  * 
- * + * *

Subscribing To Specific Partitions

- * + * * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple * instances of our program can divided up the work of processing records. @@ -219,7 +218,7 @@ import org.slf4j.LoggerFactory; *

* This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular * partitions: - * + * *

  *     String topic = "foo";
  *     TopicPartition partition0 = new TopicPartition(topic, 0);
@@ -227,15 +226,15 @@ import org.slf4j.LoggerFactory;
  *     consumer.subscribe(partition0);
  *     consumer.subscribe(partition1);
  * 
- * + * * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made. *

* It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load * balancing) using the same consumer instance. - * + * *

Managing Your Own Offsets

- * + * * The consumer application need not use Kafka's built-in offset storage, it can store offsets in a store of it's own * choosing. The primary use case for this is allowing the application to store both the offset and the results of the * consumption in the same system in a way that both the results and offsets are stored atomically. This is not always @@ -255,14 +254,14 @@ import org.slf4j.LoggerFactory; * This means that in this case the indexing process that comes back having lost recent updates just resumes indexing * from what it has ensuring that no updates are lost. * - * + * * Each record comes with it's own offset, so to manage your own offset you just need to do the following: *
    *
  1. Configure enable.auto.commit=false *
  2. Use the offset provided with each {@link ConsumerRecord} to save your position. *
  3. On restart restore the position of the consumer using {@link #seek(TopicPartition, long)}. *
- * + * * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the * search index use case described above). If the partition assignment is done automatically special care will also be * needed to handle the case where partition assignments change. This can be handled using a special callback specified @@ -275,9 +274,9 @@ import org.slf4j.LoggerFactory; *

* Another common use for {@link ConsumerRebalanceCallback} is to flush any caches the application maintains for * partitions that are moved elsewhere. - * + * *

Controlling The Consumer's Position

- * + * * In most use cases the consumer will simply consume records from beginning to end, periodically committing it's * position (either automatically or manually). However Kafka allows the consumer to manually control it's position, * moving forward or backwards in a partition at will. This means a consumer can re-consume older records, or skip to @@ -292,20 +291,20 @@ import org.slf4j.LoggerFactory; * the consumer will want to initialize it's position on start-up to whatever is contained in the local store. Likewise * if the local state is destroyed (say because the disk is lost) the state may be recreated on a new machine by * reconsuming all the data and recreating the state (assuming that Kafka is retaining sufficient history). - * + * * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special * methods for seeking to the earliest and latest offset the server maintains are also available ( * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively). - * + * *

Multithreaded Processing

- * + * * The Kafka consumer is threadsafe but coarsely synchronized. All network I/O happens in the thread of the application * making the call. We have intentionally avoided implementing a particular threading model for processing. *

* This leaves several options for implementing multi-threaded processing of records. - * + * *

1. One Consumer Per Thread

- * + * * A simple option is to give each thread it's own consumer instance. Here are the pros and cons of this approach: *
    *
  • PRO: It is the easiest to implement @@ -318,13 +317,13 @@ import org.slf4j.LoggerFactory; * which can cause some drop in I/O throughput. *
  • CON: The number of total threads across all processes will be limited by the total number of partitions. *
- * + * *

2. Decouple Consumption and Processing

- * + * * Another alternative is to have one or more consumer threads that do all data consumption and hands off * {@link ConsumerRecords} instances to a blocking queue consumed by a pool of processor threads that actually handle * the record processing. - * + * * This option likewise has pros and cons: *
    *
  • PRO: This option allows independently scaling the number of consumers and processors. This makes it @@ -335,11 +334,11 @@ import org.slf4j.LoggerFactory; *
  • CON: Manually committing the position becomes harder as it requires that all threads co-ordinate to ensure * that processing is complete for that partition. *
- * + * * There are many possible variations on this approach. For example each processor thread can have it's own queue, and * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify * commit. - * + * */ public class KafkaConsumer implements Consumer { @@ -362,7 +361,6 @@ public class KafkaConsumer implements Consumer { private final boolean autoCommit; private final long autoCommitIntervalMs; private final ConsumerRebalanceCallback rebalanceCallback; - private final SecurityConfig securityConfig; private long lastCommitAttemptMs; private boolean closed = false; @@ -373,7 +371,7 @@ public class KafkaConsumer implements Consumer { * string "42" or the integer 42). *

* Valid configuration strings are documented at {@link ConsumerConfig} - * + * * @param configs The consumer configs */ public KafkaConsumer(Map configs) { @@ -385,7 +383,7 @@ public class KafkaConsumer implements Consumer { * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. *

* Valid configuration strings are documented at {@link ConsumerConfig} - * + * * @param configs The consumer configs * @param callback A callback interface that the user can implement to manage customized offsets on the start and * end of every rebalance operation. @@ -419,7 +417,7 @@ public class KafkaConsumer implements Consumer { * {@link ConsumerRebalanceCallback} implementation, a key and a value {@link Deserializer}. *

* Valid configuration strings are documented at {@link ConsumerConfig} - * + * * @param properties The consumer configuration properties * @param callback A callback interface that the user can implement to manage customized offsets on the start and * end of every rebalance operation. @@ -470,12 +468,11 @@ public class KafkaConsumer implements Consumer { this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG)); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); this.metadata.update(Cluster.bootstrap(addresses), 0); - String metricGrpPrefix = "consumer"; Map metricsTags = new LinkedHashMap(); metricsTags.put("client-id", clientId); - this.securityConfig = ClientUtils.parseSecurityConfig(config.getString(ConsumerConfig.SECURITY_CONFIG_FILE_CONFIG)); - this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags, securityConfig), + + this.client = new NetworkClient(new Selector(metrics, time, metricGrpPrefix, metricsTags, config.values()), this.metadata, clientId, 100, // a fixed large enough value will suffice @@ -559,7 +556,7 @@ public class KafkaConsumer implements Consumer { *

  • An existing member of the consumer group dies *
  • A new member is added to an existing consumer group via the join API * - * + * * @param topics A variable list of topics that the consumer wants to subscribe to */ @Override @@ -576,7 +573,7 @@ public class KafkaConsumer implements Consumer { * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic * metadata change. *

    - * + * * @param partitions Partitions to incrementally subscribe to */ @Override @@ -592,7 +589,7 @@ public class KafkaConsumer implements Consumer { /** * Unsubscribe from the specific topics. This will trigger a rebalance operation and records for this topic will not * be returned from the next {@link #poll(long) poll()} onwards - * + * * @param topics Topics to unsubscribe from */ public synchronized void unsubscribe(String... topics) { @@ -606,7 +603,7 @@ public class KafkaConsumer implements Consumer { /** * Unsubscribe from the specific topic partitions. records for these partitions will not be returned from the next * {@link #poll(long) poll()} onwards - * + * * @param partitions Partitions to unsubscribe from */ public synchronized void unsubscribe(TopicPartition... partitions) { @@ -625,11 +622,11 @@ public class KafkaConsumer implements Consumer { * If {@link #seek(TopicPartition, long)} is used, it will use the specified offsets on startup and on every * rebalance, to consume data from that offset sequentially on every poll. If not, it will use the last checkpointed * offset using {@link #commit(Map, CommitType) commit(offsets, sync)} for the subscribed list of partitions. - * + * * @param timeout The time, in milliseconds, spent waiting in poll if data is not available. If 0, waits * indefinitely. Must not be negative * @return map of topic to records since the last fetch for the subscribed list of topics and partitions - * + * * @throws NoOffsetForPartitionException If there is no stored offset for a subscribed partition and no automatic * offset reset policy has been configured. */ @@ -683,7 +680,7 @@ public class KafkaConsumer implements Consumer { * A non-blocking commit will attempt to commit offsets asychronously. No error will be thrown if the commit fails. * A blocking commit will wait for a response acknowledging the commit. In the event of an error it will retry until * the commit succeeds. - * + * * @param offsets The list of offsets per partition that should be committed to Kafka. * @param commitType Control whether the commit is blocking */ @@ -708,7 +705,7 @@ public class KafkaConsumer implements Consumer { * This commits offsets only to Kafka. The offsets committed using this API will be used on the first fetch after * every rebalance and also on startup. As such, if you need to store offsets in anything other than Kafka, this API * should not be used. - * + * * @param commitType Whether or not the commit should block until it is acknowledged. */ @Override @@ -757,7 +754,7 @@ public class KafkaConsumer implements Consumer { /** * Returns the offset of the next record that will be fetched (if a record with that offset exists). - * + * * @param partition The partition to get the position for * @return The offset * @throws NoOffsetForPartitionException If a position hasn't been set for a given partition, and no reset policy is @@ -782,7 +779,7 @@ public class KafkaConsumer implements Consumer { *

    * This call may block to do a remote call if the partition in question isn't assigned to this consumer or if the * consumer hasn't yet initialized it's cache of committed offsets. - * + * * @param partition The partition to check * @return The last committed offset or null if no offset has been committed * @throws NoOffsetForPartitionException If no offset has ever been committed by any process for the given @@ -818,7 +815,7 @@ public class KafkaConsumer implements Consumer { /** * Get metadata about the partitions for a given topic. This method will issue a remote call to the server if it * does not already have any metadata about the given topic. - * + * * @param topic The topic to get partition metadata for * @return The list of partitions */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 2c21dee..1650d85 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -3,9 +3,9 @@ * 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. @@ -32,7 +32,6 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.common.errors.InterruptException; import org.apache.kafka.common.errors.RecordTooLargeException; @@ -74,11 +73,11 @@ import org.slf4j.LoggerFactory; * props.put("buffer.memory", 33554432); * props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); * props.put("value.serializer", "org.apache.kafka.common.serialization.StringSerializer"); - * + * * Producer producer = new KafkaProducer(props); * for(int i = 0; i < 100; i++) * producer.send(new ProducerRecord("my-topic", Integer.toString(i), Integer.toString(i))); - * + * * producer.close(); * } *

    @@ -93,25 +92,25 @@ import org.slf4j.LoggerFactory; * we have specified will result in blocking on the full commit of the record, the slowest but most durable setting. *

    * If the request fails, the producer can automatically retry, though since we have specified retries - * as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on + * as 0 it won't. Enabling retries also opens up the possibility of duplicates (see the documentation on * message delivery semantics for details). *

    - * The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by + * The producer maintains buffers of unsent records for each partition. These buffers are of a size specified by * the batch.size config. Making this larger can result in more batching, but requires more memory (since we will * generally have one of these buffers for each active partition). *

    - * By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you + * By default a buffer is available to send immediately even if there is additional unused space in the buffer. However if you * want to reduce the number of requests you can set linger.ms to something greater than 0. This will - * instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will - * arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above, - * likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting - * would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that - * records that arrive close together in time will generally batch together even with linger.ms=0 so under heavy load + * instruct the producer to wait up to that number of milliseconds before sending a request in hope that more records will + * arrive to fill up the same batch. This is analogous to Nagle's algorithm in TCP. For example, in the code snippet above, + * likely all 100 records would be sent in a single request since we set our linger time to 1 millisecond. However this setting + * would add 1 millisecond of latency to our request waiting for more records to arrive if we didn't fill up the buffer. Note that + * records that arrive close together in time will generally batch together even with linger.ms=0 so under heavy load * batching will occur regardless of the linger configuration; however setting this to something larger than 0 can lead to fewer, more * efficient requests when not under maximal load at the cost of a small amount of latency. *

    * The buffer.memory controls the total amount of memory available to the producer for buffering. If records - * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is + * are sent faster than they can be transmitted to the server then this buffer space will be exhausted. When the buffer space is * exhausted additional send calls will block. For uses where you want to avoid any blocking you can set block.on.buffer.full=false which * will cause the send call to result in an exception. *

    @@ -139,7 +138,6 @@ public class KafkaProducer implements Producer { private final Serializer keySerializer; private final Serializer valueSerializer; private final ProducerConfig producerConfig; - private final SecurityConfig securityConfig; /** * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings @@ -216,7 +214,6 @@ public class KafkaProducer implements Producer { this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG); this.totalMemorySize = config.getLong(ProducerConfig.BUFFER_MEMORY_CONFIG); this.compressionType = CompressionType.forName(config.getString(ProducerConfig.COMPRESSION_TYPE_CONFIG)); - this.securityConfig = ClientUtils.parseSecurityConfig(config.getString(ProducerConfig.SECURITY_CONFIG_FILE_CONFIG)); Map metricTags = new LinkedHashMap(); metricTags.put("client-id", clientId); this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.BATCH_SIZE_CONFIG), @@ -231,7 +228,7 @@ public class KafkaProducer implements Producer { List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); - NetworkClient client = new NetworkClient(new Selector(this.metrics, time, "producer", metricTags, securityConfig), + NetworkClient client = new NetworkClient(new Selector(this.metrics, time, "producer", metricTags, config.values()), this.metadata, clientId, config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION), @@ -288,7 +285,7 @@ public class KafkaProducer implements Producer { } /** - * Asynchronously send a record to a topic. Equivalent to send(record, null). + * Asynchronously send a record to a topic. Equivalent to send(record, null). * See {@link #send(ProducerRecord, Callback)} for details. */ @Override @@ -308,11 +305,11 @@ public class KafkaProducer implements Producer { *

    * Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the * {@link RecordMetadata} that will be assigned to this record. Invoking {@link java.util.concurrent.Future#get() - * get()} on this future will block until the associated request completes and then return the metadata for the record + * get()} on this future will block until the associated request completes and then return the metadata for the record * or throw any exception that occurred while sending the record. *

    * If you want to simulate a simple blocking call you can call the get() method immediately: - * + * *

          * {@code
          * byte[] key = "key".getBytes();
    @@ -323,7 +320,7 @@ public class KafkaProducer implements Producer {
          * 

    * Fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that * will be invoked when the request is complete. - * + * *

          * {@code
          * ProducerRecord record = new ProducerRecord("the-topic", key, value);
    @@ -337,10 +334,10 @@ public class KafkaProducer implements Producer {
          *               });
          * }
          * 
    - * + * * Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the * following example callback1 is guaranteed to execute before callback2: - * + * *
          * {@code
          * producer.send(new ProducerRecord(topic, partition, key1, value1), callback1);
    @@ -352,15 +349,15 @@ public class KafkaProducer implements Producer {
          * they will delay the sending of messages from other threads. If you want to execute blocking or computationally
          * expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body
          * to parallelize processing.
    -     * 
    +     *
          * @param record The record to send
          * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
          *        indicates no callback)
    -     *        
    +     *
          * @throws InterruptException If the thread is interrupted while blocked
          * @throws SerializationException If the key or value are not valid objects given the configured serializers
          * @throws BufferExhaustedException If block.on.buffer.full=false and the buffer is full.
    -     * 
    +     *
          */
         @Override
         public Future send(ProducerRecord record, Callback callback) {
    @@ -455,12 +452,12 @@ public class KafkaProducer implements Producer {
                                                   ProducerConfig.BUFFER_MEMORY_CONFIG +
                                                   " configuration.");
         }
    -    
    +
         /**
    -     * Invoking this method makes all buffered records immediately available to send (even if linger.ms is 
    +     * Invoking this method makes all buffered records immediately available to send (even if linger.ms is
          * greater than 0) and blocks on the completion of the requests associated with these records. The post-condition
    -     * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true). 
    -     * A request is considered completed when it is successfully acknowledged 
    +     * of flush() is that any previously sent record will have completed (e.g. Future.isDone() == true).
    +     * A request is considered completed when it is successfully acknowledged
          * according to the acks configuration you have specified or else it results in an error.
          * 

    * Other threads can continue sending records while one thread is blocked waiting for a flush call to complete, @@ -478,10 +475,10 @@ public class KafkaProducer implements Producer { * consumer.commit(); * } *

    - * + * * Note that the above example may drop records if the produce request fails. If we want to ensure that this does not occur * we need to set retries=<large_number> in our config. - * + * * @throws InterruptException If the thread is interrupted while blocked */ @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 83506e7..69563ee 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -3,9 +3,9 @@ * 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. @@ -26,6 +26,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.protocol.SecurityProtocol; /** * Configuration for the Kafka Producer. Documentation for these configurations can be found in the metadata.max.age.ms */ public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; - + /** batch.size */ public static final String BATCH_SIZE_CONFIG = "batch.size"; private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " @@ -169,9 +170,6 @@ public class ProducerConfig extends AbstractConfig { public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; - /** security.config.file */ - public static final String SECURITY_CONFIG_FILE_CONFIG = CommonClientConfigs.SECURITY_CONFIG_FILE_CONFIG; - private static final String SECURITY_CONFIG_FILE_DOC = CommonClientConfigs.SECURITY_CONFIG_FILE_DOC; static { @@ -223,7 +221,20 @@ public class ProducerConfig extends AbstractConfig { MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION_DOC) .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, KEY_SERIALIZER_CLASS_DOC) .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) - .define(SECURITY_CONFIG_FILE_CONFIG, Type.STRING, "", Importance.MEDIUM, SECURITY_CONFIG_FILE_DOC); + .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) + .define(CommonClientConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, CommonClientConfigs.SSL_PROTOCOL_DOC) + .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC) + .define(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, CommonClientConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_KEYSTORE_TYPE_DOC) + .define(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_LOCATION_DOC, false) + .define(CommonClientConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEY_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_TRUSTSTORE_TYPE_DOC) + .define(CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_DOC, false) + .define(CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_DOC, false) + .define(CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, "SunX509", Importance.LOW, CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_DOC) + .define(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, "SunX509", Importance.LOW, CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC) + .define(CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_DOC); } public static Map addSerializerToConfig(Map configs, diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index c4fa058..a8f15bc 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -3,9 +3,9 @@ * 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. @@ -98,6 +98,12 @@ public class AbstractConfig { return copy; } + public Map values() { + Map copy = new HashMap(); + copy.putAll(values); + return values; + } + private void logAll() { StringBuilder b = new StringBuilder(); b.append(getClass().getSimpleName()); @@ -124,7 +130,7 @@ public class AbstractConfig { /** * Get a configured instance of the give class specified by the given configuration key. If the object implements * Configurable configure it using the configuration. - * + * * @param key The configuration key for the class * @param t The interface the class should implement * @return A configured instance of the class diff --git a/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java b/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java deleted file mode 100644 index 7954a7e..0000000 --- a/clients/src/main/java/org/apache/kafka/common/config/SecurityConfig.java +++ /dev/null @@ -1,113 +0,0 @@ -/** - * 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.config; - - -import java.util.Map; - -import org.apache.kafka.common.config.ConfigDef.Importance; -import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.common.protocol.SecurityProtocol; - - -/** - * Security Related config for clients and server. - */ - -public class SecurityConfig extends AbstractConfig { - /* - * NOTE: DO NOT CHANGE EITHER CONFIG STRINGS OR THEIR JAVA VARIABLE NAMES AS THESE ARE PART OF THE PUBLIC API AND - * CHANGE WILL BREAK USER CODE. - */ - - private static final ConfigDef CONFIG; - - public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol"; - public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Currently only PLAINTEXT and SSL are supported."; - - public static final String SSL_PROTOCOL_CONFIG = "ssl.protocol"; - public static final String SSL_PROTOCOL_DOC = "The TLS protocol used for broker connections if security protocol is SSL. " - + "Any version of TLS is accepted by default."; - - public static final String SSL_CIPHER_SUITES_CONFIG = "ssl.cipher.suites"; - public static final String SSL_CIPHER_SUITES_DOC = "The list of cipher suites enabled for SSL connections. " - + "Default value is the list of cipher suites enabled for the Java Virtual Machine."; - - 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. " - + "Default value is the list of protocols enabled for the Java Virtual Machine."; - - - public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; - public static final String SSL_KEYSTORE_TYPE_DOC = "The file format of the key store file. " - + "Default value is the default key store format of the Java Virtual Machine."; - - public static final String SSL_KEYSTORE_LOCATION_CONFIG = "ssl.keystore.location"; - public static final String SSL_KEYSTORE_LOCATION_DOC = "The location of the key store file. " - + "This is optional for Client and can be used for two-way authentication for client."; - - public static final String SSL_KEYSTORE_PASSWORD_CONFIG = "ssl.keystore.password"; - public static final String SSL_KEYSTORE_PASSWORD_DOC = "The store password for the key store file. "; - - - public static final String SSL_KEY_PASSWORD_CONFIG = "ssl.key.password"; - public static final String SSL_KEY_PASSWORD_DOC = "The password of the private key in the key store file. " - + "This is optional for client."; - - public static final String SSL_TRUSTSTORE_TYPE_CONFIG = "ssl.truststore.type"; - public static final String SSL_TRUSTSTORE_TYPE_DOC = "The file format of the trust store file. " - + "Default value is JKS."; - - public static final String SSL_TRUSTSTORE_LOCATION_CONFIG = "ssl.truststore.location"; - public static final String SSL_TRUSTSTORE_LOCATION_DOC = "The location of the trust store file. "; - - public static final String SSL_TRUSTSTORE_PASSWORD_CONFIG = "ssl.truststore.password"; - public static final String SSL_TRUSTSTORE_PASSWORD_DOC = "The password for the trust store file. "; - - public static final String SSL_CLIENT_REQUIRE_CERT_CONFIG = "ssl.client.require.cert"; - public static final String SSL_CLIENT_REQUIRE_CERT_DOC = "This is to enforce two-way authentication between client and server." - + "Default value is false. If set to true client need to prover Keystrore releated config"; - - public static final String SSL_KEYMANAGER_ALGORITHM_CONFIG = "ssl.keymanager.algorithm"; - public static final String SSL_KEYMANAGER_ALGORITHM_DOC = "The algorithm used by key manager factory for SSL connections. " - + "Default value is the key manager factory algorithm configured for the Java Virtual Machine."; - - public static final String SSL_TRUSTMANAGER_ALGORITHM_CONFIG = "ssl.trustmanager.algorithm"; - public static final String SSL_TRUSTMANAGER_ALGORITHM_DOC = "The algorithm used by trust manager factory for SSL connections. " - + "Default value is the trust manager factory algorithm configured for the Java Virtual Machine."; - - - static { - CONFIG = new ConfigDef().define(SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, SECURITY_PROTOCOL_DOC) - .define(SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, SSL_PROTOCOL_DOC) - .define(SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.MEDIUM, SSL_CIPHER_SUITES_DOC, false) - .define(SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, SSL_ENABLED_PROTOCOLS_DOC) - .define(SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_TYPE_DOC, false) - .define(SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_LOCATION_DOC, false) - .define(SSL_KEYSTORE_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYSTORE_PASSWORD_DOC, false) - .define(SSL_KEY_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEY_PASSWORD_DOC, false) - .define(SSL_TRUSTSTORE_TYPE_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_TYPE_DOC, false) - .define(SSL_TRUSTSTORE_LOCATION_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_LOCATION_DOC, false) - .define(SSL_TRUSTSTORE_PASSWORD_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTSTORE_PASSWORD_DOC, false) - .define(SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, Importance.MEDIUM, SSL_KEYMANAGER_ALGORITHM_DOC, false) - .define(SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, Importance.MEDIUM, SSL_TRUSTMANAGER_ALGORITHM_DOC, false) - .define(SSL_CLIENT_REQUIRE_CERT_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, SSL_CLIENT_REQUIRE_CERT_DOC); - } - - public SecurityConfig(Map props) { - super(CONFIG, props); - } - - -} diff --git a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java index ee8516f..1b6b32a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java @@ -43,7 +43,7 @@ public interface Authenticator { /** * Returns UserPrincipal after authentication is established */ - UserPrincipal userPrincipal(); + UserPrincipal userPrincipal() throws IOException; /** diff --git a/clients/src/main/java/org/apache/kafka/common/network/Channel.java b/clients/src/main/java/org/apache/kafka/common/network/Channel.java index 3526ba3..d9d1192 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Channel.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Channel.java @@ -58,7 +58,7 @@ public class Channel implements ScatteringByteChannel, GatheringByteChannel { * Incase of PLAINTEXT and No Authentication returns ANONYMOUS as the userPrincipal * If SSL used without any SASL Authentication returns SSLSession.peerPrincipal */ - public UserPrincipal userPrincipal() { + public UserPrincipal userPrincipal() throws IOException { return authenticator.userPrincipal(); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java index c1ec794..97b1135 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java @@ -34,7 +34,7 @@ public class DefaultAuthenticator implements Authenticator { return 0; } - public UserPrincipal userPrincipal() { + public UserPrincipal userPrincipal() throws IOException { return new UserPrincipal(transportLayer.getPeerPrincipal().toString()); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java index 11cd80c..dbf0a30 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java @@ -42,7 +42,6 @@ public class PlainTextTransportLayer implements TransportLayer { public PlainTextTransportLayer(SocketChannel socketChannel) throws IOException { this.socketChannel = socketChannel; - } @@ -141,7 +140,7 @@ public class PlainTextTransportLayer implements TransportLayer { return outStream; } - public Principal getPeerPrincipal() { + public Principal getPeerPrincipal() throws IOException { return new UserPrincipal("ANONYMOUS"); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java index 9cf9051..73f976b 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.network; +import java.util.Map; +import java.util.List; import java.io.FileInputStream; import java.io.IOException; import java.security.GeneralSecurityException; @@ -24,10 +26,11 @@ import java.security.KeyStore; import javax.net.ssl.*; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.clients.CommonClientConfigs; -public class SSLFactory { +public class SSLFactory implements Configurable { public enum Mode { CLIENT, SERVER }; private String protocol; @@ -48,27 +51,31 @@ public class SSLFactory { this.mode = mode; } - - public void init(SecurityConfig securityConfig) throws IOException, GeneralSecurityException { - this.protocol = securityConfig.getString(SecurityConfig.SSL_PROTOCOL_CONFIG); - if (securityConfig.getList(SecurityConfig.SSL_CIPHER_SUITES_CONFIG) != null) - this.cipherSuites = (String[]) securityConfig.getList(SecurityConfig.SSL_CIPHER_SUITES_CONFIG).toArray(); - if (securityConfig.getList(SecurityConfig.SSL_ENABLED_PROTOCOLS_CONFIG) != null) - this.enabledProtocols = (String[]) securityConfig.getList(SecurityConfig.SSL_ENABLED_PROTOCOLS_CONFIG).toArray(); - this.requireClientCert = securityConfig.getBoolean(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG); - this.kmfAlgorithm = securityConfig.getString(SecurityConfig.SSL_KEYMANAGER_ALGORITHM_CONFIG); - this.tmfAlgorithm = securityConfig.getString(SecurityConfig.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); - if ((mode == Mode.CLIENT && requireClientCert) || (mode == Mode.SERVER)) - createKeystore(securityConfig.getString(SecurityConfig.SSL_KEYSTORE_TYPE_CONFIG), - securityConfig.getString(SecurityConfig.SSL_KEYSTORE_LOCATION_CONFIG), - securityConfig.getString(SecurityConfig.SSL_KEYSTORE_PASSWORD_CONFIG), - securityConfig.getString(SecurityConfig.SSL_KEY_PASSWORD_CONFIG)); - createTruststore(securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG), - securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG), - securityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG)); - - this.sslContext = createSSLContext(); - + @Override + public void configure(Map configs) throws KafkaException { + this.protocol = (String) configs.get(CommonClientConfigs.SSL_PROTOCOL_CONFIG); + if (configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG) != null) + this.cipherSuites = (String[]) ((List) configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG)).toArray(); + if (configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG) != null) + this.enabledProtocols = (String[]) ((List) configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG)).toArray(); + this.requireClientCert = (Boolean) configs.get(CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_CONFIG); + this.kmfAlgorithm = (String) configs.get(CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); + this.tmfAlgorithm = (String) configs.get(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); + if ((mode == Mode.CLIENT && requireClientCert) || (mode == Mode.SERVER)) { + createKeystore((String) configs.get(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG), + (String) configs.get(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG), + (String) configs.get(CommonClientConfigs.SSL_KEYSTORE_PASSWORD_CONFIG), + (String) configs.get(CommonClientConfigs.SSL_KEY_PASSWORD_CONFIG)); + } + createTruststore((String) configs.get(CommonClientConfigs.SSL_TRUSTSTORE_TYPE_CONFIG), + (String) configs.get(CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), + (String) configs.get(CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + try { + this.sslContext = createSSLContext(); + } catch (Exception e) { + e.printStackTrace(); + throw new KafkaException(e); + } } @@ -183,6 +190,4 @@ public class SSLFactory { } } - public void close() { - } } diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java index dc84975..2d6a519 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java @@ -27,9 +27,12 @@ import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLEngineResult; import javax.net.ssl.SSLEngineResult.HandshakeStatus; import javax.net.ssl.SSLEngineResult.Status; +import javax.net.ssl.SSLException; +import javax.net.ssl.SSLPeerUnverifiedException; import java.io.DataInputStream; import java.io.DataOutputStream; +import java.util.concurrent.ExecutorService; import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; @@ -41,25 +44,27 @@ import org.slf4j.LoggerFactory; public class SSLTransportLayer implements TransportLayer { private static final Logger log = LoggerFactory.getLogger(SSLTransportLayer.class); - SocketChannel socketChannel; - SSLEngine sslEngine; - HandshakeStatus handshakeStatus = null; - SSLEngineResult handshakeResult = null; - boolean handshakeComplete = false; - boolean closed = false; - boolean closing = false; - ByteBuffer netInBuffer = null; - ByteBuffer netOutBuffer = null; - ByteBuffer appReadBuffer = null; - ByteBuffer appWriteBuffer = null; - ByteBuffer emptyBuf = ByteBuffer.allocate(0); - DataInputStream inStream = null; - DataOutputStream outStream = null; - - - public SSLTransportLayer(SocketChannel socketChannel, SSLEngine sslEngine) throws IOException { + protected SSLEngine sslEngine; + + private SocketChannel socketChannel; + private HandshakeStatus handshakeStatus; + private SSLEngineResult handshakeResult; + private boolean handshakeComplete = false; + private boolean closed = false; + private boolean closing = false; + private ByteBuffer netInBuffer; + private ByteBuffer netOutBuffer; + private ByteBuffer appReadBuffer; + private ByteBuffer appWriteBuffer; + private ByteBuffer emptyBuf = ByteBuffer.allocate(0); + private DataInputStream inStream; + private DataOutputStream outStream; + private ExecutorService executorService; + + public SSLTransportLayer(SocketChannel socketChannel, SSLEngine sslEngine, ExecutorService executorService) throws IOException { this.socketChannel = socketChannel; this.sslEngine = sslEngine; + this.executorService = executorService; this.netInBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getPacketBufferSize()); this.netOutBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getPacketBufferSize()); this.appWriteBuffer = ByteBuffer.allocateDirect(sslEngine.getSession().getApplicationBufferSize()); @@ -116,56 +121,60 @@ public class SSLTransportLayer implements TransportLayer { if (handshakeComplete) return 0; //we have done our initial handshake if (!flush(netOutBuffer)) return SelectionKey.OP_WRITE; - - switch(handshakeStatus) { - case NOT_HANDSHAKING: - // SSLEnginge.getHandshakeStatus is transient and it doesn't record FINISHED status properly - if (handshakeResult.getHandshakeStatus() == HandshakeStatus.FINISHED) { + try { + switch(handshakeStatus) { + case NOT_HANDSHAKING: + // SSLEnginge.getHandshakeStatus is transient and it doesn't record FINISHED status properly + if (handshakeResult.getHandshakeStatus() == HandshakeStatus.FINISHED) { + handshakeComplete = !netOutBuffer.hasRemaining(); + if (handshakeComplete) + return 0; + else + return SelectionKey.OP_WRITE; + } else { + throw new IOException("NOT_HANDSHAKING during handshake"); + } + case FINISHED: + //we are complete if we have delivered the last package handshakeComplete = !netOutBuffer.hasRemaining(); - if (handshakeComplete) - return 0; - else + //return 0 if we are complete, otherwise we still have data to write + if (handshakeComplete) return 0; + else return SelectionKey.OP_WRITE; + case NEED_WRAP: + handshakeResult = handshakeWrap(write); + if (handshakeResult.getStatus() == Status.OK) { + if (handshakeStatus == HandshakeStatus.NEED_TASK) + handshakeStatus = tasks(); + } else { + //wrap should always work with our buffers + throw new IOException("Unexpected status [" + handshakeResult.getStatus() + "] during handshake WRAP."); + } + if (handshakeStatus != HandshakeStatus.NEED_UNWRAP || (!flush(netOutBuffer))) return SelectionKey.OP_WRITE; - } else { - //should never happen - throw new IOException("NOT_HANDSHAKING during handshake"); - } - case FINISHED: - //we are complete if we have delivered the last package - handshakeComplete = !netOutBuffer.hasRemaining(); - //return 0 if we are complete, otherwise we still have data to write - if (handshakeComplete) return 0; - else return SelectionKey.OP_WRITE; - case NEED_WRAP: - handshakeResult = handshakeWrap(write); - if (handshakeResult.getStatus() == Status.OK) { - if (handshakeStatus == HandshakeStatus.NEED_TASK) - handshakeStatus = tasks(); - } else { - //wrap should always work with our buffers - throw new IOException("Unexpected status [" + handshakeResult.getStatus() + "] during handshake WRAP."); - } - if (handshakeStatus != HandshakeStatus.NEED_UNWRAP || (!flush(netOutBuffer))) - return SelectionKey.OP_WRITE; - //fall down to NEED_UNWRAP on the same call, will result in a - //BUFFER_UNDERFLOW if it needs data - case NEED_UNWRAP: - handshakeResult = handshakeUnwrap(read); - if (handshakeResult.getStatus() == Status.OK) { - if (handshakeStatus == HandshakeStatus.NEED_TASK) - handshakeStatus = tasks(); - } else if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { - return SelectionKey.OP_READ; - } else { - throw new IOException(String.format("Unexpected status [%s] during handshake UNWRAP", handshakeStatus)); - } - break; - case NEED_TASK: - handshakeStatus = tasks(); - break; - default: - throw new IllegalStateException(String.format("Unexpected status [%s]", handshakeStatus)); + //fall down to NEED_UNWRAP on the same call, will result in a + //BUFFER_UNDERFLOW if it needs data + case NEED_UNWRAP: + handshakeResult = handshakeUnwrap(read); + if (handshakeResult.getStatus() == Status.OK) { + if (handshakeStatus == HandshakeStatus.NEED_TASK) + handshakeStatus = tasks(); + } else if (handshakeResult.getStatus() == Status.BUFFER_UNDERFLOW) { + return SelectionKey.OP_READ; + } else { + throw new IOException(String.format("Unexpected status [%s] during handshake UNWRAP", handshakeStatus)); + } + break; + case NEED_TASK: + handshakeStatus = tasks(); + break; + default: + throw new IllegalStateException(String.format("Unexpected status [%s]", handshakeStatus)); + } + } catch (SSLException e) { + handshakeFailure(); + throw e; } + //return 0 if we are complete, otherwise re-register for any activity that //would cause this method to be called again. if (handshakeComplete) return 0; @@ -173,12 +182,22 @@ public class SSLTransportLayer implements TransportLayer { } /** - * Executes all the tasks needed on the same thread. + * Executes all the tasks needed on the executorservice thread. * @return HandshakeStatus */ private HandshakeStatus tasks() { - Runnable r = null; - while ((r = sslEngine.getDelegatedTask()) != null) r.run(); + for (;;) { + final Runnable task = sslEngine.getDelegatedTask(); + if (task == null) + break; + + executorService.submit(new Runnable() { + @Override + public void run() { + task.run(); + } + }); + } return sslEngine.getHandshakeStatus(); } @@ -237,10 +256,6 @@ public class SSLTransportLayer implements TransportLayer { } - public int getOutboundRemaining() { - return netOutBuffer.remaining(); - } - /** * Sends a SSL close message, will not physically close the connection here.
    * @throws IOException if an I/O error occurs @@ -283,7 +298,7 @@ public class SSLTransportLayer implements TransportLayer { * @param dst The buffer into which bytes are to be transferred * @return The number of bytes read, possible zero or -1 if the channel has reached end-of-stream * @throws IOException if some other I/O error occurs - * @throws IllegalStateException if the destination buffer is different than appBufHandler.getReadBuffer() + * @throws IllegalStateException if handshake is not complete. */ public int read(ByteBuffer dst) throws IOException { if (closing || closed) return -1; @@ -305,16 +320,10 @@ public class SSLTransportLayer implements TransportLayer { if (unwrap.getHandshakeStatus() == HandshakeStatus.NEED_TASK) tasks(); //if we need more network data, than return for now. if (unwrap.getStatus() == Status.BUFFER_UNDERFLOW) return readFromAppBuffer(dst); - } else if (unwrap.getStatus() == Status.BUFFER_OVERFLOW && read > 0) { + } else if (unwrap.getStatus() == Status.BUFFER_OVERFLOW) { appReadBuffer = Utils.ensureCapacity(appReadBuffer, applicationBufferSize()); - //buffer overflow can happen, if we have read data, then //empty out the dst buffer before we do another read return readFromAppBuffer(dst); - } else { - //here we should trap BUFFER_OVERFLOW and call expand on the buffer - // for now, throw an exception, as we initialized the buffers - // in constructor - throw new IOException(String.format("Unable to unwrap data, invalid status [%s]", unwrap.getStatus())); } } while(netInBuffer.position() != 0); return readFromAppBuffer(dst); @@ -335,7 +344,6 @@ public class SSLTransportLayer implements TransportLayer { return totalRead; } - /** * Writes a sequence of bytes to this channel from the given buffer. * @@ -346,24 +354,20 @@ public class SSLTransportLayer implements TransportLayer { public int write(ByteBuffer src) throws IOException { int written = 0; - if (src == this.netOutBuffer) - written = socketChannel.write(src); - else { - if (closing || closed) throw new IOException("Channel is in closing state"); - if (!flush(netOutBuffer)) - return written; - netOutBuffer.clear(); - SSLEngineResult result = sslEngine.wrap(src, netOutBuffer); - written = result.bytesConsumed(); - netOutBuffer.flip(); - if (result.getStatus() == Status.OK) { - if (result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) - tasks(); - } else { - throw new IOException(String.format("Unable to wrap data, invalid status %s", result.getStatus())); - } - flush(netOutBuffer); + if (closing || closed) throw new IOException("Channel is in closing state"); + if (!flush(netOutBuffer)) + return written; + netOutBuffer.clear(); + SSLEngineResult result = sslEngine.wrap(src, netOutBuffer); + written = result.bytesConsumed(); + netOutBuffer.flip(); + if (result.getStatus() == Status.OK) { + if (result.getHandshakeStatus() == HandshakeStatus.NEED_TASK) + tasks(); + } else { + throw new IOException(String.format("Unable to wrap data, invalid status %s", result.getStatus())); } + flush(netOutBuffer); return written; } @@ -396,9 +400,12 @@ public class SSLTransportLayer implements TransportLayer { return outStream; } - public Principal getPeerPrincipal() { - //return sslEngine.getSession().getPeerPrincipal(); - return null; + public Principal getPeerPrincipal() throws IOException { + try { + return sslEngine.getSession().getPeerPrincipal(); + } catch (SSLPeerUnverifiedException se) { + throw new IOException(String.format("Unable to retrieve getPeerPrincipal due to %s", se)); + } } private int readFromAppBuffer(ByteBuffer dst) { @@ -427,4 +434,14 @@ public class SSLTransportLayer implements TransportLayer { private int applicationBufferSize() { return sslEngine.getSession().getApplicationBufferSize(); } + + private void handshakeFailure() { + //Release all resources such as internal buffers that SSLEngine is managing + sslEngine.closeOutbound(); + try { + sslEngine.closeInbound(); + } catch (SSLException e) { + log.debug("SSLEngine.closeInBound() raised an exception.", e); + } + } } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 0068143..90e2cee 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -29,9 +29,10 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Measurable; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.MetricName; @@ -43,6 +44,7 @@ import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.clients.CommonClientConfigs; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,14 +92,14 @@ public class Selector implements Selectable { private final SelectorMetrics sensors; private final String metricGrpPrefix; private final Map metricTags; - private final SecurityConfig securityConfig; private final SecurityProtocol securityProtocol; private SSLFactory sslFactory = null; + private ExecutorService executorService = null; /** * Create a new selector */ - public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, SecurityConfig securityConfig) { + public Selector(Metrics metrics, Time time, String metricGrpPrefix, Map metricTags, Map configs) { try { this.selector = java.nio.channels.Selector.open(); } catch (IOException e) { @@ -114,17 +116,13 @@ public class Selector implements Selectable { this.disconnected = new ArrayList(); this.failedSends = new ArrayList(); this.sensors = new SelectorMetrics(metrics); - this.securityConfig = securityConfig; - this.securityProtocol = SecurityProtocol.valueOf(securityConfig.getString(SecurityConfig.SECURITY_PROTOCOL_CONFIG)); - try { - if (securityProtocol == SecurityProtocol.SSL) { - this.sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT); - this.sslFactory.init(this.securityConfig); - } - } catch (Exception e) { - throw new KafkaException(e); + this.securityProtocol = configs.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) ? + SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) : SecurityProtocol.PLAINTEXT; + if (securityProtocol == SecurityProtocol.SSL) { + this.executorService = Executors.newScheduledThreadPool(1); + this.sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT); + this.sslFactory.configure(configs); } - } /** @@ -166,7 +164,8 @@ public class Selector implements Selectable { if (securityProtocol == SecurityProtocol.SSL) { transportLayer = new SSLTransportLayer(socketChannel, sslFactory.createSSLEngine(socket.getInetAddress().getHostName(), - socket.getPort())); + socket.getPort()), + executorService); } else { transportLayer = new PlainTextTransportLayer(socketChannel); } @@ -206,8 +205,12 @@ public class Selector implements Selectable { close(key); try { this.selector.close(); + if (this.executorService != null) + this.executorService.shutdown(); } catch (IOException e) { log.error("Exception closing selector:", e); + } catch (SecurityException se) { + log.error("Exception closing selector:", se); } } @@ -463,7 +466,7 @@ public class Selector implements Selectable { } /** - * Get the socket channel associated with this selection key + * Get the Channel associated with this selection key */ private Channel channel(SelectionKey key) { return this.channels.get(key); diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java index 6ce013b..ae10f7c 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -82,5 +82,5 @@ public interface TransportLayer { boolean flush(ByteBuffer buffer) throws IOException; - Principal getPeerPrincipal(); + Principal getPeerPrincipal() throws IOException; } diff --git a/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java index 47dda69..7f34738 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java +++ b/clients/src/test/java/org/apache/kafka/common/network/EchoServer.java @@ -12,7 +12,7 @@ */ package org.apache.kafka.common.network; -import org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.protocol.SecurityProtocol; import java.io.DataInputStream; @@ -22,8 +22,9 @@ import java.net.ServerSocket; import java.net.Socket; import java.util.ArrayList; import java.util.Collections; +import java.util.Map; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; + /** @@ -34,17 +35,16 @@ class EchoServer extends Thread { private final ServerSocket serverSocket; private final List threads; private final List sockets; - private SecurityProtocol protocol; + private SecurityProtocol protocol = SecurityProtocol.PLAINTEXT; private SSLFactory sslFactory; - private final AtomicBoolean startHandshake = new AtomicBoolean(); - public EchoServer(SecurityConfig securityConfig) throws Exception { - this.protocol = SecurityProtocol.valueOf(securityConfig.getString(SecurityConfig.SECURITY_PROTOCOL_CONFIG)); + public EchoServer(Map configs) throws Exception { + this.protocol = configs.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) ? + SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) : SecurityProtocol.PLAINTEXT; if (protocol == SecurityProtocol.SSL) { this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); - this.sslFactory.init(securityConfig); + this.sslFactory.configure(configs); this.serverSocket = sslFactory.createSSLServerSocketFactory().createServerSocket(0); - this.startHandshake.set(true); } else { this.serverSocket = new ServerSocket(0); } @@ -105,4 +105,4 @@ class EchoServer extends Thread { t.join(); join(); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java index f3c6153..c5f8ecf 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SSLSelectorTest.java @@ -14,12 +14,12 @@ package org.apache.kafka.common.network; import static org.junit.Assert.assertEquals; +import java.util.Map; import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.LinkedHashMap; -import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.test.TestSSLUtils; @@ -42,12 +42,10 @@ public class SSLSelectorTest { @Before public void setup() throws Exception { - SecurityConfig serverSecurityConfig = TestSSLUtils.createSSLConfigFile(SSLFactory.Mode.SERVER, null); - this.server = new EchoServer(serverSecurityConfig); + Map> sslConfigs = TestSSLUtils.createSSLConfigs(false, true); + this.server = new EchoServer(sslConfigs.get(SSLFactory.Mode.SERVER)); this.server.start(); - String trustStoreServer = serverSecurityConfig.getString(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG); - SecurityConfig clientSecurityConfig = TestSSLUtils.createSSLConfigFile(SSLFactory.Mode.CLIENT, trustStoreServer); - this.selector = new Selector(new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), clientSecurityConfig); + this.selector = new Selector(new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), sslConfigs.get(SSLFactory.Mode.CLIENT)); } @After @@ -68,6 +66,44 @@ public class SSLSelectorTest { assertEquals(big, blockingRequest(node, big)); } + + /** + * Validate that when the server disconnects, a client send ends up with that node in the disconnected list. + */ + @Test + public void testServerDisconnect() throws Exception { + int node = 0; + // connect and do a simple request + blockingConnect(node); + assertEquals("hello", blockingRequest(node, "hello")); + + // disconnect + this.server.closeConnections(); + while (!selector.disconnected().contains(node)) + selector.poll(1000L); + + // reconnect and do another request + blockingConnect(node); + assertEquals("hello", blockingRequest(node, "hello")); + } + + + /** + * Tests wrap BUFFER_OVERFLOW and unwrap BUFFER_UNDERFLOW + * @throws Exception + */ + @Test + public void testLargeMessageSequence() throws Exception { + int bufferSize = 512 * 1024; + int node = 0; + int reqs = 50; + InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + String requestPrefix = TestUtils.randomString(bufferSize); + sendAndReceive(node, requestPrefix, 0, reqs); + } + + private String blockingRequest(int node, String s) throws IOException { selector.send(createSend(node, s)); selector.poll(1000L); @@ -94,4 +130,26 @@ public class SSLSelectorTest { selector.poll(10000L); } -} \ No newline at end of file + + private void sendAndReceive(int node, String requestPrefix, int startIndex, int endIndex) throws Exception { + int requests = startIndex; + int responses = startIndex; + selector.send(createSend(node, requestPrefix + "-" + startIndex)); + requests++; + while (responses < endIndex) { + // do the i/o + selector.poll(0L); + assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); + + // handle requests and responses of the fast node + for (NetworkReceive receive : selector.completedReceives()) { + assertEquals(requestPrefix + "-" + responses, asString(receive)); + responses++; + } + + for (int i = 0; i < selector.completedSends().size() && requests < endIndex; i++, requests++) { + selector.send(createSend(node, requestPrefix + "-" + requests)); + } + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java index e4100d3..3fd8fe2 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java @@ -3,9 +3,9 @@ * 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. @@ -15,14 +15,14 @@ package org.apache.kafka.common.network; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.HashMap; +import java.util.Map; import java.io.IOException; import java.net.InetSocketAddress; import java.net.ServerSocket; import java.nio.ByteBuffer; import java.util.LinkedHashMap; -import org.apache.kafka.clients.ClientUtils; -import org.apache.kafka.common.config.SecurityConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; @@ -43,10 +43,10 @@ public class SelectorTest { @Before public void setup() throws Exception { - SecurityConfig securityConfig = ClientUtils.parseSecurityConfig(""); - this.server = new EchoServer(securityConfig); + Map configs = new HashMap(); + this.server = new EchoServer(configs); this.server.start(); - this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap(), securityConfig); + this.selector = new Selector(new Metrics(), new MockTime() , "MetricGroup", new LinkedHashMap(), configs); } @After diff --git a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java index c811096..bfb52ef 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java @@ -17,19 +17,8 @@ package org.apache.kafka.test; -import org.apache.kafka.common.config.SecurityConfig; +import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.network.SSLFactory; -import sun.security.x509.AlgorithmId; -import sun.security.x509.CertificateAlgorithmId; -import sun.security.x509.CertificateIssuerName; -import sun.security.x509.CertificateSerialNumber; -import sun.security.x509.CertificateSubjectName; -import sun.security.x509.CertificateValidity; -import sun.security.x509.CertificateVersion; -import sun.security.x509.CertificateX509Key; -import sun.security.x509.X500Name; -import sun.security.x509.X509CertImpl; -import sun.security.x509.X509CertInfo; import java.io.File; import java.io.FileOutputStream; @@ -41,14 +30,19 @@ import java.security.KeyPair; import java.security.KeyPairGenerator; import java.security.KeyStore; import java.security.NoSuchAlgorithmException; -import java.security.PrivateKey; import java.security.SecureRandom; import java.security.cert.Certificate; import java.security.cert.X509Certificate; +import java.security.InvalidKeyException; +import java.security.NoSuchProviderException; +import java.security.SignatureException; +import java.security.cert.CertificateEncodingException; +import javax.security.auth.x500.X500Principal; +import org.bouncycastle.x509.X509V1CertificateGenerator; + import java.util.Date; import java.util.HashMap; import java.util.Map; -import java.util.Properties; public class TestSSLUtils { @@ -66,36 +60,24 @@ public class TestSSLUtils { * @throws GeneralSecurityException thrown if an Security error ocurred. */ public static X509Certificate generateCertificate(String dn, KeyPair pair, - int days, String algorithm) throws GeneralSecurityException, IOException { - PrivateKey privkey = pair.getPrivate(); - X509CertInfo info = new X509CertInfo(); + int days, String algorithm) + throws CertificateEncodingException, InvalidKeyException, IllegalStateException, + NoSuchProviderException, NoSuchAlgorithmException, SignatureException { Date from = new Date(); Date to = new Date(from.getTime() + days * 86400000L); - CertificateValidity interval = new CertificateValidity(from, to); BigInteger sn = new BigInteger(64, new SecureRandom()); - X500Name owner = new X500Name(dn); - - info.set(X509CertInfo.VALIDITY, interval); - info.set(X509CertInfo.SERIAL_NUMBER, new CertificateSerialNumber(sn)); - info.set(X509CertInfo.SUBJECT, new CertificateSubjectName(owner)); - info.set(X509CertInfo.ISSUER, new CertificateIssuerName(owner)); - info.set(X509CertInfo.KEY, new CertificateX509Key(pair.getPublic())); - info - .set(X509CertInfo.VERSION, new CertificateVersion(CertificateVersion.V3)); - AlgorithmId algo = new AlgorithmId(AlgorithmId.md5WithRSAEncryption_oid); - info.set(X509CertInfo.ALGORITHM_ID, new CertificateAlgorithmId(algo)); - - // Sign the cert to identify the algorithm that's used. - X509CertImpl cert = new X509CertImpl(info); - cert.sign(privkey, algorithm); - - // Update the algorith, and resign. - algo = (AlgorithmId) cert.get(X509CertImpl.SIG_ALG); - info - .set(CertificateAlgorithmId.NAME + "." + CertificateAlgorithmId.ALGORITHM, - algo); - cert = new X509CertImpl(info); - cert.sign(privkey, algorithm); + KeyPair keyPair = pair; + X509V1CertificateGenerator certGen = new X509V1CertificateGenerator(); + X500Principal dnName = new X500Principal(dn); + + certGen.setSerialNumber(sn); + certGen.setIssuerDN(dnName); + certGen.setNotBefore(from); + certGen.setNotAfter(to); + certGen.setSubjectDN(dnName); + certGen.setPublicKey(keyPair.getPublic()); + certGen.setSignatureAlgorithm(algorithm); + X509Certificate cert = certGen.generate(pair.getPrivate()); return cert; } @@ -168,41 +150,72 @@ public class TestSSLUtils { saveKeyStore(ks, filename, password); } - public static SecurityConfig createSSLConfigFile(SSLFactory.Mode mode, String trustStoreFileClient) throws IOException, GeneralSecurityException { - Properties securityConfigProps = new Properties(); + public static Map createX509Certificates(KeyPair keyPair) + throws GeneralSecurityException { Map certs = new HashMap(); - KeyPair keyPair = generateKeyPair("RSA"); X509Certificate cert = generateCertificate("CN=localhost, O=localhost", keyPair, 30, "SHA1withRSA"); - String password = "test"; - - if (mode == SSLFactory.Mode.SERVER) { - File keyStoreFile = File.createTempFile("keystore", ".jks"); - createKeyStore(keyStoreFile.getPath(), password, password, "localhost", keyPair.getPrivate(), cert); - certs.put("localhost", cert); - securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_LOCATION_CONFIG, keyStoreFile.getPath()); - securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_TYPE_CONFIG, "JKS"); - securityConfigProps.put(SecurityConfig.SSL_KEYMANAGER_ALGORITHM_CONFIG, "SunX509"); - securityConfigProps.put(SecurityConfig.SSL_KEYSTORE_PASSWORD_CONFIG, password); - securityConfigProps.put(SecurityConfig.SSL_KEY_PASSWORD_CONFIG, password); - - File trustStoreFile = File.createTempFile("truststore", ".jks"); - createTrustStore(trustStoreFile.getPath(), password, certs); - - securityConfigProps.put(SecurityConfig.SECURITY_PROTOCOL_CONFIG, "SSL"); - securityConfigProps.put(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG, "false"); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFile.getPath()); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, password); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); - } else { - securityConfigProps.put(SecurityConfig.SECURITY_PROTOCOL_CONFIG, "SSL"); - securityConfigProps.put(SecurityConfig.SSL_CLIENT_REQUIRE_CERT_CONFIG, "false"); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFileClient); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_PASSWORD_CONFIG, password); - securityConfigProps.put(SecurityConfig.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); + certs.put("localhost", cert); + return certs; + } + + public static Map createSSLConfig(SSLFactory.Mode mode, File keyStoreFile, String password, String keyPassword, + File trustStoreFile, String trustStorePassword, boolean useClientCert) { + Map sslConfigs = new HashMap(); + sslConfigs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); // kafka security protocol + sslConfigs.put(CommonClientConfigs.SSL_PROTOCOL_CONFIG, "TLS"); // protocol to create SSLContext + + if (mode == SSLFactory.Mode.SERVER || (mode == SSLFactory.Mode.CLIENT && keyStoreFile != null)) { + sslConfigs.put(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG, keyStoreFile.getPath()); + sslConfigs.put(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG, "JKS"); + sslConfigs.put(CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, "SunX509"); + sslConfigs.put(CommonClientConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, password); + sslConfigs.put(CommonClientConfigs.SSL_KEY_PASSWORD_CONFIG, keyPassword); + } + + sslConfigs.put(CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_CONFIG, useClientCert); + sslConfigs.put(CommonClientConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, trustStoreFile.getPath()); + sslConfigs.put(CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, trustStorePassword); + sslConfigs.put(CommonClientConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); + sslConfigs.put(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, "SunX509"); + return sslConfigs; + } + + public static Map> createSSLConfigs(boolean useClientCert, boolean trustStore) + throws IOException, GeneralSecurityException { + Map> sslConfigs = new HashMap>(); + Map certs = new HashMap(); + File trustStoreFile = File.createTempFile("truststore", ".jks"); + File clientKeyStoreFile = null; + File serverKeyStoreFile = File.createTempFile("serverKS", ".jks"); + String clientPassword = "ClientPassword"; + String serverPassword = "ServerPassword"; + String trustStorePassword = "TrustStorePassword"; + + if (useClientCert) { + clientKeyStoreFile = File.createTempFile("clientKS", ".jks"); + KeyPair cKP = generateKeyPair("RSA"); + X509Certificate cCert = generateCertificate("CN=localhost, O=client", cKP, 30, "SHA1withRSA"); + createKeyStore(clientKeyStoreFile.getPath(), clientPassword, "client", cKP.getPrivate(), cCert); + certs.put("client", cCert); + } + + KeyPair sKP = generateKeyPair("RSA"); + X509Certificate sCert = generateCertificate("CN=localhost, O=server", sKP, 30, + "SHA1withRSA"); + createKeyStore(serverKeyStoreFile.getPath(), serverPassword, serverPassword, "server", sKP.getPrivate(), sCert); + certs.put("server", sCert); + + if (trustStore) { + createTrustStore(trustStoreFile.getPath(), trustStorePassword, certs); } - securityConfigProps.put(SecurityConfig.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, "SunX509"); - return new SecurityConfig(securityConfigProps); + Map clientSSLConfig = createSSLConfig(SSLFactory.Mode.CLIENT, clientKeyStoreFile, clientPassword, + clientPassword, trustStoreFile, trustStorePassword, useClientCert); + Map serverSSLConfig = createSSLConfig(SSLFactory.Mode.SERVER, serverKeyStoreFile, serverPassword, + serverPassword, trustStoreFile, trustStorePassword, useClientCert); + sslConfigs.put(SSLFactory.Mode.CLIENT, clientSSLConfig); + sslConfigs.put(SSLFactory.Mode.SERVER, serverSSLConfig); + return sslConfigs; } } -- 2.3.2 (Apple Git-55) From 6183828b8cdef5ea8bf199fb337e70eed79a7343 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 10 May 2015 23:18:13 -0700 Subject: [PATCH 3/6] KAFKA-1690. new java producer needs ssl support as a client. --- .../src/main/java/org/apache/kafka/clients/ClientUtils.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 3657279..748576b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -13,21 +13,17 @@ package org.apache.kafka.clients; import java.io.Closeable; -import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.List; -import java.util.Properties; import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.config.ConfigException; -import org.apache.kafka.common.config.SecurityConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; -import static org.apache.kafka.common.utils.Utils.loadProps; public class ClientUtils { @@ -67,11 +63,4 @@ public class ClientUtils { } } - public static SecurityConfig parseSecurityConfig(String securityConfigFile) throws IOException { - Properties securityProps = new Properties(); - if (securityConfigFile != null && securityConfigFile != "") { - securityProps = loadProps(securityConfigFile); - } - return new SecurityConfig(securityProps); - } } -- 2.3.2 (Apple Git-55) From c72d4fd67862532746126aa68a14249547e7895b Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 10 May 2015 23:31:25 -0700 Subject: [PATCH 4/6] KAFKA-1690. new java producer needs ssl support as a client. --- .../main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 4ccd423..85fdc09 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -157,11 +157,6 @@ public class ConsumerConfig extends AbstractConfig { public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; - /** security.config.file */ - public static final String SECURITY_CONFIG_FILE_CONFIG = CommonClientConfigs.SECURITY_CONFIG_FILE_CONFIG; - private static final String SECURITY_CONFIG_FILE_DOC = CommonClientConfigs.SECURITY_CONFIG_FILE_DOC; - - static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, -- 2.3.2 (Apple Git-55) From f61cccee9818ffdf90381d2dadc2fdb4e4921274 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 11 May 2015 16:09:01 -0700 Subject: [PATCH 5/6] KAFKA-1690. new java producer needs ssl support as a client. SSLFactory tests. --- .../kafka/clients/consumer/ConsumerConfig.java | 2 +- .../kafka/clients/producer/ProducerConfig.java | 2 +- .../apache/kafka/common/network/SSLFactory.java | 16 +++++-- .../kafka/common/network/SSLFactoryTest.java | 56 ++++++++++++++++++++++ .../java/org/apache/kafka/test/TestSSLUtils.java | 7 +++ 5 files changed, 76 insertions(+), 7 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 85fdc09..95f3a46 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -280,7 +280,7 @@ public class ConsumerConfig extends AbstractConfig { VALUE_DESERIALIZER_CLASS_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(CommonClientConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, CommonClientConfigs.SSL_PROTOCOL_DOC) - .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC) + .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC, false) .define(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, CommonClientConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_KEYSTORE_TYPE_DOC) .define(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_LOCATION_DOC, false) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 69563ee..552cafb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -223,7 +223,7 @@ public class ProducerConfig extends AbstractConfig { .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, Importance.HIGH, VALUE_SERIALIZER_CLASS_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, SecurityProtocol.PLAINTEXT.toString(), Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(CommonClientConfigs.SSL_PROTOCOL_CONFIG, Type.STRING, "TLS", Importance.MEDIUM, CommonClientConfigs.SSL_PROTOCOL_DOC) - .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, "", Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC) + .define(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG, Type.LIST, Importance.LOW, CommonClientConfigs.SSL_CIPHER_SUITES_DOC, false) .define(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Type.LIST, "TLSv1.2, TLSv1.1, TLSv1", Importance.MEDIUM, CommonClientConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(CommonClientConfigs.SSL_KEYSTORE_TYPE_CONFIG, Type.STRING, "JKS", Importance.MEDIUM, CommonClientConfigs.SSL_KEYSTORE_TYPE_DOC) .define(CommonClientConfigs.SSL_KEYSTORE_LOCATION_CONFIG, Type.STRING, Importance.HIGH, CommonClientConfigs.SSL_KEYSTORE_LOCATION_DOC, false) diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java index 73f976b..7fbb9d7 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLFactory.java @@ -54,10 +54,17 @@ public class SSLFactory implements Configurable { @Override public void configure(Map configs) throws KafkaException { this.protocol = (String) configs.get(CommonClientConfigs.SSL_PROTOCOL_CONFIG); - if (configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG) != null) - this.cipherSuites = (String[]) ((List) configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG)).toArray(); - if (configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG) != null) - this.enabledProtocols = (String[]) ((List) configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG)).toArray(); + + if (configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG) != null) { + List cipherSuitesList = (List) configs.get(CommonClientConfigs.SSL_CIPHER_SUITES_CONFIG); + this.cipherSuites = (String[]) cipherSuitesList.toArray(new String[cipherSuitesList.size()]); + } + + if (configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG) != null) { + List enabledProtocolsList = (List) configs.get(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); + this.enabledProtocols = (String[]) enabledProtocolsList.toArray(new String[enabledProtocolsList.size()]); + } + this.requireClientCert = (Boolean) configs.get(CommonClientConfigs.SSL_CLIENT_REQUIRE_CERT_CONFIG); this.kmfAlgorithm = (String) configs.get(CommonClientConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG); this.tmfAlgorithm = (String) configs.get(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG); @@ -73,7 +80,6 @@ public class SSLFactory implements Configurable { try { this.sslContext = createSSLContext(); } catch (Exception e) { - e.printStackTrace(); throw new KafkaException(e); } } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java new file mode 100644 index 0000000..02a3eff --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/SSLFactoryTest.java @@ -0,0 +1,56 @@ +/** + * 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 javax.net.ssl.*; + +import java.util.Map; + +import org.apache.kafka.test.TestSSLUtils; + +import org.junit.Test; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + + +/** + * A set of tests for the selector over ssl. These use a test harness that runs a simple socket server that echos back responses. + */ + +public class SSLFactoryTest { + + @Test + public void testSSLFactoryConfiguration() throws Exception { + Map> sslConfigs = TestSSLUtils.createSSLConfigs(false, true); + Map serverSSLConfig = sslConfigs.get(SSLFactory.Mode.SERVER); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER); + sslFactory.configure(serverSSLConfig); + SSLEngine engine = sslFactory.createSSLEngine("localhost", 9093); + assertNotNull(engine); + String[] expectedProtocols = {"TLSv1.2"}; + assertEquals(expectedProtocols, engine.getEnabledProtocols()); + assertEquals(false, engine.getUseClientMode()); + } + + @Test + public void testClientMode() throws Exception { + Map> sslConfigs = TestSSLUtils.createSSLConfigs(false, true); + Map clientSSLConfig = sslConfigs.get(SSLFactory.Mode.CLIENT); + SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT); + sslFactory.configure(clientSSLConfig); + SSLEngine engine = sslFactory.createSSLEngine("localhost", 9093); + assertTrue(engine.getUseClientMode()); + } + +} diff --git a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java index bfb52ef..590f1f5 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java @@ -43,6 +43,8 @@ import org.bouncycastle.x509.X509V1CertificateGenerator; import java.util.Date; import java.util.HashMap; import java.util.Map; +import java.util.List; +import java.util.ArrayList; public class TestSSLUtils { @@ -177,6 +179,11 @@ public class TestSSLUtils { sslConfigs.put(CommonClientConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, trustStorePassword); sslConfigs.put(CommonClientConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); sslConfigs.put(CommonClientConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, "SunX509"); + + List enabledProtocols = new ArrayList(); + enabledProtocols.add("TLSv1.2"); + sslConfigs.put(CommonClientConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, enabledProtocols); + return sslConfigs; } -- 2.3.2 (Apple Git-55) From 8c5ff865c22fbfe1cd020f7736c6d4489e6fe514 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Tue, 12 May 2015 16:09:38 -0700 Subject: [PATCH 6/6] KAFKA-1690. new java producer needs ssl support as a client. Added PrincipalBuilder. --- checkstyle/checkstyle.xml | 25 +++++----- checkstyle/import-control.xml | 17 ++++--- .../apache/kafka/common/network/Authenticator.java | 18 +++----- .../org/apache/kafka/common/network/Channel.java | 7 ++- .../kafka/common/network/DefaultAuthenticator.java | 21 ++++++--- .../common/network/PlainTextTransportLayer.java | 17 ++++--- .../kafka/common/network/SSLTransportLayer.java | 11 ++++- .../org/apache/kafka/common/network/Selector.java | 5 +- .../kafka/common/network/TransportLayer.java | 15 +++++- .../security/auth/DefaultPrincipalBuilder.java | 37 +++++++++++++++ .../kafka/common/security/auth/KafkaPrincipal.java | 54 ++++++++++++++++++++++ .../common/security/auth/PrincipalBuilder.java | 38 +++++++++++++++ 12 files changed, 212 insertions(+), 53 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index 5fbf562..5031415 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -1,6 +1,6 @@ +--> - + - + - + - + + - + @@ -59,12 +60,12 @@ - + - + @@ -79,4 +80,4 @@ - \ No newline at end of file + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 339c620..a921757 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -26,7 +26,6 @@ - @@ -41,13 +40,9 @@ - - - - @@ -57,12 +52,17 @@ + + + + - - - + + + + @@ -91,7 +91,6 @@ - diff --git a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java index 1b6b32a..920ed88 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Authenticator.java @@ -22,39 +22,33 @@ package org.apache.kafka.common.network; */ import java.io.IOException; -import com.sun.security.auth.UserPrincipal; +import java.security.Principal; +import org.apache.kafka.common.KafkaException; public interface Authenticator { /** - * Closes this channel + * Closes this Authenticator * * @throws IOException if any I/O error occurs */ void close() throws IOException; /** - * - * @throws IOException - */ - void init() throws IOException; - - /** - * Returns UserPrincipal after authentication is established + * Returns Principal after authentication is established */ - UserPrincipal userPrincipal() throws IOException; - + Principal principal() throws KafkaException; /** * Does authentication and returns SelectionKey.OP if further communication needed + * If no further authentication needs to be done return 0. */ int authenticate(boolean read, boolean write) throws IOException; /** * returns true if authentication is complete otherwise returns false; */ - boolean isComplete(); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/Channel.java b/clients/src/main/java/org/apache/kafka/common/network/Channel.java index d9d1192..7b2489e 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Channel.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Channel.java @@ -27,7 +27,7 @@ import java.nio.channels.ScatteringByteChannel; import java.nio.channels.GatheringByteChannel; import java.nio.channels.SocketChannel; -import com.sun.security.auth.UserPrincipal; +import java.security.Principal; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,7 +45,6 @@ public class Channel implements ScatteringByteChannel, GatheringByteChannel { public Channel(TransportLayer transportLayer, Authenticator authenticator) throws IOException { this.transportLayer = transportLayer; this.authenticator = authenticator; - this.authenticator.init(); } public void close() throws IOException { @@ -58,8 +57,8 @@ public class Channel implements ScatteringByteChannel, GatheringByteChannel { * Incase of PLAINTEXT and No Authentication returns ANONYMOUS as the userPrincipal * If SSL used without any SASL Authentication returns SSLSession.peerPrincipal */ - public UserPrincipal userPrincipal() throws IOException { - return authenticator.userPrincipal(); + public Principal principal() throws IOException { + return authenticator.principal(); } public int connect(boolean read, boolean write) throws IOException { diff --git a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java index 97b1135..3a66e7b 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/network/DefaultAuthenticator.java @@ -17,25 +17,34 @@ package org.apache.kafka.common.network; -import com.sun.security.auth.UserPrincipal; +import java.security.Principal; import java.io.IOException; +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.KafkaException; + public class DefaultAuthenticator implements Authenticator { TransportLayer transportLayer; + PrincipalBuilder principalBuilder; + Principal principal; - public DefaultAuthenticator(TransportLayer transportLayer) { + public DefaultAuthenticator(TransportLayer transportLayer, PrincipalBuilder principalBuilder) { this.transportLayer = transportLayer; + this.principalBuilder = principalBuilder; } - public void init() {} - + /* + * No-Op for default authenticator + */ public int authenticate(boolean read, boolean write) throws IOException { return 0; } - public UserPrincipal userPrincipal() throws IOException { - return new UserPrincipal(transportLayer.getPeerPrincipal().toString()); + public Principal principal() throws KafkaException { + if (principal != null) + principal = principalBuilder.buildPrincipal(transportLayer, this); + return principal; } public void close() throws IOException {} diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java index dbf0a30..17b014f 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/PlainTextTransportLayer.java @@ -22,14 +22,15 @@ package org.apache.kafka.common.network; */ import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.SocketChannel; - import java.io.DataInputStream; import java.io.DataOutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.SocketChannel; +import javax.net.ssl.SSLSession; import java.security.Principal; -import com.sun.security.auth.UserPrincipal; + +import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -140,8 +141,12 @@ public class PlainTextTransportLayer implements TransportLayer { return outStream; } - public Principal getPeerPrincipal() throws IOException { - return new UserPrincipal("ANONYMOUS"); + public Principal peerPrincipal() throws IOException { + return new KafkaPrincipal("ANONYMOUS"); + } + + public SSLSession sslSession() throws IllegalStateException, UnsupportedOperationException { + throw new UnsupportedOperationException("sslSession not supported for PlainTextTransportLayer"); } } diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java index 2d6a519..0267e85 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SSLTransportLayer.java @@ -28,6 +28,7 @@ import javax.net.ssl.SSLEngineResult; import javax.net.ssl.SSLEngineResult.HandshakeStatus; import javax.net.ssl.SSLEngineResult.Status; import javax.net.ssl.SSLException; +import javax.net.ssl.SSLSession; import javax.net.ssl.SSLPeerUnverifiedException; import java.io.DataInputStream; @@ -267,7 +268,7 @@ public class SSLTransportLayer implements TransportLayer { sslEngine.closeOutbound(); if (!flush(netOutBuffer)) { - throw new IOException("Remaining data in the network buffer, can't send SSL close message, force a close with close(true) instead"); + throw new IOException("Remaining data in the network buffer, can't send SSL close message."); } //prep the buffer for the close message netOutBuffer.clear(); @@ -400,7 +401,7 @@ public class SSLTransportLayer implements TransportLayer { return outStream; } - public Principal getPeerPrincipal() throws IOException { + public Principal peerPrincipal() throws IOException { try { return sslEngine.getSession().getPeerPrincipal(); } catch (SSLPeerUnverifiedException se) { @@ -408,6 +409,12 @@ public class SSLTransportLayer implements TransportLayer { } } + public SSLSession sslSession() throws IllegalStateException, UnsupportedOperationException { + if (!handshakeComplete) + throw new IllegalStateException("Handshake incomplete."); + return sslEngine.getSession(); + } + private int readFromAppBuffer(ByteBuffer dst) { appReadBuffer.flip(); try { diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 90e2cee..ea95858 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -43,6 +43,8 @@ import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.security.auth.DefaultPrincipalBuilder; import org.apache.kafka.common.utils.Time; import org.apache.kafka.clients.CommonClientConfigs; import org.slf4j.Logger; @@ -169,7 +171,8 @@ public class Selector implements Selectable { } else { transportLayer = new PlainTextTransportLayer(socketChannel); } - Authenticator authenticator = new DefaultAuthenticator(transportLayer); + PrincipalBuilder principalBuilder = new DefaultPrincipalBuilder(); + Authenticator authenticator = new DefaultAuthenticator(transportLayer, principalBuilder); Channel channel = new Channel(transportLayer, authenticator); SelectionKey key = socketChannel.register(this.selector, SelectionKey.OP_CONNECT); key.attach(new Transmissions(id)); diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java index ae10f7c..0531d6f 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -27,6 +27,7 @@ import java.nio.channels.SocketChannel; import java.io.DataInputStream; import java.io.DataOutputStream; +import javax.net.ssl.SSLSession; import java.security.Principal; @@ -82,5 +83,17 @@ public interface TransportLayer { boolean flush(ByteBuffer buffer) throws IOException; - Principal getPeerPrincipal() throws IOException; + + /** + * returns SSLSession.getPeerPrinicpal if SSLTransportLayer used + * for non-secure returns a "ANONYMOUS" as the peerPrincipal + */ + Principal peerPrincipal() throws IOException; + + /** + * returns a SSL Session after the handshake is established + * throws IlleagalStateException if the handshake is not established + * throws UnsupportedOperationException for non-secure implementation + */ + SSLSession sslSession() throws IllegalStateException, UnsupportedOperationException; } diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java new file mode 100644 index 0000000..d594e4d --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/DefaultPrincipalBuilder.java @@ -0,0 +1,37 @@ +/** + * 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.security.auth; + +import java.security.Principal; + +import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.KafkaException; + + +public class DefaultPrincipalBuilder implements PrincipalBuilder { + + public Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException { + try { + return transportLayer.peerPrincipal(); + } catch (Exception e) { + throw new KafkaException("Failed to build principal due to: ", e); + } + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java new file mode 100644 index 0000000..4a7ace8 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/KafkaPrincipal.java @@ -0,0 +1,54 @@ +/** + * 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.security.auth; + +import java.security.Principal; + +public class KafkaPrincipal implements Principal { + private final String name; + + public KafkaPrincipal(String name) { + if (name == null) + throw new IllegalArgumentException("name is null"); + this.name = name; + } + + public boolean equals(Object object) { + if (this == object) + return true; + + if (object instanceof KafkaPrincipal) { + return name.equals(((KafkaPrincipal) object).getName()); + } + + return false; + } + + public int hashCode() { + return name.hashCode(); + } + + public String getName() { + return name; + } + + public String toString() { + return name; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java new file mode 100644 index 0000000..5b39222 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/PrincipalBuilder.java @@ -0,0 +1,38 @@ +/** + * 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.security.auth; + +/* + * PrincipalBuilder for Authenticator + */ +import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.KafkaException; + +import java.security.Principal; + +public interface PrincipalBuilder { + + /** + * Returns Principal + * @param TransportLayer + * @param Authenticator + */ + Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException; + +} -- 2.3.2 (Apple Git-55)