diff --git a/build.gradle b/build.gradle
index 2b0f66be59ed5..cea945e87a332 100644
--- a/build.gradle
+++ b/build.gradle
@@ -268,8 +268,8 @@ project(':core') {
testCompile "org.scalatest:scalatest_$baseScalaVersion:2.2.5"
testCompile project(':clients')
testCompile project(':clients').sourceSets.test.output
+ testCompile 'org.apache.hadoop:hadoop-minikdc:2.7.1'
testRuntime "$slf4jlog4j"
-
zinc 'com.typesafe.zinc:zinc:0.3.7'
}
@@ -282,6 +282,9 @@ project(':core') {
compile.exclude module: 'jmxtools'
compile.exclude module: 'mail'
compile.exclude module: 'netty'
+ // To prevent a UniqueResourceException due the same resource existing in both
+ // org.apache.directory.api/api-all and org.apache.directory.api/api-ldap-schema-data
+ testCompile.exclude module: 'api-ldap-schema-data'
}
tasks.create(name: "copyDependantLibs", type: Copy) {
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 289f1d088ed87..6474865ef6bf6 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -29,15 +29,17 @@
-
+
-
+
+
+
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 e7514f815f9db..41e5d74fc673c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
+++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
@@ -20,9 +20,10 @@
import java.util.concurrent.atomic.AtomicReference;
import org.apache.kafka.common.network.ChannelBuilders;
+import org.apache.kafka.common.network.LoginType;
+import org.apache.kafka.common.network.Mode;
import org.apache.kafka.common.protocol.SecurityProtocol;
import org.apache.kafka.common.network.ChannelBuilder;
-import org.apache.kafka.common.security.ssl.SSLFactory;
import org.apache.kafka.common.config.ConfigException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -74,9 +75,9 @@ public static void closeQuietly(Closeable c, String name, AtomicReference configs) {
SecurityProtocol securityProtocol = SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
- if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT)
- throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG);
- return ChannelBuilders.create(securityProtocol, SSLFactory.Mode.CLIENT, configs);
+ if (securityProtocol == SecurityProtocol.TRACE)
+ throw new ConfigException("Invalid SecurityProtocol " + securityProtocol);
+ return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs);
}
}
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 10422089b1d5b..1894822149bc4 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
@@ -19,6 +19,7 @@
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.config.SSLConfigs;
+import org.apache.kafka.common.config.SaslConfigs;
import java.util.HashMap;
import java.util.Map;
@@ -302,6 +303,13 @@ public class ConsumerConfig extends AbstractConfig {
.define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, Type.STRING, Importance.MEDIUM, SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, Type.STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, Importance.LOW, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
+ .define(SaslConfigs.SASL_KAFKA_SERVER_REALM, Type.STRING, Importance.LOW, SaslConfigs.SASL_KAFKA_SERVER_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
+ .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
+ .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
+ .define(SaslConfigs.AUTH_TO_LOCAL, Type.LIST, SaslConfigs.DEFAULT_AUTH_TO_LOCAL, Importance.MEDIUM, SaslConfigs.AUTH_TO_LOCAL_DOC)
.define(REQUEST_TIMEOUT_MS_CONFIG,
Type.INT,
40 * 1000,
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 4badd3363739a..b3cfe7057d3e7 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
@@ -23,6 +23,7 @@
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.SSLConfigs;
+import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
@@ -279,6 +280,13 @@ public class ProducerConfig extends AbstractConfig {
.define(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_KEYMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, Type.STRING, SSLConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM, Importance.LOW, SSLConfigs.SSL_TRUSTMANAGER_ALGORITHM_DOC)
.define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, Type.STRING, Importance.MEDIUM, SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, Type.STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, Importance.LOW, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
+ .define(SaslConfigs.SASL_KAFKA_SERVER_REALM, Type.STRING, Importance.LOW, SaslConfigs.SASL_KAFKA_SERVER_DOC, false)
+ .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
+ .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
+ .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
+ .define(SaslConfigs.AUTH_TO_LOCAL, Type.LIST, SaslConfigs.DEFAULT_AUTH_TO_LOCAL, Importance.MEDIUM, SaslConfigs.AUTH_TO_LOCAL_DOC)
/* default is set to be a bit lower than the server default (10 min), to avoid both client and server closing connection at same time */
.define(CONNECTIONS_MAX_IDLE_MS_CONFIG, Type.LONG, 9 * 60 * 1000, Importance.MEDIUM, CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC)
.define(PARTITIONER_CLASS_CONFIG, Type.CLASS, "org.apache.kafka.clients.producer.internals.DefaultPartitioner", Importance.MEDIUM, PARTITIONER_CLASS_DOC);
diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
new file mode 100644
index 0000000000000..0abefe715b3ab
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.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.config;
+
+import java.util.Collections;
+import java.util.List;
+
+public class SaslConfigs {
+ /*
+ * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
+ */
+
+ public static final String SASL_KAFKA_SERVER_REALM = "sasl.kafka.server.realm";
+ public static final String SASL_KAFKA_SERVER_DOC = "The sasl kafka server realm. "
+ + "Default will be from kafka jaas config";
+
+ public static final String SASL_KERBEROS_SERVICE_NAME = "sasl.kerberos.service.name";
+ public static final String SASL_KERBEROS_SERVICE_NAME_DOC = "The Kerberos principal name that Kafka runs as. "
+ + "This can be defined either in the JAAS config or in the Kakfa config.";
+
+ public static final String SASL_KERBEROS_KINIT_CMD = "sasl.kerberos.kinit.cmd";
+ public static final String SASL_KERBEROS_KINIT_CMD_DOC = "Kerberos kinit command path. "
+ + "Default will be /usr/bin/kinit";
+ public static final String DEFAULT_KERBEROS_KINIT_CMD = "/usr/bin/kinit";
+
+ public static final String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR = "sasl.kerberos.ticket.renew.window.factor";
+ public static final String SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC = "LoginThread will sleep until specified window factor of time from last refresh"
+ + " to ticket's expiry has been reached, at which time it will wake and try to renew the ticket.";
+ public static final double DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR = 0.80;
+
+ public static final String SASL_KERBEROS_TICKET_RENEW_JITTER = "sasl.kerberos.ticket.renew.jitter";
+ public static final String SASL_KERBEROS_TICKET_RENEW_JITTER_DOC = "Percentage of random jitter added to the renewal time";
+ public static final double DEFAULT_KERBEROS_TICKET_RENEW_JITTER = 0.05;
+
+ public static final String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN = "sasl.kerberos.min.time.before.relogin";
+ public static final String SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC = "LoginThread sleep time between refresh attempts";
+ public static final long DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L;
+
+ public static final String AUTH_TO_LOCAL = "kafka.security.auth.to.local";
+ public static final String AUTH_TO_LOCAL_DOC = "Rules for the mapping between principal names and operating system user names";
+ public static final List DEFAULT_AUTH_TO_LOCAL = Collections.singletonList("DEFAULT");
+
+}
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 261f571715f7d..7f6eb8cc44631 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,6 +22,7 @@
*/
import java.io.IOException;
+import java.util.Map;
import java.security.Principal;
import org.apache.kafka.common.security.auth.PrincipalBuilder;
@@ -30,11 +31,13 @@
public interface Authenticator {
/**
- * configures Authenticator using principalbuilder and transportLayer.
- * @param TransportLayer transportLayer
- * @param PrincipalBuilder principalBuilder
+ * Configures Authenticator using the provided parameters.
+ *
+ * @param transportLayer The transport layer used to read or write tokens
+ * @param principalBuilder The builder used to construct `Principal`
+ * @param configs Additional configuration parameters as key/value pairs
*/
- void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder);
+ void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs);
/**
* Implements any authentication mechanism. Use transportLayer to read or write tokens.
diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java
index 52a7aab1e15ea..0b7c328d5c8d1 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilder.java
@@ -32,6 +32,8 @@ public interface ChannelBuilder {
* returns a Channel with TransportLayer and Authenticator configured.
* @param id channel id
* @param key SelectionKey
+ * @param maxReceiveSize
+ * @return KafkaChannel
*/
KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException;
diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
index 2332d3f42815e..1e5d8405045e2 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
@@ -14,7 +14,6 @@
package org.apache.kafka.common.network;
import org.apache.kafka.common.protocol.SecurityProtocol;
-import org.apache.kafka.common.security.ssl.SSLFactory;
import java.util.Map;
@@ -24,20 +23,28 @@ private ChannelBuilders() { }
/**
* @param securityProtocol the securityProtocol
- * @param mode the SSL mode, it must be non-null if `securityProcol` is `SSL` and it is ignored otherwise
+ * @param mode the mode, it must be non-null if `securityProtocol` is not `PLAINTEXT`;
+ * it is ignored otherwise
+ * @param loginType the loginType, it must be non-null if `securityProtocol` is SASL_*; it is ignored otherwise
* @param configs client/server configs
* @return the configured `ChannelBuilder`
* @throws IllegalArgumentException if `mode` invariants described above is not maintained
*/
- public static ChannelBuilder create(SecurityProtocol securityProtocol, SSLFactory.Mode mode, Map configs) {
- ChannelBuilder channelBuilder = null;
+ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode, LoginType loginType, Map configs) {
+ ChannelBuilder channelBuilder;
switch (securityProtocol) {
case SSL:
- if (mode == null)
- throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `SSL`");
+ requireNonNullMode(mode, securityProtocol);
channelBuilder = new SSLChannelBuilder(mode);
break;
+ case SASL_SSL:
+ case SASL_PLAINTEXT:
+ requireNonNullMode(mode, securityProtocol);
+ if (loginType == null)
+ throw new IllegalArgumentException("`loginType` must be non-null if `securityProtocol` is `" + securityProtocol + "`");
+ channelBuilder = new SaslChannelBuilder(mode, loginType, securityProtocol);
+ break;
case PLAINTEXT:
case TRACE:
channelBuilder = new PlaintextChannelBuilder();
@@ -49,4 +56,10 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, SSLFactor
channelBuilder.configure(configs);
return channelBuilder;
}
+
+ private static void requireNonNullMode(Mode mode, SecurityProtocol securityProtocol) {
+ if (mode == null)
+ throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `" + securityProtocol + "`");
+ }
+
}
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 813a4aafeb927..650ad4151bc9f 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
@@ -19,6 +19,7 @@
import java.security.Principal;
import java.io.IOException;
+import java.util.Map;
import org.apache.kafka.common.security.auth.PrincipalBuilder;
import org.apache.kafka.common.KafkaException;
@@ -29,7 +30,7 @@ public class DefaultAuthenticator implements Authenticator {
private PrincipalBuilder principalBuilder;
private Principal principal;
- public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) {
+ public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) {
this.transportLayer = transportLayer;
this.principalBuilder = principalBuilder;
}
@@ -54,7 +55,7 @@ public void close() throws IOException {}
/**
* DefaultAuthenticator doesn't implement any additional authentication mechanism.
- * @returns true
+ * @return true
*/
public boolean complete() {
return true;
diff --git a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java
index 172f4cda38d2f..ac436c37a0922 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java
@@ -61,11 +61,9 @@ public Principal principal() throws IOException {
}
/**
- * Does handshake of transportLayer and Authentication using configured authenticator
+ * Does handshake of transportLayer and authentication using configured authenticator
*/
public void prepare() throws IOException {
- if (transportLayer.ready() && authenticator.complete())
- return;
if (!transportLayer.ready())
transportLayer.handshake();
if (transportLayer.ready() && !authenticator.complete())
diff --git a/clients/src/main/java/org/apache/kafka/common/network/LoginType.java b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java
new file mode 100644
index 0000000000000..9216cb0c5c08f
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java
@@ -0,0 +1,39 @@
+/**
+ * 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.security.JaasUtils;
+
+/**
+ * The type of the login context, it should be SERVER for the broker and CLIENT for the clients (i.e. consumer and
+ * producer). It provides the the login context name which defines the section of the JAAS configuration file to be used
+ * for login.
+ */
+public enum LoginType {
+ CLIENT(JaasUtils.LOGIN_CONTEXT_CLIENT),
+ SERVER(JaasUtils.LOGIN_CONTEXT_SERVER);
+
+ private final String contextName;
+
+ LoginType(String contextName) {
+ this.contextName = contextName;
+ }
+
+ public String contextName() {
+ return contextName;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/network/Mode.java b/clients/src/main/java/org/apache/kafka/common/network/Mode.java
new file mode 100644
index 0000000000000..67de44d711700
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/network/Mode.java
@@ -0,0 +1,19 @@
+/**
+ * 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;
+
+public enum Mode { CLIENT, SERVER };
diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
index 76dbf93a02a81..a028159085673 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextChannelBuilder.java
@@ -27,11 +27,13 @@
public class PlaintextChannelBuilder implements ChannelBuilder {
private static final Logger log = LoggerFactory.getLogger(PlaintextChannelBuilder.class);
private PrincipalBuilder principalBuilder;
+ private Map configs;
public void configure(Map configs) throws KafkaException {
try {
+ this.configs = configs;
this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class>) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG));
- this.principalBuilder.configure(configs);
+ this.principalBuilder.configure(this.configs);
} catch (Exception e) {
throw new KafkaException(e);
}
@@ -42,7 +44,7 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize
try {
PlaintextTransportLayer transportLayer = new PlaintextTransportLayer(key);
Authenticator authenticator = new DefaultAuthenticator();
- authenticator.configure(transportLayer, this.principalBuilder);
+ authenticator.configure(transportLayer, this.principalBuilder, this.configs);
channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize);
} catch (Exception e) {
log.warn("Failed to create channel due to ", e);
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 1149c99d7ea50..8949e5efbb25d 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
@@ -52,10 +52,7 @@ public boolean ready() {
@Override
public void finishConnect() throws IOException {
socketChannel.finishConnect();
- int ops = key.interestOps();
- ops &= ~SelectionKey.OP_CONNECT;
- ops |= SelectionKey.OP_READ;
- key.interestOps(ops);
+ key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
}
@Override
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java
index e2cce5cac1d54..1dd1ecd11a2d6 100644
--- a/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java
@@ -29,18 +29,20 @@ public class SSLChannelBuilder implements ChannelBuilder {
private static final Logger log = LoggerFactory.getLogger(SSLChannelBuilder.class);
private SSLFactory sslFactory;
private PrincipalBuilder principalBuilder;
- private SSLFactory.Mode mode;
+ private Mode mode;
+ private Map configs;
- public SSLChannelBuilder(SSLFactory.Mode mode) {
+ public SSLChannelBuilder(Mode mode) {
this.mode = mode;
}
public void configure(Map configs) throws KafkaException {
try {
+ this.configs = configs;
this.sslFactory = new SSLFactory(mode);
- this.sslFactory.configure(configs);
+ this.sslFactory.configure(this.configs);
this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class>) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG));
- this.principalBuilder.configure(configs);
+ this.principalBuilder.configure(this.configs);
} catch (Exception e) {
throw new KafkaException(e);
}
@@ -51,7 +53,7 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize
try {
SSLTransportLayer transportLayer = buildTransportLayer(sslFactory, id, key);
Authenticator authenticator = new DefaultAuthenticator();
- authenticator.configure(transportLayer, this.principalBuilder);
+ authenticator.configure(transportLayer, this.principalBuilder, this.configs);
channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize);
} catch (Exception e) {
log.info("Failed to create channel due to ", e);
@@ -66,10 +68,8 @@ public void close() {
protected SSLTransportLayer buildTransportLayer(SSLFactory sslFactory, String id, SelectionKey key) throws IOException {
SocketChannel socketChannel = (SocketChannel) key.channel();
- SSLTransportLayer transportLayer = new SSLTransportLayer(id, key,
- sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(),
- socketChannel.socket().getPort()));
- transportLayer.startHandshake();
- return transportLayer;
+ return SSLTransportLayer.create(id, key,
+ sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(),
+ socketChannel.socket().getPort()));
}
}
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 35ea9aae56abb..e7afa02927a4b 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
@@ -58,7 +58,14 @@ public class SSLTransportLayer implements TransportLayer {
private ByteBuffer appReadBuffer;
private ByteBuffer emptyBuf = ByteBuffer.allocate(0);
- public SSLTransportLayer(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException {
+ public static SSLTransportLayer create(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException {
+ SSLTransportLayer transportLayer = new SSLTransportLayer(channelId, key, sslEngine);
+ transportLayer.startHandshake();
+ return transportLayer;
+ }
+
+ // Prefer `create`, only use this in tests
+ SSLTransportLayer(String channelId, SelectionKey key, SSLEngine sslEngine) throws IOException {
this.channelId = channelId;
this.key = key;
this.socketChannel = (SocketChannel) key.channel();
diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
new file mode 100644
index 0000000000000..53953c5504821
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.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.network;
+
+import java.io.IOException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.security.auth.PrincipalBuilder;
+import org.apache.kafka.common.security.kerberos.KerberosNameParser;
+import org.apache.kafka.common.security.kerberos.LoginManager;
+import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
+import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator;
+import org.apache.kafka.common.security.ssl.SSLFactory;
+import org.apache.kafka.common.protocol.SecurityProtocol;
+import org.apache.kafka.common.config.SSLConfigs;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.common.KafkaException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SaslChannelBuilder implements ChannelBuilder {
+ private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class);
+
+ private final SecurityProtocol securityProtocol;
+ private final Mode mode;
+ private final LoginType loginType;
+
+ private LoginManager loginManager;
+ private PrincipalBuilder principalBuilder;
+ private SSLFactory sslFactory;
+ private Map configs;
+ private KerberosNameParser kerberosNameParser;
+
+ public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol) {
+ this.mode = mode;
+ this.loginType = loginType;
+ this.securityProtocol = securityProtocol;
+ }
+
+ public void configure(Map configs) throws KafkaException {
+ try {
+ this.configs = configs;
+ this.loginManager = LoginManager.acquireLoginManager(loginType, configs);
+ this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class>) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG));
+ this.principalBuilder.configure(configs);
+
+ String defaultRealm;
+ try {
+ defaultRealm = JaasUtils.defaultRealm();
+ } catch (Exception ke) {
+ defaultRealm = "";
+ }
+ kerberosNameParser = new KerberosNameParser(defaultRealm, (List) configs.get(SaslConfigs.AUTH_TO_LOCAL));
+
+ if (this.securityProtocol == SecurityProtocol.SASL_SSL) {
+ this.sslFactory = new SSLFactory(mode);
+ this.sslFactory.configure(this.configs);
+ }
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException {
+ try {
+ SocketChannel socketChannel = (SocketChannel) key.channel();
+ TransportLayer transportLayer = buildTransportLayer(id, key, socketChannel);
+ Authenticator authenticator;
+ if (mode == Mode.SERVER)
+ authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosNameParser);
+ else
+ authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(),
+ socketChannel.socket().getInetAddress().getHostName(), kerberosNameParser);
+ authenticator.configure(transportLayer, this.principalBuilder, this.configs);
+ return new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize);
+ } catch (Exception e) {
+ log.info("Failed to create channel due to ", e);
+ throw new KafkaException(e);
+ }
+ }
+
+ public void close() {
+ this.principalBuilder.close();
+ this.loginManager.release();
+ }
+
+ protected TransportLayer buildTransportLayer(String id, SelectionKey key, SocketChannel socketChannel) throws IOException {
+ if (this.securityProtocol == SecurityProtocol.SASL_SSL) {
+ return SSLTransportLayer.create(id, key,
+ sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(),
+ socketChannel.socket().getPort()));
+ } else {
+ return new PlaintextTransportLayer(key);
+ }
+ }
+
+}
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 e1e5b4a3e6192..34de616dd2425 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
@@ -200,6 +200,7 @@ public void close() {
log.error("Exception closing nioSelector:", se);
}
sensors.close();
+ channelBuilder.close();
}
/**
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 ff7a3bf85dd59..591fb8ded0d01 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
@@ -74,8 +74,7 @@ public interface TransportLayer extends ScatteringByteChannel, GatheringByteChan
boolean hasPendingWrites();
/**
- * returns SSLSession.getPeerPrinicpal if SSLTransportLayer used
- * for non-secure returns a "ANONYMOUS" as the peerPrincipal
+ * Returns `SSLSession.getPeerPrincipal` if SSLTransportLayer is used and `KakfaPrincipal.ANONYMOUS` otherwise.
*/
Principal peerPrincipal() throws IOException;
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 a624741bff35e..70130a4eb9849 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
@@ -26,6 +26,10 @@ public enum SecurityProtocol {
PLAINTEXT(0, "PLAINTEXT"),
/** SSL channel */
SSL(1, "SSL"),
+ /** SASL authenticated, non-encrypted channel */
+ SASL_PLAINTEXT(2, "SASL_PLAINTEXT"),
+ /** SASL authenticated, SSL channel */
+ SASL_SSL(3, "SASL_SSL"),
/** 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/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
index ce0be6205980f..b8c870dac17ad 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
@@ -14,26 +14,77 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.kafka.common.security;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.AppConfigurationEntry;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.io.IOException;
import java.io.File;
import java.net.URI;
import java.security.URIParameter;
-import javax.security.auth.login.Configuration;
+
import org.apache.kafka.common.KafkaException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class JaasUtils {
private static final Logger LOG = LoggerFactory.getLogger(JaasUtils.class);
+ public static final String JAVA_LOGIN_CONFIG_PARAM = "java.security.auth.login.config";
+
public static final String LOGIN_CONTEXT_SERVER = "KafkaServer";
public static final String LOGIN_CONTEXT_CLIENT = "KafkaClient";
public static final String SERVICE_NAME = "serviceName";
- public static final String JAVA_LOGIN_CONFIG_PARAM = "java.security.auth.login.config";
+
public static final String ZK_SASL_CLIENT = "zookeeper.sasl.client";
public static final String ZK_LOGIN_CONTEXT_NAME_KEY = "zookeeper.sasl.clientconfig";
+ /**
+ * Construct a JAAS configuration object per kafka jaas configuration file
+ * @param loginContextName
+ * @param key
+ * @return JAAS configuration object
+ */
+ public static String jaasConfig(String loginContextName, String key) throws IOException {
+ AppConfigurationEntry[] configurationEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+ if (configurationEntries == null) {
+ String errorMessage = "Could not find a '" + loginContextName + "' entry in this configuration.";
+ throw new IOException(errorMessage);
+ }
+
+ for (AppConfigurationEntry entry: configurationEntries) {
+ Object val = entry.getOptions().get(key);
+ if (val != null)
+ return (String) val;
+ }
+ return null;
+ }
+
+ public static String defaultRealm()
+ throws ClassNotFoundException, NoSuchMethodException,
+ IllegalArgumentException, IllegalAccessException,
+ InvocationTargetException {
+
+ //TODO Find a way to avoid using these proprietary classes as access to Java 9 will block access by default
+ //due to the Jigsaw module system
+
+ Object kerbConf;
+ Class> classRef;
+ Method getInstanceMethod;
+ Method getDefaultRealmMethod;
+ if (System.getProperty("java.vendor").contains("IBM")) {
+ classRef = Class.forName("com.ibm.security.krb5.internal.Config");
+ } else {
+ classRef = Class.forName("sun.security.krb5.Config");
+ }
+ getInstanceMethod = classRef.getMethod("getInstance", new Class[0]);
+ kerbConf = getInstanceMethod.invoke(classRef, new Object[0]);
+ getDefaultRealmMethod = classRef.getDeclaredMethod("getDefaultRealm",
+ new Class[0]);
+ return (String) getDefaultRealmMethod.invoke(kerbConf, new Object[0]);
+ }
+
public static boolean isZkSecurityEnabled(String loginConfigFile) {
boolean isSecurityEnabled = false;
boolean zkSaslEnabled = Boolean.getBoolean(System.getProperty(ZK_SASL_CLIENT, "true"));
@@ -61,4 +112,5 @@ public static boolean isZkSecurityEnabled(String loginConfigFile) {
return isSecurityEnabled;
}
-}
\ No newline at end of file
+}
+
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
index b7cc378bde73c..99b6d21aff3a2 100644
--- 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
@@ -17,10 +17,6 @@
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;
@@ -29,23 +25,24 @@
import java.util.Map;
import java.security.Principal;
+/*
+ * PrincipalBuilder for Authenticator
+ */
public interface PrincipalBuilder extends Configurable {
/**
- * configure this class with give key-value pair
+ * Configures this class with given key-value pairs.
*/
- public void configure(Map configs);
+ void configure(Map configs);
/**
- * Returns Principal
- * @param TransportLayer
- * @param Authenticator
+ * Returns Principal.
*/
Principal buildPrincipal(TransportLayer transportLayer, Authenticator authenticator) throws KafkaException;
/**
- * Close this PrincipalBuilder
+ * Closes this instance.
*/
- public void close() throws KafkaException;
+ void close() throws KafkaException;
}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
new file mode 100644
index 0000000000000..39291607886f7
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java
@@ -0,0 +1,259 @@
+/**
+ * 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.authenticator;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+import java.util.Arrays;
+import java.util.Map;
+
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.security.auth.Subject;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.NameCallback;
+import javax.security.auth.callback.PasswordCallback;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslClient;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.network.Authenticator;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.security.auth.PrincipalBuilder;
+import org.apache.kafka.common.KafkaException;
+
+import org.apache.kafka.common.security.kerberos.KerberosNameParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class SaslClientAuthenticator implements Authenticator {
+
+ public enum SaslState {
+ INITIAL, INTERMEDIATE, COMPLETE, FAILED
+ }
+
+ private static final Logger LOG = LoggerFactory.getLogger(SaslClientAuthenticator.class);
+
+ private final Subject subject;
+ private final String servicePrincipal;
+ private final String host;
+ private final String node;
+ private final KerberosNameParser kerberosNameParser;
+
+ // assigned in `configure`
+ private SaslClient saslClient;
+ private String clientPrincipalName;
+ private TransportLayer transportLayer;
+
+ // buffers used in `authenticate`
+ private NetworkReceive netInBuffer;
+ private NetworkSend netOutBuffer;
+
+ private SaslState saslState = SaslState.INITIAL;
+
+ public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host, KerberosNameParser kerberosNameParser) throws IOException {
+ this.node = node;
+ this.subject = subject;
+ this.host = host;
+ this.servicePrincipal = servicePrincipal;
+ this.kerberosNameParser = kerberosNameParser;
+ }
+
+ public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) throws KafkaException {
+ try {
+ this.transportLayer = transportLayer;
+
+ // determine client principal from subject.
+ Principal clientPrincipal = subject.getPrincipals().iterator().next();
+ this.clientPrincipalName = kerberosNameParser.parse(clientPrincipal.getName()).toString();
+ this.saslClient = createSaslClient();
+ } catch (Exception e) {
+ throw new KafkaException("Failed to configure SaslClientAuthenticator", e);
+ }
+ }
+
+ private SaslClient createSaslClient() {
+ try {
+ return Subject.doAs(subject, new PrivilegedExceptionAction() {
+ public SaslClient run() throws SaslException {
+ String[] mechs = {"GSSAPI"};
+ LOG.debug("Creating SaslClient: client={};service={};serviceHostname={};mechs={}",
+ clientPrincipalName, servicePrincipal, host, Arrays.toString(mechs));
+ return Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null,
+ new ClientCallbackHandler());
+ }
+ });
+ } catch (Exception e) {
+ throw new KafkaException("Failed to create SASL client", e);
+ }
+ }
+
+ /**
+ * Sends an empty message to the server to initiate the authentication process. It then evaluates server challenges
+ * via `SaslClient.evaluateChallenge` and returns client responses until authentication succeeds or fails.
+ *
+ * The messages are sent and received as size delimited bytes that consists of a 4 byte network-ordered size N
+ * followed by N bytes representing the opaque payload.
+ */
+ public void authenticate() throws IOException {
+ if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps())
+ return;
+
+ switch (saslState) {
+ case INITIAL:
+ sendSaslToken(new byte[0]);
+ saslState = SaslState.INTERMEDIATE;
+ break;
+ case INTERMEDIATE:
+ if (netInBuffer == null) netInBuffer = new NetworkReceive(node);
+ netInBuffer.readFrom(transportLayer);
+ if (netInBuffer.complete()) {
+ netInBuffer.payload().rewind();
+ byte[] serverToken = new byte[netInBuffer.payload().remaining()];
+ netInBuffer.payload().get(serverToken, 0, serverToken.length);
+ netInBuffer = null; // reset the networkReceive as we read all the data.
+ sendSaslToken(serverToken);
+ }
+ if (saslClient.isComplete()) {
+ saslState = SaslState.COMPLETE;
+ transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
+ }
+ break;
+ case COMPLETE:
+ break;
+ case FAILED:
+ throw new IOException("SASL handshake failed");
+ }
+ }
+
+ private void sendSaslToken(byte[] serverToken) throws IOException {
+ if (!saslClient.isComplete()) {
+ try {
+ byte[] saslToken = createSaslToken(serverToken);
+ if (saslToken != null) {
+ netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken));
+ flushNetOutBufferAndUpdateInterestOps();
+ }
+ } catch (SaslException se) {
+ saslState = SaslState.FAILED;
+ throw new IOException("Failed to authenticate using SASL " + se);
+ }
+ }
+ }
+
+ private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException {
+ boolean flushedCompletely = flushNetOutBuffer();
+ if (flushedCompletely)
+ transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
+ else
+ transportLayer.addInterestOps(SelectionKey.OP_WRITE);
+ return flushedCompletely;
+ }
+
+ public Principal principal() {
+ return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, clientPrincipalName);
+ }
+
+ public boolean complete() {
+ return saslState == SaslState.COMPLETE;
+ }
+
+ public void close() throws IOException {
+ saslClient.dispose();
+ }
+
+ private byte[] createSaslToken(final byte[] saslToken) throws SaslException {
+ if (saslToken == null) {
+ throw new SaslException("Error in authenticating with a Kafka Broker: the kafka broker saslToken is null.");
+ }
+
+ try {
+ return Subject.doAs(subject, new PrivilegedExceptionAction() {
+ public byte[] run() throws SaslException {
+ return saslClient.evaluateChallenge(saslToken);
+ }
+ });
+ } catch (PrivilegedActionException e) {
+ String error = "An error: (" + e + ") occurred when evaluating Kafka Brokers " +
+ " received SASL token.";
+ // Try to provide hints to use about what went wrong so they can fix their configuration.
+ // TODO: introspect about e: look for GSS information.
+ final String unknownServerErrorText =
+ "(Mechanism level: Server not found in Kerberos database (7) - UNKNOWN_SERVER)";
+ if (e.toString().indexOf(unknownServerErrorText) > -1) {
+ error += " This may be caused by Java's being unable to resolve the Kafka Broker's" +
+ " hostname correctly. You may want to try to adding" +
+ " '-Dsun.net.spi.nameservice.provider.1=dns,sun' to your client's JVMFLAGS environment." +
+ " Users must configure FQDN of kafka brokers when authenticating using SASL and" +
+ " `socketChannel.socket().getInetAddress().getHostName()` must match the hostname in `principal/hostname@realm`";
+ }
+ error += " Kafka Client will go to AUTH_FAILED state.";
+ throw new SaslException(error);
+ }
+ }
+
+ private boolean flushNetOutBuffer() throws IOException {
+ if (!netOutBuffer.completed()) {
+ netOutBuffer.writeTo(transportLayer);
+ }
+ return netOutBuffer.completed();
+ }
+
+ public static class ClientCallbackHandler implements CallbackHandler {
+
+ public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+ for (Callback callback : callbacks) {
+ if (callback instanceof NameCallback) {
+ NameCallback nc = (NameCallback) callback;
+ nc.setName(nc.getDefaultName());
+ } else if (callback instanceof PasswordCallback) {
+ // Call `setPassword` once we support obtaining a password from the user and update message below
+ LOG.warn("Could not login: the client is being asked for a password, but the Kafka" +
+ " client code does not currently support obtaining a password from the user." +
+ " Make sure -Djava.security.auth.login.config property passed to JVM and" +
+ " the client is configured to use a ticket cache (using" +
+ " the JAAS configuration setting 'useTicketCache=true)'. Make sure you are using" +
+ " FQDN of the Kafka broker you are trying to connect to.");
+ } else if (callback instanceof RealmCallback) {
+ RealmCallback rc = (RealmCallback) callback;
+ rc.setText(rc.getDefaultText());
+ } else if (callback instanceof AuthorizeCallback) {
+ AuthorizeCallback ac = (AuthorizeCallback) callback;
+ String authId = ac.getAuthenticationID();
+ String authzId = ac.getAuthorizationID();
+ ac.setAuthorized(authId.equals(authzId));
+ if (ac.isAuthorized())
+ ac.setAuthorizedID(authzId);
+ } else {
+ throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback");
+ }
+ }
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
new file mode 100644
index 0000000000000..d06a22a192139
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java
@@ -0,0 +1,203 @@
+/**
+ * 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.authenticator;
+
+import java.io.IOException;
+import java.util.Map;
+
+import java.nio.ByteBuffer;
+import java.nio.channels.SelectionKey;
+
+import java.security.Principal;
+import java.security.PrivilegedActionException;
+import java.security.PrivilegedExceptionAction;
+
+import javax.security.auth.login.Configuration;
+import javax.security.auth.Subject;
+import javax.security.sasl.Sasl;
+import javax.security.sasl.SaslServer;
+import javax.security.sasl.SaslException;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.security.kerberos.KerberosName;
+import org.apache.kafka.common.security.kerberos.KerberosNameParser;
+import org.ietf.jgss.GSSContext;
+import org.ietf.jgss.GSSCredential;
+import org.ietf.jgss.GSSException;
+import org.ietf.jgss.GSSManager;
+import org.ietf.jgss.GSSName;
+import org.ietf.jgss.Oid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kafka.common.security.auth.KafkaPrincipal;
+import org.apache.kafka.common.network.Authenticator;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.security.auth.PrincipalBuilder;
+
+public class SaslServerAuthenticator implements Authenticator {
+
+ private static final Logger LOG = LoggerFactory.getLogger(SaslServerAuthenticator.class);
+
+ private final SaslServer saslServer;
+ private final Subject subject;
+ private final String node;
+ private final KerberosNameParser kerberosNameParser;
+
+ // assigned in `configure`
+ private TransportLayer transportLayer;
+
+ // buffers used in `authenticate`
+ private NetworkReceive netInBuffer;
+ private NetworkSend netOutBuffer;
+
+ public SaslServerAuthenticator(String node, final Subject subject, KerberosNameParser kerberosNameParser) throws IOException {
+ if (subject == null)
+ throw new IllegalArgumentException("subject cannot be null");
+ if (subject.getPrincipals().isEmpty())
+ throw new IllegalArgumentException("subject must have at least one principal");
+ this.node = node;
+ this.subject = subject;
+ this.kerberosNameParser = kerberosNameParser;
+ saslServer = createSaslServer();
+ }
+
+ public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) {
+ this.transportLayer = transportLayer;
+ }
+
+ private SaslServer createSaslServer() throws IOException {
+ // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject.
+ final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler(
+ Configuration.getConfiguration(), kerberosNameParser);
+ final Principal servicePrincipal = subject.getPrincipals().iterator().next();
+ KerberosName kerberosName;
+ try {
+ kerberosName = kerberosNameParser.parse(servicePrincipal.getName());
+ } catch (IllegalArgumentException e) {
+ throw new KafkaException("Principal has name with unexpected format " + servicePrincipal);
+ }
+ final String servicePrincipalName = kerberosName.serviceName();
+ final String serviceHostname = kerberosName.hostName();
+
+ final String mech = "GSSAPI";
+
+ LOG.debug("Creating SaslServer for {} with mechanism {}", kerberosName, mech);
+
+ // As described in http://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/jgss-features.html:
+ // "To enable Java GSS to delegate to the native GSS library and its list of native mechanisms,
+ // set the system property "sun.security.jgss.native" to true"
+ // "In addition, when performing operations as a particular Subject, for example, Subject.doAs(...)
+ // or Subject.doAsPrivileged(...), the to-be-used GSSCredential should be added to Subject's
+ // private credential set. Otherwise, the GSS operations will fail since no credential is found."
+ boolean usingNativeJgss = Boolean.getBoolean("sun.security.jgss.native");
+ if (usingNativeJgss) {
+ try {
+ GSSManager manager = GSSManager.getInstance();
+ // This Oid is used to represent the Kerberos version 5 GSS-API mechanism. It is defined in
+ // RFC 1964.
+ Oid krb5Mechanism = new Oid("1.2.840.113554.1.2.2");
+ GSSName gssName = manager.createName(servicePrincipalName + "@" + serviceHostname, GSSName.NT_HOSTBASED_SERVICE);
+ GSSCredential cred = manager.createCredential(gssName, GSSContext.INDEFINITE_LIFETIME, krb5Mechanism, GSSCredential.ACCEPT_ONLY);
+ subject.getPrivateCredentials().add(cred);
+ } catch (GSSException ex) {
+ LOG.warn("Cannot add private credential to subject; clients authentication may fail", ex);
+ }
+ }
+
+ try {
+ return Subject.doAs(subject, new PrivilegedExceptionAction() {
+ public SaslServer run() {
+ try {
+ return Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler);
+ } catch (SaslException e) {
+ throw new KafkaException("Kafka Server failed to create a SaslServer to interact with a client during session authentication", e);
+ }
+ }
+ });
+ } catch (PrivilegedActionException e) {
+ throw new KafkaException("Kafka Broker experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context", e);
+ }
+ }
+
+ /**
+ * Evaluates client responses via `SaslServer.evaluateResponse` and returns the issued challenge to the client until
+ * authentication succeeds or fails.
+ *
+ * The messages are sent and received as size delimited bytes that consists of a 4 byte network-ordered size N
+ * followed by N bytes representing the opaque payload.
+ */
+ public void authenticate() throws IOException {
+ if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps())
+ return;
+
+ if (saslServer.isComplete()) {
+ transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
+ return;
+ }
+
+ if (netInBuffer == null) netInBuffer = new NetworkReceive(node);
+
+ netInBuffer.readFrom(transportLayer);
+
+ if (netInBuffer.complete()) {
+ netInBuffer.payload().rewind();
+ byte[] clientToken = new byte[netInBuffer.payload().remaining()];
+ netInBuffer.payload().get(clientToken, 0, clientToken.length);
+ netInBuffer = null; // reset the networkReceive as we read all the data.
+ try {
+ byte[] response = saslServer.evaluateResponse(clientToken);
+ if (response != null) {
+ netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response));
+ flushNetOutBufferAndUpdateInterestOps();
+ }
+ } catch (Exception e) {
+ throw new IOException(e);
+ }
+ }
+ }
+
+ public Principal principal() {
+ return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, saslServer.getAuthorizationID());
+ }
+
+ public boolean complete() {
+ return saslServer.isComplete();
+ }
+
+ public void close() throws IOException {
+ saslServer.dispose();
+ }
+
+ private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException {
+ boolean flushedCompletely = flushNetOutBuffer();
+ if (flushedCompletely)
+ transportLayer.removeInterestOps(SelectionKey.OP_WRITE);
+ else
+ transportLayer.addInterestOps(SelectionKey.OP_WRITE);
+ return flushedCompletely;
+ }
+
+ private boolean flushNetOutBuffer() throws IOException {
+ if (!netOutBuffer.completed())
+ netOutBuffer.writeTo(transportLayer);
+ return netOutBuffer.completed();
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java
new file mode 100644
index 0000000000000..8474faf491edd
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java
@@ -0,0 +1,81 @@
+/**
+ * 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.authenticator;
+
+import java.io.IOException;
+
+import org.apache.kafka.common.security.kerberos.KerberosNameParser;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import javax.security.auth.callback.Callback;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.callback.UnsupportedCallbackException;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.sasl.AuthorizeCallback;
+import javax.security.sasl.RealmCallback;
+
+import org.apache.kafka.common.security.kerberos.KerberosName;
+import org.apache.kafka.common.security.JaasUtils;
+
+public class SaslServerCallbackHandler implements CallbackHandler {
+ private static final Logger LOG = LoggerFactory.getLogger(SaslServerCallbackHandler.class);
+ private final KerberosNameParser kerberosNameParser;
+
+ public SaslServerCallbackHandler(Configuration configuration, KerberosNameParser kerberosNameParser) throws IOException {
+ AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(JaasUtils.LOGIN_CONTEXT_SERVER);
+ if (configurationEntries == null)
+ throw new IOException("Could not find a 'KafkaServer' entry in this configuration: Kafka Server cannot start.");
+ this.kerberosNameParser = kerberosNameParser;
+ }
+
+ public void handle(Callback[] callbacks) throws UnsupportedCallbackException {
+ for (Callback callback : callbacks) {
+ if (callback instanceof RealmCallback) {
+ handleRealmCallback((RealmCallback) callback);
+ } else if (callback instanceof AuthorizeCallback) {
+ handleAuthorizeCallback((AuthorizeCallback) callback);
+ }
+ }
+ }
+
+ private void handleRealmCallback(RealmCallback rc) {
+ LOG.trace("Client supplied realm: {} ", rc.getDefaultText());
+ rc.setText(rc.getDefaultText());
+ }
+
+ private void handleAuthorizeCallback(AuthorizeCallback ac) {
+ String authenticationID = ac.getAuthenticationID();
+ String authorizationID = ac.getAuthorizationID();
+
+ LOG.info("Successfully authenticated client: authenticationID={}; authorizationID={}.", authenticationID,
+ authorizationID);
+ ac.setAuthorized(true);
+
+ KerberosName kerberosName = kerberosNameParser.parse(authenticationID);
+ try {
+ String userName = kerberosName.shortName();
+ LOG.info("Setting authorizedID: {}", userName);
+ ac.setAuthorizedID(userName);
+ } catch (IOException e) {
+ LOG.error("Failed to set name based on Kerberos authentication rules.");
+ }
+ }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java
new file mode 100644
index 0000000000000..aef10db83ced1
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java
@@ -0,0 +1,114 @@
+/**
+ * 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.kerberos;
+
+import java.io.IOException;
+import java.util.List;
+
+public class KerberosName {
+
+ /** The first component of the name */
+ private final String serviceName;
+ /** The second component of the name. It may be null. */
+ private final String hostName;
+ /** The realm of the name. */
+ private final String realm;
+
+ /* Rules for the translation of the principal name into an operating system name */
+ private final List authToLocalRules;
+
+ /**
+ * Creates an instance of `KerberosName` with the provided parameters.
+ */
+ public KerberosName(String serviceName, String hostName, String realm, List authToLocalRules) {
+ if (serviceName == null)
+ throw new IllegalArgumentException("serviceName must not be null");
+ this.serviceName = serviceName;
+ this.hostName = hostName;
+ this.realm = realm;
+ this.authToLocalRules = authToLocalRules;
+ }
+
+ /**
+ * Put the name back together from the parts.
+ */
+ @Override
+ public String toString() {
+ StringBuilder result = new StringBuilder();
+ result.append(serviceName);
+ if (hostName != null) {
+ result.append('/');
+ result.append(hostName);
+ }
+ if (realm != null) {
+ result.append('@');
+ result.append(realm);
+ }
+ return result.toString();
+ }
+
+ /**
+ * Get the first component of the name.
+ * @return the first section of the Kerberos principal name
+ */
+ public String serviceName() {
+ return serviceName;
+ }
+
+ /**
+ * Get the second component of the name.
+ * @return the second section of the Kerberos principal name, and may be null
+ */
+ public String hostName() {
+ return hostName;
+ }
+
+ /**
+ * Get the realm of the name.
+ * @return the realm of the name, may be null
+ */
+ public String realm() {
+ return realm;
+ }
+
+ /**
+ * Get the translation of the principal name into an operating system
+ * user name.
+ * @return the short name
+ * @throws IOException
+ */
+ public String shortName() throws IOException {
+ String[] params;
+ if (hostName == null) {
+ // if it is already simple, just return it
+ if (realm == null)
+ return serviceName;
+ params = new String[]{realm, serviceName};
+ } else {
+ params = new String[]{realm, serviceName, hostName};
+ }
+ for (KerberosRule r : authToLocalRules) {
+ String result = r.apply(params);
+ if (result != null)
+ return result;
+ }
+ throw new NoMatchingRule("No rules applied to " + toString());
+ }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosNameParser.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosNameParser.java
new file mode 100644
index 0000000000000..95eb170e90599
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosNameParser.java
@@ -0,0 +1,103 @@
+/**
+ * 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.kerberos;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * This class implements parsing and handling of Kerberos principal names. In
+ * particular, it splits them apart and translates them down into local
+ * operating system names.
+ */
+public class KerberosNameParser {
+
+ /**
+ * A pattern that matches a Kerberos name with at most 3 components.
+ */
+ private static final Pattern NAME_PARSER = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)");
+
+ /**
+ * A pattern for parsing a auth_to_local rule.
+ */
+ private static final Pattern RULE_PARSER = Pattern.compile("((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?(s/([^/]*)/([^/]*)/(g)?)?))");
+
+ /**
+ * The list of translation rules.
+ */
+ private final List authToLocalRules;
+
+ public KerberosNameParser(String defaultRealm, List authToLocalRules) {
+ this.authToLocalRules = parseRules(defaultRealm, authToLocalRules);
+ }
+
+ /**
+ * Create a name from the full Kerberos principal name.
+ */
+ public KerberosName parse(String principalName) {
+ Matcher match = NAME_PARSER.matcher(principalName);
+ if (!match.matches()) {
+ if (principalName.contains("@")) {
+ throw new IllegalArgumentException("Malformed Kerberos name: " + principalName);
+ } else {
+ return new KerberosName(principalName, null, null, authToLocalRules);
+ }
+ } else {
+ return new KerberosName(match.group(1), match.group(3), match.group(4), authToLocalRules);
+ }
+ }
+
+ private static List parseRules(String defaultRealm, List rules) {
+ List result = new ArrayList<>();
+ for (String rule : rules) {
+ Matcher matcher = RULE_PARSER.matcher(rule);
+ if (!matcher.lookingAt()) {
+ throw new IllegalArgumentException("Invalid rule: " + rule);
+ }
+ if (rule.length() != matcher.end())
+ throw new IllegalArgumentException("Invalid rule: `" + rule + "`, unmatched substring: `" + rule.substring(matcher.end()) + "`");
+ if (matcher.group(2) != null) {
+ result.add(new KerberosRule(defaultRealm));
+ } else {
+ result.add(new KerberosRule(defaultRealm,
+ Integer.parseInt(matcher.group(4)),
+ matcher.group(5),
+ matcher.group(7),
+ matcher.group(9),
+ matcher.group(10),
+ "g".equals(matcher.group(11))));
+
+ }
+ }
+ return result;
+ }
+
+ public static class BadFormatString extends IOException {
+ BadFormatString(String msg) {
+ super(msg);
+ }
+ BadFormatString(String msg, Throwable err) {
+ super(msg, err);
+ }
+ }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java
new file mode 100644
index 0000000000000..c1789db400532
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java
@@ -0,0 +1,189 @@
+/**
+ * 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.kerberos;
+
+import java.io.IOException;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * An encoding of a rule for translating kerberos names.
+ */
+class KerberosRule {
+
+ /**
+ * A pattern that matches a string without '$' and then a single
+ * parameter with $n.
+ */
+ private static final Pattern PARAMETER_PATTERN = Pattern.compile("([^$]*)(\\$(\\d*))?");
+
+ /**
+ * A pattern that recognizes simple/non-simple names.
+ */
+ private static final Pattern NON_SIMPLE_PATTERN = Pattern.compile("[/@]");
+
+ private final String defaultRealm;
+ private final boolean isDefault;
+ private final int numOfComponents;
+ private final String format;
+ private final Pattern match;
+ private final Pattern fromPattern;
+ private final String toPattern;
+ private final boolean repeat;
+
+ KerberosRule(String defaultRealm) {
+ this.defaultRealm = defaultRealm;
+ isDefault = true;
+ numOfComponents = 0;
+ format = null;
+ match = null;
+ fromPattern = null;
+ toPattern = null;
+ repeat = false;
+ }
+
+ KerberosRule(String defaultRealm, int numOfComponents, String format, String match, String fromPattern,
+ String toPattern, boolean repeat) {
+ this.defaultRealm = defaultRealm;
+ isDefault = false;
+ this.numOfComponents = numOfComponents;
+ this.format = format;
+ this.match = match == null ? null : Pattern.compile(match);
+ this.fromPattern =
+ fromPattern == null ? null : Pattern.compile(fromPattern);
+ this.toPattern = toPattern;
+ this.repeat = repeat;
+ }
+
+ @Override
+ public String toString() {
+ StringBuilder buf = new StringBuilder();
+ if (isDefault) {
+ buf.append("DEFAULT");
+ } else {
+ buf.append("RULE:[");
+ buf.append(numOfComponents);
+ buf.append(':');
+ buf.append(format);
+ buf.append(']');
+ if (match != null) {
+ buf.append('(');
+ buf.append(match);
+ buf.append(')');
+ }
+ if (fromPattern != null) {
+ buf.append("s/");
+ buf.append(fromPattern);
+ buf.append('/');
+ buf.append(toPattern);
+ buf.append('/');
+ if (repeat) {
+ buf.append('g');
+ }
+ }
+ }
+ return buf.toString();
+ }
+
+ /**
+ * Replace the numbered parameters of the form $n where n is from 1 to
+ * the length of params. Normal text is copied directly and $n is replaced
+ * by the corresponding parameter.
+ * @param format the string to replace parameters again
+ * @param params the list of parameters
+ * @return the generated string with the parameter references replaced.
+ * @throws KerberosNameParser.BadFormatString
+ */
+ static String replaceParameters(String format,
+ String[] params) throws KerberosNameParser.BadFormatString {
+ Matcher match = PARAMETER_PATTERN.matcher(format);
+ int start = 0;
+ StringBuilder result = new StringBuilder();
+ while (start < format.length() && match.find(start)) {
+ result.append(match.group(1));
+ String paramNum = match.group(3);
+ if (paramNum != null) {
+ try {
+ int num = Integer.parseInt(paramNum);
+ if (num < 0 || num > params.length) {
+ throw new KerberosNameParser.BadFormatString("index " + num + " from " + format +
+ " is outside of the valid range 0 to " +
+ (params.length - 1));
+ }
+ result.append(params[num]);
+ } catch (NumberFormatException nfe) {
+ throw new KerberosNameParser.BadFormatString("bad format in username mapping in " +
+ paramNum, nfe);
+ }
+
+ }
+ start = match.end();
+ }
+ return result.toString();
+ }
+
+ /**
+ * Replace the matches of the from pattern in the base string with the value
+ * of the to string.
+ * @param base the string to transform
+ * @param from the pattern to look for in the base string
+ * @param to the string to replace matches of the pattern with
+ * @param repeat whether the substitution should be repeated
+ * @return
+ */
+ static String replaceSubstitution(String base, Pattern from, String to,
+ boolean repeat) {
+ Matcher match = from.matcher(base);
+ if (repeat) {
+ return match.replaceAll(to);
+ } else {
+ return match.replaceFirst(to);
+ }
+ }
+
+ /**
+ * Try to apply this rule to the given name represented as a parameter
+ * array.
+ * @param params first element is the realm, second and later elements are
+ * are the components of the name "a/b@FOO" -> {"FOO", "a", "b"}
+ * @return the short name if this rule applies or null
+ * @throws IOException throws if something is wrong with the rules
+ */
+ String apply(String[] params) throws IOException {
+ String result = null;
+ if (isDefault) {
+ if (defaultRealm.equals(params[0])) {
+ result = params[1];
+ }
+ } else if (params.length - 1 == numOfComponents) {
+ String base = replaceParameters(format, params);
+ if (match == null || match.matcher(base).matches()) {
+ if (fromPattern == null) {
+ result = base;
+ } else {
+ result = replaceSubstitution(base, fromPattern, toPattern, repeat);
+ }
+ }
+ }
+ if (result != null && NON_SIMPLE_PATTERN.matcher(result).find()) {
+ throw new NoMatchingRule("Non-simple name " + result + " after auth_to_local rule " + this);
+ }
+ return result;
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java
new file mode 100644
index 0000000000000..dd885e542f555
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java
@@ -0,0 +1,389 @@
+/**
+ * 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.kerberos;
+
+import javax.security.auth.kerberos.KerberosPrincipal;
+import javax.security.auth.login.AppConfigurationEntry;
+import javax.security.auth.login.Configuration;
+import javax.security.auth.login.LoginContext;
+import javax.security.auth.login.LoginException;
+import javax.security.auth.callback.CallbackHandler;
+import javax.security.auth.kerberos.KerberosTicket;
+import javax.security.auth.Subject;
+
+import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.ClientCallbackHandler;
+import org.apache.kafka.common.security.JaasUtils;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.utils.Shell;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.SystemTime;
+
+import org.apache.kafka.common.utils.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Date;
+import java.util.Random;
+import java.util.Set;
+import java.util.Map;
+
+/**
+ * This class is responsible for refreshing Kerberos credentials for
+ * logins for both Kafka client and server.
+ */
+public class Login {
+ private static final Logger log = LoggerFactory.getLogger(Login.class);
+
+ private static final Random RNG = new Random();
+
+ private final Thread t;
+ private final boolean isKrbTicket;
+ private final boolean isUsingTicketCache;
+
+ private final String loginContextName;
+ private final String principal;
+ private final Time time = new SystemTime();
+ private final CallbackHandler callbackHandler = new ClientCallbackHandler();
+
+ // LoginThread will sleep until 80% of time from last refresh to
+ // ticket's expiry has been reached, at which time it will wake
+ // and try to renew the ticket.
+ private final double ticketRenewWindowFactor;
+
+ /**
+ * Percentage of random jitter added to the renewal time
+ */
+ private final double ticketRenewJitter;
+
+ // Regardless of ticketRenewWindowFactor setting above and the ticket expiry time,
+ // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute).
+ // Change the '1' to e.g. 5, to change this to 5 minutes.
+ private final long minTimeBeforeRelogin;
+
+ private final String kinitCmd;
+
+ private volatile Subject subject;
+
+ private LoginContext login;
+ private long lastLogin;
+
+ /**
+ * Login constructor. The constructor starts the thread used
+ * to periodically re-login to the Kerberos Ticket Granting Server.
+ * @param loginContextName
+ * name of section in JAAS file that will be use to login.
+ * Passed as first param to javax.security.auth.login.LoginContext().
+ * @param configs configure Login with the given key-value pairs.
+ * @throws javax.security.auth.login.LoginException
+ * Thrown if authentication fails.
+ */
+ public Login(final String loginContextName, Map configs) throws LoginException {
+ this.loginContextName = loginContextName;
+ this.ticketRenewWindowFactor = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR);
+ this.ticketRenewJitter = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER);
+ this.minTimeBeforeRelogin = (Long) configs.get(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN);
+ this.kinitCmd = (String) configs.get(SaslConfigs.SASL_KERBEROS_KINIT_CMD);
+
+ this.lastLogin = currentElapsedTime();
+ login = login(loginContextName);
+ subject = login.getSubject();
+ isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty();
+
+ AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+ if (entries.length == 0) {
+ isUsingTicketCache = false;
+ principal = null;
+ } else {
+ // there will only be a single entry
+ AppConfigurationEntry entry = entries[0];
+ if (entry.getOptions().get("useTicketCache") != null) {
+ String val = (String) entry.getOptions().get("useTicketCache");
+ isUsingTicketCache = val.equals("true");
+ } else
+ isUsingTicketCache = false;
+ if (entry.getOptions().get("principal") != null)
+ principal = (String) entry.getOptions().get("principal");
+ else
+ principal = null;
+ }
+
+ if (!isKrbTicket) {
+ log.debug("It is not a Kerberos ticket");
+ t = null;
+ // if no TGT, do not bother with ticket management.
+ return;
+ }
+ log.debug("It is a Kerberos ticket");
+
+ // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the
+ // TGT's existing expiry date and the configured minTimeBeforeRelogin. For testing and development,
+ // you can decrease the interval of expiration of tickets (for example, to 3 minutes) by running:
+ // "modprinc -maxlife 3mins " in kadmin.
+ t = Utils.newThread("kafka-kerberos-refresh-thread", new Runnable() {
+ public void run() {
+ log.info("TGT refresh thread started.");
+ while (true) { // renewal thread's main loop. if it exits from here, thread will exit.
+ KerberosTicket tgt = getTGT();
+ long now = currentWallTime();
+ long nextRefresh;
+ Date nextRefreshDate;
+ if (tgt == null) {
+ nextRefresh = now + minTimeBeforeRelogin;
+ nextRefreshDate = new Date(nextRefresh);
+ log.warn("No TGT found: will try again at {}", nextRefreshDate);
+ } else {
+ nextRefresh = getRefreshTime(tgt);
+ long expiry = tgt.getEndTime().getTime();
+ Date expiryDate = new Date(expiry);
+ if (isUsingTicketCache && tgt.getRenewTill() != null && tgt.getRenewTill().getTime() >= expiry) {
+ log.error("The TGT cannot be renewed beyond the next expiry date: {}." +
+ "This process will not be able to authenticate new SASL connections after that " +
+ "time (for example, it will not be able to authenticate a new connection with a Kafka " +
+ "Broker). Ask your system administrator to either increase the " +
+ "'renew until' time by doing : 'modprinc -maxrenewlife {} ' within " +
+ "kadmin, or instead, to generate a keytab for {}. Because the TGT's " +
+ "expiry cannot be further extended by refreshing, exiting refresh thread now.",
+ expiryDate, principal, principal);
+ return;
+ }
+ // determine how long to sleep from looking at ticket's expiry.
+ // We should not allow the ticket to expire, but we should take into consideration
+ // minTimeBeforeRelogin. Will not sleep less than minTimeBeforeRelogin, unless doing so
+ // would cause ticket expiration.
+ if ((nextRefresh > expiry) || (now + minTimeBeforeRelogin > expiry)) {
+ // expiry is before next scheduled refresh).
+ log.info("Refreshing now because expiry is before next scheduled refresh time.");
+ nextRefresh = now;
+ } else {
+ if (nextRefresh < (now + minTimeBeforeRelogin)) {
+ // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN).
+ Date until = new Date(nextRefresh);
+ Date newUntil = new Date(now + minTimeBeforeRelogin);
+ log.warn("TGT refresh thread time adjusted from {} to {} since the former is sooner " +
+ "than the minimum refresh interval ({} seconds) from now.",
+ until, newUntil, minTimeBeforeRelogin / 1000);
+ }
+ nextRefresh = Math.max(nextRefresh, now + minTimeBeforeRelogin);
+ }
+ nextRefreshDate = new Date(nextRefresh);
+ if (nextRefresh > expiry) {
+ log.error("Next refresh: {} is later than expiry {}. This may indicate a clock skew problem." +
+ "Check that this host and the KDC hosts' clocks are in sync. Exiting refresh thread.",
+ nextRefreshDate, expiryDate);
+ return;
+ }
+ }
+ if (now < nextRefresh) {
+ Date until = new Date(nextRefresh);
+ log.info("TGT refresh sleeping until: {}", until);
+ try {
+ Thread.sleep(nextRefresh - now);
+ } catch (InterruptedException ie) {
+ log.warn("TGT renewal thread has been interrupted and will exit.");
+ return;
+ }
+ } else {
+ log.error("NextRefresh: {} is in the past: exiting refresh thread. Check"
+ + " clock sync between this host and KDC - (KDC's clock is likely ahead of this host)."
+ + " Manual intervention will be required for this client to successfully authenticate."
+ + " Exiting refresh thread.", nextRefreshDate);
+ return;
+ }
+ if (isUsingTicketCache) {
+ String kinitArgs = "-R";
+ int retry = 1;
+ while (retry >= 0) {
+ try {
+ log.debug("Running ticket cache refresh command: {} {}", kinitCmd, kinitArgs);
+ Shell.execCommand(kinitCmd, kinitArgs);
+ break;
+ } catch (Exception e) {
+ if (retry > 0) {
+ --retry;
+ // sleep for 10 seconds
+ try {
+ Thread.sleep(10 * 1000);
+ } catch (InterruptedException ie) {
+ log.error("Interrupted while renewing TGT, exiting Login thread");
+ return;
+ }
+ } else {
+ log.warn("Could not renew TGT due to problem running shell command: '" + kinitCmd
+ + " " + kinitArgs + "'" + "; exception was: " + e + ". Exiting refresh thread.", e);
+ return;
+ }
+ }
+ }
+ }
+ try {
+ int retry = 1;
+ while (retry >= 0) {
+ try {
+ reLogin();
+ break;
+ } catch (LoginException le) {
+ if (retry > 0) {
+ --retry;
+ // sleep for 10 seconds.
+ try {
+ Thread.sleep(10 * 1000);
+ } catch (InterruptedException e) {
+ log.error("Interrupted during login retry after LoginException:", le);
+ throw le;
+ }
+ } else {
+ log.error("Could not refresh TGT for principal: " + principal + ".", le);
+ }
+ }
+ }
+ } catch (LoginException le) {
+ log.error("Failed to refresh TGT: refresh thread exiting now.", le);
+ return;
+ }
+ }
+ }
+ }, false);
+ }
+
+ public void startThreadIfNeeded() {
+ // thread object 't' will be null if a refresh thread is not needed.
+ if (t != null) {
+ t.start();
+ }
+ }
+
+ public void shutdown() {
+ if ((t != null) && (t.isAlive())) {
+ t.interrupt();
+ try {
+ t.join();
+ } catch (InterruptedException e) {
+ log.warn("Error while waiting for Login thread to shutdown: " + e, e);
+ }
+ }
+ }
+
+ public Subject subject() {
+ return subject;
+ }
+
+ private synchronized LoginContext login(final String loginContextName) throws LoginException {
+ String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM);
+ if (jaasConfigFile == null) {
+ throw new IllegalArgumentException("You must pass " + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + " in secure mode.");
+ }
+
+ AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+ if (configEntries == null) {
+ // Forcing a reload of the configuration in case it's been overridden by third-party code.
+ // Without this, our tests fail sometimes depending on the order the tests are executed.
+ // Singletons are bad.
+ Configuration.setConfiguration(null);
+ configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName);
+ if (configEntries == null) {
+ String errorMessage = "Could not find a '" + loginContextName + "' entry in `" + jaasConfigFile + "`.";
+ throw new IllegalArgumentException(errorMessage);
+ } else {
+ log.info("Found `" + loginContextName + "` in JAAS configuration after forced reload.");
+ }
+ }
+
+ LoginContext loginContext = new LoginContext(loginContextName, callbackHandler);
+ loginContext.login();
+ log.info("Successfully logged in.");
+ return loginContext;
+ }
+
+ private long getRefreshTime(KerberosTicket tgt) {
+ long start = tgt.getStartTime().getTime();
+ long expires = tgt.getEndTime().getTime();
+ log.info("TGT valid starting at: {}", tgt.getStartTime());
+ log.info("TGT expires: {}", tgt.getEndTime());
+ long proposedRefresh = start + (long) ((expires - start) *
+ (ticketRenewWindowFactor + (ticketRenewJitter * RNG.nextDouble())));
+
+ if (proposedRefresh > expires)
+ // proposedRefresh is too far in the future: it's after ticket expires: simply return now.
+ return currentWallTime();
+ else
+ return proposedRefresh;
+ }
+
+ private synchronized KerberosTicket getTGT() {
+ Set tickets = subject.getPrivateCredentials(KerberosTicket.class);
+ for (KerberosTicket ticket : tickets) {
+ KerberosPrincipal server = ticket.getServer();
+ if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) {
+ log.debug("Found TGT {}.", ticket);
+ return ticket;
+ }
+ }
+ return null;
+ }
+
+ private boolean hasSufficientTimeElapsed() {
+ long now = currentElapsedTime();
+ if (now - lastLogin < minTimeBeforeRelogin) {
+ log.warn("Not attempting to re-login since the last re-login was attempted less than {} seconds before.",
+ minTimeBeforeRelogin / 1000);
+ return false;
+ }
+ return true;
+ }
+
+ /**
+ * Re-login a principal. This method assumes that {@link #login(String)} has happened already.
+ * @throws javax.security.auth.login.LoginException on a failure
+ */
+ private synchronized void reLogin()
+ throws LoginException {
+ if (!isKrbTicket) {
+ return;
+ }
+ if (login == null) {
+ throw new LoginException("Login must be done first");
+ }
+ if (!hasSufficientTimeElapsed()) {
+ return;
+ }
+ log.info("Initiating logout for {}", principal);
+ synchronized (Login.class) {
+ // register most recent relogin attempt
+ lastLogin = currentElapsedTime();
+ //clear up the kerberos state. But the tokens are not cleared! As per
+ //the Java kerberos login module code, only the kerberos credentials
+ //are cleared
+ login.logout();
+ //login and also update the subject field of this instance to
+ //have the new credentials (pass it to the LoginContext constructor)
+ login = new LoginContext(loginContextName, subject);
+ log.info("Initiating re-login for {}", principal);
+ login.login();
+ }
+ }
+
+ private long currentElapsedTime() {
+ return time.nanoseconds() / 1000000;
+ }
+
+ private long currentWallTime() {
+ return time.milliseconds();
+ }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java
new file mode 100644
index 0000000000000..18651c8611e0e
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java
@@ -0,0 +1,118 @@
+/**
+ * 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.kerberos;
+
+import javax.security.auth.Subject;
+import javax.security.auth.login.LoginException;
+import java.io.IOException;
+import java.util.EnumMap;
+import java.util.Map;
+
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.network.LoginType;
+import org.apache.kafka.common.security.JaasUtils;
+
+public class LoginManager {
+
+ private static final EnumMap CACHED_INSTANCES = new EnumMap(LoginType.class);
+
+ private final Login login;
+ private final String serviceName;
+ private final LoginType loginType;
+ private int refCount;
+
+ private LoginManager(LoginType loginType, Map configs) throws IOException, LoginException {
+ this.loginType = loginType;
+ String loginContext = loginType.contextName();
+ login = new Login(loginContext, configs);
+ this.serviceName = getServiceName(loginContext, configs);
+ login.startThreadIfNeeded();
+ }
+
+ private static String getServiceName(String loginContext, Map configs) throws IOException {
+ String jaasServiceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME);
+ String configServiceName = (String) configs.get(SaslConfigs.SASL_KERBEROS_SERVICE_NAME);
+ if (jaasServiceName != null && configServiceName != null && jaasServiceName != configServiceName) {
+ String message = "Conflicting serviceName values found in JAAS and Kafka configs " +
+ "value in JAAS file " + jaasServiceName + ", value in Kafka config " + configServiceName;
+ throw new IllegalArgumentException(message);
+ }
+
+ if (jaasServiceName != null)
+ return jaasServiceName;
+ if (configServiceName != null)
+ return configServiceName;
+
+ throw new IllegalArgumentException("No serviceName defined in either JAAS or Kafka config");
+ }
+
+ /**
+ * Returns an instance of `LoginManager` and increases its reference count.
+ *
+ * `release()` should be invoked when the `LoginManager` is no longer needed. This method will try to reuse an
+ * existing `LoginManager` for the provided `mode` if available. However, it expects `configs` to be the same for
+ * every invocation and it will ignore them in the case where it's returning a cached instance of `LoginManager`.
+ *
+ * This is a bit ugly and it would be nicer if we could pass the `LoginManager` to `ChannelBuilders.create` and
+ * shut it down when the broker or clients are closed. It's straightforward to do the former, but it's more
+ * complicated to do the latter without making the consumer API more complex.
+ *
+ * @param loginType the type of the login context, it should be SERVER for the broker and CLIENT for the clients
+ * (i.e. consumer and producer)
+ * @param configs configuration as key/value pairs
+ */
+ public static final LoginManager acquireLoginManager(LoginType loginType, Map configs) throws IOException, LoginException {
+ synchronized (LoginManager.class) {
+ LoginManager loginManager = CACHED_INSTANCES.get(loginType);
+ if (loginManager == null) {
+ loginManager = new LoginManager(loginType, configs);
+ CACHED_INSTANCES.put(loginType, loginManager);
+ }
+ return loginManager.acquire();
+ }
+ }
+
+ public Subject subject() {
+ return login.subject();
+ }
+
+ public String serviceName() {
+ return serviceName;
+ }
+
+ private LoginManager acquire() {
+ ++refCount;
+ return this;
+ }
+
+ /**
+ * Decrease the reference count for this instance and release resources if it reaches 0.
+ */
+ public void release() {
+ synchronized (LoginManager.class) {
+ if (refCount == 0)
+ throw new IllegalStateException("release called on LoginManager with refCount == 0");
+ else if (refCount == 1) {
+ CACHED_INSTANCES.remove(loginType);
+ login.shutdown();
+ }
+ --refCount;
+ }
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java
new file mode 100644
index 0000000000000..6c2d2677c31d7
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java
@@ -0,0 +1,27 @@
+/**
+ * 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.kerberos;
+
+import java.io.IOException;
+
+public class NoMatchingRule extends IOException {
+ NoMatchingRule(String msg) {
+ super(msg);
+ }
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
index b291409e1df06..163b8c677b5c5 100644
--- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
+++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SSLFactory.java
@@ -19,19 +19,19 @@
import org.apache.kafka.common.Configurable;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.config.SSLConfigs;
+import org.apache.kafka.common.network.Mode;
import javax.net.ssl.*;
import java.io.FileInputStream;
import java.io.IOException;
import java.security.GeneralSecurityException;
import java.security.KeyStore;
+
import java.util.List;
import java.util.Map;
-
public class SSLFactory implements Configurable {
- public enum Mode { CLIENT, SERVER };
private String protocol;
private String provider;
private String kmfAlgorithm;
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Shell.java b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java
new file mode 100644
index 0000000000000..f5db5c333be29
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java
@@ -0,0 +1,304 @@
+/**
+ * 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.utils;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A base class for running a Unix command.
+ *
+ * Shell can be used to run unix commands like du or
+ * df.
+ */
+abstract public class Shell {
+
+ private static final Logger LOG = LoggerFactory.getLogger(Shell.class);
+
+ /** Return an array containing the command name and its parameters */
+ protected abstract String[] execString();
+
+ /** Parse the execution result */
+ protected abstract void parseExecResult(BufferedReader lines) throws IOException;
+
+ private final long timeout;
+
+ private int exitCode;
+ private Process process; // sub process used to execute the command
+
+ /* If or not script finished executing */
+ private volatile AtomicBoolean completed;
+
+ /**
+ * @param timeout Specifies the time in milliseconds, after which the command will be killed. -1 means no timeout.
+ */
+ public Shell(long timeout) {
+ this.timeout = timeout;
+ }
+
+ /** get the exit code
+ * @return the exit code of the process
+ */
+ public int exitCode() {
+ return exitCode;
+ }
+
+ /** get the current sub-process executing the given command
+ * @return process executing the command
+ */
+ public Process process() {
+ return process;
+ }
+
+ protected void run() throws IOException {
+ exitCode = 0; // reset for next run
+ runCommand();
+ }
+
+ /** Run a command */
+ private void runCommand() throws IOException {
+ ProcessBuilder builder = new ProcessBuilder(execString());
+ Timer timeoutTimer = null;
+ completed = new AtomicBoolean(false);
+
+ process = builder.start();
+ if (timeout > -1) {
+ timeoutTimer = new Timer();
+ //One time scheduling.
+ timeoutTimer.schedule(new ShellTimeoutTimerTask(this), timeout);
+ }
+ final BufferedReader errReader = new BufferedReader(new InputStreamReader(process.getErrorStream()));
+ BufferedReader inReader = new BufferedReader(new InputStreamReader(process.getInputStream()));
+ final StringBuffer errMsg = new StringBuffer();
+
+ // read error and input streams as this would free up the buffers
+ // free the error stream buffer
+ Thread errThread = Utils.newThread("kafka-shell-thread", new Runnable() {
+ @Override
+ public void run() {
+ try {
+ String line = errReader.readLine();
+ while ((line != null) && !Thread.currentThread().isInterrupted()) {
+ errMsg.append(line);
+ errMsg.append(System.getProperty("line.separator"));
+ line = errReader.readLine();
+ }
+ } catch (IOException ioe) {
+ LOG.warn("Error reading the error stream", ioe);
+ }
+ }
+ }, false);
+ errThread.start();
+
+ try {
+ parseExecResult(inReader); // parse the output
+ // clear the input stream buffer
+ String line = null;
+ while (line != null) {
+ line = inReader.readLine();
+ }
+ // wait for the process to finish and check the exit code
+ exitCode = process.waitFor();
+ try {
+ // make sure that the error thread exits
+ errThread.join();
+ } catch (InterruptedException ie) {
+ LOG.warn("Interrupted while reading the error stream", ie);
+ }
+ completed.set(true);
+ //the timeout thread handling
+ //taken care in finally block
+ if (exitCode != 0) {
+ throw new ExitCodeException(exitCode, errMsg.toString());
+ }
+ } catch (InterruptedException ie) {
+ throw new IOException(ie.toString());
+ } finally {
+ if (timeoutTimer != null)
+ timeoutTimer.cancel();
+
+ // close the input stream
+ try {
+ inReader.close();
+ } catch (IOException ioe) {
+ LOG.warn("Error while closing the input stream", ioe);
+ }
+ if (!completed.get())
+ errThread.interrupt();
+
+ try {
+ errReader.close();
+ } catch (IOException ioe) {
+ LOG.warn("Error while closing the error stream", ioe);
+ }
+
+ process.destroy();
+ }
+ }
+
+
+ /**
+ * This is an IOException with exit code added.
+ */
+ @SuppressWarnings("serial")
+ public static class ExitCodeException extends IOException {
+ int exitCode;
+
+ public ExitCodeException(int exitCode, String message) {
+ super(message);
+ this.exitCode = exitCode;
+ }
+
+ public int getExitCode() {
+ return exitCode;
+ }
+ }
+
+ /**
+ * A simple shell command executor.
+ *
+ * ShellCommandExecutorshould be used in cases where the output
+ * of the command needs no explicit parsing and where the command, working
+ * directory and the environment remains unchanged. The output of the command
+ * is stored as-is and is expected to be small.
+ */
+ public static class ShellCommandExecutor extends Shell {
+
+ private final String[] command;
+ private StringBuffer output;
+
+ /**
+ * Create a new instance of the ShellCommandExecutor to execute a command.
+ *
+ * @param execString The command to execute with arguments
+ * @param timeout Specifies the time in milliseconds, after which the
+ * command will be killed. -1 means no timeout.
+ */
+
+ public ShellCommandExecutor(String[] execString, long timeout) {
+ super(timeout);
+ command = execString.clone();
+ }
+
+
+ /** Execute the shell command. */
+ public void execute() throws IOException {
+ this.run();
+ }
+
+ protected String[] execString() {
+ return command;
+ }
+
+ protected void parseExecResult(BufferedReader reader) throws IOException {
+ output = new StringBuffer();
+ char[] buf = new char[512];
+ int nRead;
+ while ((nRead = reader.read(buf, 0, buf.length)) > 0) {
+ output.append(buf, 0, nRead);
+ }
+ }
+
+ /** Get the output of the shell command.*/
+ public String output() {
+ return (output == null) ? "" : output.toString();
+ }
+
+ /**
+ * Returns the commands of this instance.
+ * Arguments with spaces in are presented with quotes round; other
+ * arguments are presented raw
+ *
+ * @return a string representation of the object.
+ */
+ public String toString() {
+ StringBuilder builder = new StringBuilder();
+ String[] args = execString();
+ for (String s : args) {
+ if (s.indexOf(' ') >= 0) {
+ builder.append('"').append(s).append('"');
+ } else {
+ builder.append(s);
+ }
+ builder.append(' ');
+ }
+ return builder.toString();
+ }
+ }
+
+ /**
+ * Static method to execute a shell command.
+ * Covers most of the simple cases without requiring the user to implement
+ * the Shell interface.
+ * @param cmd shell command to execute.
+ * @return the output of the executed command.
+ */
+ public static String execCommand(String ... cmd) throws IOException {
+ return execCommand(cmd, -1);
+ }
+
+ /**
+ * Static method to execute a shell command.
+ * Covers most of the simple cases without requiring the user to implement
+ * the Shell interface.
+ * @param cmd shell command to execute.
+ * @param timeout time in milliseconds after which script should be killed. -1 means no timeout.
+ * @return the output of the executed command.
+ */
+ public static String execCommand(String[] cmd, long timeout) throws IOException {
+ ShellCommandExecutor exec = new ShellCommandExecutor(cmd, timeout);
+ exec.execute();
+ return exec.output();
+ }
+
+ /**
+ * Timer which is used to timeout scripts spawned off by shell.
+ */
+ private static class ShellTimeoutTimerTask extends TimerTask {
+
+ private final Shell shell;
+
+ public ShellTimeoutTimerTask(Shell shell) {
+ this.shell = shell;
+ }
+
+ @Override
+ public void run() {
+ Process p = shell.process();
+ try {
+ p.exitValue();
+ } catch (Exception e) {
+ //Process has not terminated.
+ //So check if it has completed
+ //if not just destroy it.
+ if (p != null && !shell.completed.get()) {
+ p.destroy();
+ }
+ }
+ }
+ }
+
+}
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Time.java b/clients/src/main/java/org/apache/kafka/common/utils/Time.java
index 66c44de74521e..b2fad7fa96ecd 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Time.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Time.java
@@ -24,16 +24,16 @@ public interface Time {
/**
* The current time in milliseconds
*/
- public long milliseconds();
+ long milliseconds();
/**
* The current time in nanoseconds
*/
- public long nanoseconds();
+ long nanoseconds();
/**
* Sleep for the given number of milliseconds
*/
- public void sleep(long ms);
+ void sleep(long ms);
}
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 f13c21ab94317..4a6d30469b436 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
@@ -45,7 +45,7 @@ public EchoServer(Map configs) throws Exception {
this.protocol = configs.containsKey("security.protocol") ?
SecurityProtocol.valueOf((String) configs.get("security.protocol")) : SecurityProtocol.PLAINTEXT;
if (protocol == SecurityProtocol.SSL) {
- this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER);
+ this.sslFactory = new SSLFactory(Mode.SERVER);
this.sslFactory.configure(configs);
SSLContext sslContext = this.sslFactory.sslContext();
this.serverSocket = sslContext.getServerSocketFactory().createServerSocket(0);
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 c60053fd24ca0..6475ff0861e39 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
@@ -22,7 +22,6 @@
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.config.SSLConfigs;
-import org.apache.kafka.common.security.ssl.SSLFactory;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.test.TestSSLUtils;
import org.junit.After;
@@ -40,15 +39,15 @@ public class SSLSelectorTest extends SelectorTest {
public void setup() throws Exception {
File trustStoreFile = File.createTempFile("truststore", ".jks");
- Map sslServerConfigs = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.SERVER, trustStoreFile, "server");
+ Map sslServerConfigs = TestSSLUtils.createSSLConfig(false, true, Mode.SERVER, trustStoreFile, "server");
sslServerConfigs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS));
this.server = new EchoServer(sslServerConfigs);
this.server.start();
this.time = new MockTime();
- Map sslClientConfigs = TestSSLUtils.createSSLConfig(false, false, SSLFactory.Mode.SERVER, trustStoreFile, "client");
+ Map sslClientConfigs = TestSSLUtils.createSSLConfig(false, false, Mode.SERVER, trustStoreFile, "client");
sslClientConfigs.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS));
- this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT);
+ this.channelBuilder = new SSLChannelBuilder(Mode.CLIENT);
this.channelBuilder.configure(sslClientConfigs);
this.metrics = new Metrics();
this.selector = new Selector(5000, metrics, time, "MetricGroup", new LinkedHashMap(), channelBuilder);
diff --git a/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java
index 6993f5242bd1c..987f4bb96b688 100644
--- a/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/SSLTransportLayerTest.java
@@ -73,7 +73,7 @@ public void setup() throws Exception {
sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores);
sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores);
- this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT);
+ this.channelBuilder = new SSLChannelBuilder(Mode.CLIENT);
this.channelBuilder.configure(sslClientConfigs);
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder);
}
@@ -270,7 +270,7 @@ public void testClientAuthenticationRequestedNotProvided() throws Exception {
*/
@Test
public void testInvalidTruststorePassword() throws Exception {
- SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT);
+ SSLChannelBuilder channelBuilder = new SSLChannelBuilder(Mode.CLIENT);
try {
sslClientConfigs.put(SSLConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, "invalid");
channelBuilder.configure(sslClientConfigs);
@@ -285,7 +285,7 @@ public void testInvalidTruststorePassword() throws Exception {
*/
@Test
public void testInvalidKeystorePassword() throws Exception {
- SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT);
+ SSLChannelBuilder channelBuilder = new SSLChannelBuilder(Mode.CLIENT);
try {
sslClientConfigs.put(SSLConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, "invalid");
channelBuilder.configure(sslClientConfigs);
@@ -437,7 +437,7 @@ private void createSelector(Map sslClientConfigs) {
private void createSelector(Map sslClientConfigs, final Integer netReadBufSize, final Integer netWriteBufSize, final Integer appBufSize) {
- this.channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.CLIENT) {
+ this.channelBuilder = new SSLChannelBuilder(Mode.CLIENT) {
@Override
protected SSLTransportLayer buildTransportLayer(SSLFactory sslFactory, String id, SelectionKey key) throws IOException {
@@ -461,7 +461,7 @@ private static class CertStores {
CertStores(boolean server) throws Exception {
String name = server ? "server" : "client";
- SSLFactory.Mode mode = server ? SSLFactory.Mode.SERVER : SSLFactory.Mode.CLIENT;
+ Mode mode = server ? Mode.SERVER : Mode.CLIENT;
File truststoreFile = File.createTempFile(name + "TS", ".jks");
sslConfig = TestSSLUtils.createSSLConfig(!server, true, mode, truststoreFile, name);
sslConfig.put(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SSLConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS));
@@ -549,7 +549,7 @@ private class SSLEchoServer extends Thread {
private final ConcurrentLinkedQueue inflightSends = new ConcurrentLinkedQueue();
public SSLEchoServer(Map configs, String serverHost) throws Exception {
- this.sslFactory = new SSLFactory(SSLFactory.Mode.SERVER);
+ this.sslFactory = new SSLFactory(Mode.SERVER);
this.sslFactory.configure(configs);
serverSocketChannel = ServerSocketChannel.open();
serverSocketChannel.configureBlocking(false);
@@ -557,7 +557,7 @@ public SSLEchoServer(Map configs, String serverHost) throws Exception
this.port = serverSocketChannel.socket().getLocalPort();
this.socketChannels = Collections.synchronizedList(new ArrayList());
this.newChannels = Collections.synchronizedList(new ArrayList());
- SSLChannelBuilder channelBuilder = new SSLChannelBuilder(SSLFactory.Mode.SERVER);
+ SSLChannelBuilder channelBuilder = new SSLChannelBuilder(Mode.SERVER);
channelBuilder.configure(sslServerConfigs);
this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder);
setName("echoserver");
diff --git a/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java
new file mode 100644
index 0000000000000..9781f6d159d52
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java
@@ -0,0 +1,59 @@
+/**
+ * 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.kerberos;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class KerberosNameTest {
+
+ @Test
+ public void testParse() throws IOException {
+ List rules = new ArrayList<>(Arrays.asList(
+ "RULE:[1:$1](App\\..*)s/App\\.(.*)/$1/g",
+ "RULE:[2:$1](App\\..*)s/App\\.(.*)/$1/g",
+ "DEFAULT"
+ ));
+ KerberosNameParser parser = new KerberosNameParser("REALM.COM", rules);
+
+ KerberosName name = parser.parse("App.service-name/example.com@REALM.COM");
+ assertEquals("App.service-name", name.serviceName());
+ assertEquals("example.com", name.hostName());
+ assertEquals("REALM.COM", name.realm());
+ assertEquals("service-name", name.shortName());
+
+ name = parser.parse("App.service-name@REALM.COM");
+ assertEquals("App.service-name", name.serviceName());
+ assertNull(name.hostName());
+ assertEquals("REALM.COM", name.realm());
+ assertEquals("service-name", name.shortName());
+
+ name = parser.parse("user/host@REALM.COM");
+ assertEquals("user", name.serviceName());
+ assertEquals("host", name.hostName());
+ assertEquals("REALM.COM", name.realm());
+ assertEquals("user", name.shortName());
+ }
+}
diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java
index 0aec666b20bf0..e90ec2bb6abb8 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/SSLFactoryTest.java
@@ -18,6 +18,7 @@
import java.util.Map;
import org.apache.kafka.test.TestSSLUtils;
+import org.apache.kafka.common.network.Mode;
import org.junit.Test;
import static org.junit.Assert.assertNotNull;
@@ -35,8 +36,8 @@ public class SSLFactoryTest {
@Test
public void testSSLFactoryConfiguration() throws Exception {
File trustStoreFile = File.createTempFile("truststore", ".jks");
- Map serverSSLConfig = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.SERVER, trustStoreFile, "server");
- SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.SERVER);
+ Map serverSSLConfig = TestSSLUtils.createSSLConfig(false, true, Mode.SERVER, trustStoreFile, "server");
+ SSLFactory sslFactory = new SSLFactory(Mode.SERVER);
sslFactory.configure(serverSSLConfig);
//host and port are hints
SSLEngine engine = sslFactory.createSSLEngine("localhost", 0);
@@ -49,8 +50,8 @@ public void testSSLFactoryConfiguration() throws Exception {
@Test
public void testClientMode() throws Exception {
File trustStoreFile = File.createTempFile("truststore", ".jks");
- Map clientSSLConfig = TestSSLUtils.createSSLConfig(false, true, SSLFactory.Mode.CLIENT, trustStoreFile, "client");
- SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT);
+ Map clientSSLConfig = TestSSLUtils.createSSLConfig(false, true, Mode.CLIENT, trustStoreFile, "client");
+ SSLFactory sslFactory = new SSLFactory(Mode.CLIENT);
sslFactory.configure(clientSSLConfig);
//host and port are hints
SSLEngine engine = sslFactory.createSSLEngine("localhost", 0);
diff --git a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
index eb7fcf07e529e..387e48f1ab33b 100644
--- a/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.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.
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 c01cf3748ff16..b23169297e365 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java
@@ -18,7 +18,7 @@
package org.apache.kafka.test;
import org.apache.kafka.common.config.SSLConfigs;
-import org.apache.kafka.common.security.ssl.SSLFactory;
+import org.apache.kafka.common.network.Mode;
import org.apache.kafka.clients.CommonClientConfigs;
import java.io.File;
@@ -177,13 +177,13 @@ public static Map createX509Certificates(KeyPair keyPai
return certs;
}
- public static Map createSSLConfig(SSLFactory.Mode mode, File keyStoreFile, String password, String keyPassword,
+ public static Map createSSLConfig(Mode mode, File keyStoreFile, String password, String keyPassword,
File trustStoreFile, String trustStorePassword) {
Map sslConfigs = new HashMap();
sslConfigs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); // kafka security protocol
sslConfigs.put(SSLConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2"); // protocol to create SSLContext
- if (mode == SSLFactory.Mode.SERVER || (mode == SSLFactory.Mode.CLIENT && keyStoreFile != null)) {
+ if (mode == Mode.SERVER || (mode == Mode.CLIENT && keyStoreFile != null)) {
sslConfigs.put(SSLConfigs.SSL_KEYSTORE_LOCATION_CONFIG, keyStoreFile.getPath());
sslConfigs.put(SSLConfigs.SSL_KEYSTORE_TYPE_CONFIG, "JKS");
sslConfigs.put(SSLConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, TrustManagerFactory.getDefaultAlgorithm());
@@ -203,13 +203,13 @@ public static Map createSSLConfig(SSLFactory.Mode mode, File key
return sslConfigs;
}
- public static Map createSSLConfig(boolean useClientCert, boolean trustStore, SSLFactory.Mode mode, File trustStoreFile, String certAlias)
+ public static Map createSSLConfig(boolean useClientCert, boolean trustStore, Mode mode, File trustStoreFile, String certAlias)
throws IOException, GeneralSecurityException {
Map certs = new HashMap();
File keyStoreFile;
String password;
- if (mode == SSLFactory.Mode.SERVER)
+ if (mode == Mode.SERVER)
password = "ServerPassword";
else
password = "ClientPassword";
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index c2076a20fead3..37568227ab62d 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -21,10 +21,9 @@ import kafka.utils._
import org.apache.kafka.clients.{ClientResponse, ClientRequest, ManualMetadataUpdater, NetworkClient}
import org.apache.kafka.common.{TopicPartition, Node}
import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.network.{Selectable, ChannelBuilders, Selector, NetworkReceive}
+import org.apache.kafka.common.network.{LoginType, Selectable, ChannelBuilders, Selector, NetworkReceive, Mode}
import org.apache.kafka.common.protocol.{SecurityProtocol, ApiKeys}
import org.apache.kafka.common.requests._
-import org.apache.kafka.common.security.ssl.SSLFactory
import org.apache.kafka.common.utils.Time
import collection.mutable.HashMap
import kafka.cluster.Broker
@@ -97,7 +96,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf
"controller-channel",
Map("broker-id" -> broker.id.toString).asJava,
false,
- ChannelBuilders.create(config.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, config.channelConfigs)
+ ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.channelConfigs)
)
new NetworkClient(
selector,
diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala
index 41a3705e62994..1066fbee1a5c0 100644
--- a/core/src/main/scala/kafka/network/SocketServer.scala
+++ b/core/src/main/scala/kafka/network/SocketServer.scala
@@ -33,9 +33,8 @@ import kafka.server.KafkaConfig
import kafka.utils._
import org.apache.kafka.common.MetricName
import org.apache.kafka.common.metrics._
-import org.apache.kafka.common.network.{Selector => KSelector, ChannelBuilders, InvalidReceiveException}
+import org.apache.kafka.common.network.{Selector => KSelector, LoginType, Mode, ChannelBuilders, InvalidReceiveException}
import org.apache.kafka.common.security.auth.KafkaPrincipal
-import org.apache.kafka.common.security.ssl.SSLFactory
import org.apache.kafka.common.protocol.SecurityProtocol
import org.apache.kafka.common.protocol.types.SchemaException
import org.apache.kafka.common.utils.{Time, Utils}
@@ -378,7 +377,7 @@ private[kafka] class Processor(val id: Int,
private val newConnections = new ConcurrentLinkedQueue[SocketChannel]()
private val inflightResponses = mutable.Map[String, RequestChannel.Response]()
- private val channelBuilder = ChannelBuilders.create(protocol, SSLFactory.Mode.SERVER, channelConfigs)
+ private val channelBuilder = ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs)
private val metricTags = new util.HashMap[String, String]()
metricTags.put("networkProcessor", id.toString)
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index 913d49bbb3cfa..194ee9cb80917 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -26,6 +26,8 @@ import kafka.consumer.ConsumerConfig
import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet}
import kafka.utils.CoreUtils
import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.common.config.SaslConfigs
+
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, SSLConfigs}
import org.apache.kafka.common.metrics.MetricsReporter
import org.apache.kafka.common.protocol.SecurityProtocol
@@ -33,7 +35,6 @@ import org.apache.kafka.common.security.auth.PrincipalBuilder
import scala.collection.{Map, immutable}
-
object Defaults {
/** ********* Zookeeper Configuration ***********/
val ZkSessionTimeoutMs = 6000
@@ -168,6 +169,14 @@ object Defaults {
val SSLClientAuthNone = "none"
val SSLClientAuth = SSLClientAuthNone
val SSLCipherSuites = ""
+
+ /** ********* Sasl configuration ***********/
+ val SaslKerberosKinitCmd = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD
+ val SaslKerberosTicketRenewWindowFactor = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR
+ val SaslKerberosTicketRenewJitter = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER
+ val SaslKerberosMinTimeBeforeRelogin = SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN
+ val AuthToLocal = SaslConfigs.DEFAULT_AUTH_TO_LOCAL
+
}
object KafkaConfig {
@@ -316,6 +325,13 @@ object KafkaConfig {
val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG
val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_CONFIG
+ /** ********* SASL Configuration ****************/
+ val SaslKerberosServiceNameProp = SaslConfigs.SASL_KERBEROS_SERVICE_NAME
+ val SaslKerberosKinitCmdProp = SaslConfigs.SASL_KERBEROS_KINIT_CMD
+ val SaslKerberosTicketRenewWindowFactorProp = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR
+ val SaslKerberosTicketRenewJitterProp = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER
+ val SaslKerberosMinTimeBeforeReloginProp = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN
+ val AuthToLocalProp = SaslConfigs.AUTH_TO_LOCAL
/* Documentation */
/** ********* Zookeeper Configuration ***********/
@@ -487,6 +503,14 @@ object KafkaConfig {
val SSLEndpointIdentificationAlgorithmDoc = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC
val SSLClientAuthDoc = SSLConfigs.SSL_CLIENT_AUTH_DOC
+ /** ********* Sasl Configuration ****************/
+ val SaslKerberosServiceNameDoc = SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC
+ val SaslKerberosKinitCmdDoc = SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC
+ val SaslKerberosTicketRenewWindowFactorDoc = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC
+ val SaslKerberosTicketRenewJitterDoc = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC
+ val SaslKerberosMinTimeBeforeReloginDoc = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC
+ val AuthToLocalDoc = SaslConfigs.AUTH_TO_LOCAL_DOC
+
private val configDef = {
import ConfigDef.Importance._
import ConfigDef.Range._
@@ -642,6 +666,15 @@ object KafkaConfig {
.define(SSLTrustManagerAlgorithmProp, STRING, Defaults.SSLTrustManagerAlgorithm, MEDIUM, SSLTrustManagerAlgorithmDoc)
.define(SSLClientAuthProp, STRING, Defaults.SSLClientAuth, in(Defaults.SSLClientAuthRequired, Defaults.SSLClientAuthRequested, Defaults.SSLClientAuthNone), MEDIUM, SSLClientAuthDoc)
.define(SSLCipherSuitesProp, LIST, Defaults.SSLCipherSuites, MEDIUM, SSLCipherSuitesDoc)
+
+ /** ********* Sasl Configuration ****************/
+ .define(SaslKerberosServiceNameProp, STRING, MEDIUM, SaslKerberosServiceNameDoc, false)
+ .define(SaslKerberosKinitCmdProp, STRING, Defaults.SaslKerberosKinitCmd, MEDIUM, SaslKerberosKinitCmdDoc)
+ .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SaslKerberosTicketRenewWindowFactor, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc)
+ .define(SaslKerberosTicketRenewJitterProp, DOUBLE, Defaults.SaslKerberosTicketRenewJitter, MEDIUM, SaslKerberosTicketRenewJitterDoc)
+ .define(SaslKerberosMinTimeBeforeReloginProp, LONG, Defaults.SaslKerberosMinTimeBeforeRelogin, MEDIUM, SaslKerberosMinTimeBeforeReloginDoc)
+ .define(AuthToLocalProp, LIST, Defaults.AuthToLocal, MEDIUM, AuthToLocalDoc)
+
}
def configNames() = {
@@ -802,6 +835,14 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka
val sslClientAuth = getString(KafkaConfig.SSLClientAuthProp)
val sslCipher = getList(KafkaConfig.SSLCipherSuitesProp)
+ /** ********* Sasl Configuration **************/
+ val saslKerberosServiceName = getString(KafkaConfig.SaslKerberosServiceNameProp)
+ val saslKerberosKinitCmd = getString(KafkaConfig.SaslKerberosKinitCmdProp)
+ val saslKerberosTicketRenewWindowFactor = getDouble(KafkaConfig.SaslKerberosTicketRenewWindowFactorProp)
+ val saslKerberosTicketRenewJitter = getDouble(KafkaConfig.SaslKerberosTicketRenewJitterProp)
+ val saslKerberosMinTimeBeforeRelogin = getLong(KafkaConfig.SaslKerberosMinTimeBeforeReloginProp)
+ val authToLocal = getList(KafkaConfig.AuthToLocalProp)
+
/** ********* Quota Configuration **************/
val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp)
val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp)
@@ -823,7 +864,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka
val millis: java.lang.Long =
Option(getLong(KafkaConfig.LogRetentionTimeMillisProp)).getOrElse(
Option(getInt(KafkaConfig.LogRetentionTimeMinutesProp)) match {
- case Some(mins) => millisInMinute * mins
+ case Some(mins) => millisInMinute * mins
case None => getInt(KafkaConfig.LogRetentionTimeHoursProp) * millisInHour
})
@@ -927,20 +968,30 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka
def channelConfigs: java.util.Map[String, Object] = {
val channelConfigs = new java.util.HashMap[String, Object]()
import kafka.server.KafkaConfig._
- channelConfigs.put(PrincipalBuilderClassProp, Class.forName(principalBuilderClass))
- channelConfigs.put(SSLProtocolProp, sslProtocol)
- channelConfigs.put(SSLEnabledProtocolsProp, sslEnabledProtocols)
- channelConfigs.put(SSLKeystoreTypeProp, sslKeystoreType)
- channelConfigs.put(SSLKeystoreLocationProp, sslKeystoreLocation)
- channelConfigs.put(SSLKeystorePasswordProp, sslKeystorePassword)
- channelConfigs.put(SSLKeyPasswordProp, sslKeyPassword)
- channelConfigs.put(SSLTruststoreTypeProp, sslTruststoreType)
- channelConfigs.put(SSLTruststoreLocationProp, sslTruststoreLocation)
- channelConfigs.put(SSLTruststorePasswordProp, sslTruststorePassword)
- channelConfigs.put(SSLKeyManagerAlgorithmProp, sslKeyManagerAlgorithm)
- channelConfigs.put(SSLTrustManagerAlgorithmProp, sslTrustManagerAlgorithm)
- channelConfigs.put(SSLClientAuthProp, sslClientAuth)
- channelConfigs.put(SSLCipherSuitesProp, sslCipher)
+ Seq(
+ (PrincipalBuilderClassProp, Class.forName(principalBuilderClass)),
+ (SSLProtocolProp, sslProtocol),
+ (SSLEnabledProtocolsProp, sslEnabledProtocols),
+ (SSLKeystoreTypeProp, sslKeystoreType),
+ (SSLKeystoreLocationProp, sslKeystoreLocation),
+ (SSLKeystorePasswordProp, sslKeystorePassword),
+ (SSLKeyPasswordProp, sslKeyPassword),
+ (SSLTruststoreTypeProp, sslTruststoreType),
+ (SSLTruststoreLocationProp, sslTruststoreLocation),
+ (SSLTruststorePasswordProp, sslTruststorePassword),
+ (SSLKeyManagerAlgorithmProp, sslKeyManagerAlgorithm),
+ (SSLTrustManagerAlgorithmProp, sslTrustManagerAlgorithm),
+ (SSLClientAuthProp, sslClientAuth),
+ (SSLCipherSuitesProp, sslCipher),
+ (SaslKerberosServiceNameProp, saslKerberosServiceName),
+ (SaslKerberosKinitCmdProp, saslKerberosKinitCmd),
+ (SaslKerberosTicketRenewWindowFactorProp, saslKerberosTicketRenewWindowFactor),
+ (SaslKerberosTicketRenewJitterProp, saslKerberosTicketRenewJitter),
+ (SaslKerberosMinTimeBeforeReloginProp, saslKerberosMinTimeBeforeRelogin),
+ (AuthToLocalProp, authToLocal)
+ ).foreach { case (key, value) =>
+ if (value != null) channelConfigs.put(key, value)
+ }
channelConfigs
}
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index f50c26608b0d0..510957b704bd8 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -21,7 +21,7 @@ import java.net.{SocketTimeoutException}
import java.util
import kafka.admin._
-import kafka.api.{KAFKA_090, ApiVersion}
+import kafka.api.KAFKA_090
import kafka.log.LogConfig
import kafka.log.CleanerConfig
import kafka.log.LogManager
@@ -34,17 +34,16 @@ import kafka.utils._
import org.apache.kafka.clients.{ManualMetadataUpdater, ClientRequest, NetworkClient}
import org.apache.kafka.common.Node
import org.apache.kafka.common.metrics._
-import org.apache.kafka.common.network.{Selectable, ChannelBuilders, NetworkReceive, Selector}
+import org.apache.kafka.common.network.{LoginType, Selectable, ChannelBuilders, NetworkReceive, Selector, Mode}
import org.apache.kafka.common.protocol.{Errors, ApiKeys, SecurityProtocol}
import org.apache.kafka.common.metrics.{JmxReporter, Metrics}
import org.apache.kafka.common.requests.{ControlledShutdownResponse, ControlledShutdownRequest, RequestSend}
import org.apache.kafka.common.security.JaasUtils
-import org.apache.kafka.common.security.ssl.SSLFactory
import org.apache.kafka.common.utils.AppInfoParser
import scala.collection.mutable
import scala.collection.JavaConverters._
-import org.I0Itec.zkclient.{ZkClient, ZkConnection}
+import org.I0Itec.zkclient.ZkClient
import kafka.controller.{ControllerStats, KafkaController}
import kafka.cluster.{EndPoint, Broker}
import kafka.common.{ErrorMapping, InconsistentBrokerIdException, GenerateBrokerIdException}
@@ -308,7 +307,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
"kafka-server-controlled-shutdown",
Map.empty.asJava,
false,
- ChannelBuilders.create(config.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, config.channelConfigs)
+ ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.channelConfigs)
)
new NetworkClient(
selector,
@@ -491,7 +490,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr
if (config.interBrokerProtocolVersion.onOrAfter(KAFKA_090))
networkClientControlledShutdown(config.controlledShutdownMaxRetries.intValue)
else blockingChannelControlledShutdown(config.controlledShutdownMaxRetries.intValue)
-
+
if (!shutdownSucceeded)
warn("Proceeding to do an unclean shutdown as all the controlled shutdown attempts failed")
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 0a17fd08d3969..5aa817d7113d9 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -27,7 +27,7 @@ import kafka.api.KAFKA_090
import kafka.common.{KafkaStorageException, TopicAndPartition}
import ReplicaFetcherThread._
import org.apache.kafka.clients.{ManualMetadataUpdater, NetworkClient, ClientRequest, ClientResponse}
-import org.apache.kafka.common.network.{Selectable, ChannelBuilders, NetworkReceive, Selector}
+import org.apache.kafka.common.network.{LoginType, Selectable, ChannelBuilders, NetworkReceive, Selector, Mode}
import org.apache.kafka.common.requests.{ListOffsetResponse, FetchResponse, RequestSend, AbstractRequest, ListOffsetRequest}
import org.apache.kafka.common.requests.{FetchRequest => JFetchRequest}
import org.apache.kafka.common.{Node, TopicPartition}
@@ -74,7 +74,7 @@ class ReplicaFetcherThread(name: String,
"replica-fetcher",
Map("broker-id" -> sourceBroker.id.toString).asJava,
false,
- ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, brokerConfig.channelConfigs)
+ ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, brokerConfig.channelConfigs)
)
new NetworkClient(
selector,
diff --git a/core/src/test/resources/kafka_jaas.conf b/core/src/test/resources/kafka_jaas.conf
new file mode 100644
index 0000000000000..b097e260b7a82
--- /dev/null
+++ b/core/src/test/resources/kafka_jaas.conf
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+KafkaClient {
+ com.sun.security.auth.module.Krb5LoginModule required debug=true
+ useKeyTab=true
+ storeKey=true
+ serviceName="kafka"
+ keyTab="$keytab-location"
+ principal="client@EXAMPLE.COM";
+};
+
+KafkaServer {
+ com.sun.security.auth.module.Krb5LoginModule required debug=true
+ useKeyTab=true
+ storeKey=true
+ serviceName="kafka"
+ keyTab="$keytab-location"
+ principal="kafka/localhost@EXAMPLE.COM";
+};
diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
similarity index 94%
rename from core/src/test/scala/integration/kafka/api/ConsumerTest.scala
rename to core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
index d973d9ad73531..84bebef49ee50 100644
--- a/core/src/test/scala/integration/kafka/api/ConsumerTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
@@ -13,7 +13,7 @@
package kafka.api
import java.util.regex.Pattern
-import java.{lang, util}
+import java.util
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
@@ -31,11 +31,10 @@ import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
import kafka.coordinator.ConsumerCoordinator
-
/**
* Integration tests for the new consumer that cover basic usage as well as server failures
*/
-class ConsumerTest extends IntegrationTestHarness with Logging {
+abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
val producerCount = 1
val consumerCount = 2
@@ -76,7 +75,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
assertEquals(1, this.consumers(0).assignment.size)
this.consumers(0).seek(tp, 0)
- consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
+ consumeAndVerifyRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
// check async commit callbacks
val commitCallback = new CountConsumerCommitCallback()
@@ -299,7 +298,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
def testAutoOffsetReset() {
sendRecords(1)
this.consumers(0).assign(List(tp))
- consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+ consumeAndVerifyRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
}
@Test
@@ -315,19 +314,19 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
consumer.seekToBeginning(tp)
assertEquals(0, consumer.position(tp), 0)
- consumeRecords(consumer, numRecords = 1, startingOffset = 0)
+ consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = 0)
val mid = totalRecords / 2
consumer.seek(tp, mid)
assertEquals(mid, consumer.position(tp))
- consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt)
+ consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = mid.toInt, startingKeyAndValueIndex = mid.toInt)
}
@Test
def testGroupConsumption() {
sendRecords(10)
this.consumers(0).subscribe(List(topic))
- consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
+ consumeAndVerifyRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
}
@@ -368,7 +367,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
this.consumers(0).commitSync()
assertEquals(0L, this.consumers(0).committed(tp).offset)
- consumeRecords(this.consumers(0), 5, 0)
+ consumeAndVerifyRecords(this.consumers(0), 5, 0)
assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp))
this.consumers(0).commitSync()
assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp).offset)
@@ -377,7 +376,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
// another consumer in the same group should get the same position
this.consumers(1).assign(List(tp))
- consumeRecords(this.consumers(1), 1, 5)
+ consumeAndVerifyRecords(this.consumers(1), 1, 5)
}
@Test
@@ -506,12 +505,12 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
def testPartitionPauseAndResume() {
sendRecords(5)
this.consumers(0).assign(List(tp))
- consumeRecords(this.consumers(0), 5, 0)
+ consumeAndVerifyRecords(this.consumers(0), 5, 0)
this.consumers(0).pause(tp)
sendRecords(5)
assertTrue(this.consumers(0).poll(0).isEmpty)
this.consumers(0).resume(tp)
- consumeRecords(this.consumers(0), 5, 5)
+ consumeAndVerifyRecords(this.consumers(0), 5, 5)
}
@Test
@@ -522,7 +521,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
sendRecords(5)
consumer0.subscribe(List(topic))
- consumeRecords(consumer0, 5, 0)
+ consumeAndVerifyRecords(consumer0, 5, 0)
consumer0.pause(tp)
// subscribe to a new topic to trigger a rebalance
@@ -530,7 +529,7 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
// after rebalance, our position should be reset and our pause state lost,
// so we should be able to consume from the beginning
- consumeRecords(consumer0, 0, 5)
+ consumeAndVerifyRecords(consumer0, 0, 5)
}
private class TestConsumerReassignmentListener extends ConsumerRebalanceListener {
@@ -551,20 +550,20 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
}
private def sendRecords(numRecords: Int, tp: TopicPartition) {
- val futures = (0 until numRecords).map { i =>
- this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toString.getBytes, i.toString.getBytes))
- }
- futures.map(_.get)
+ (0 until numRecords).map { i =>
+ this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), s"key $i".getBytes, s"value $i".getBytes))
+ }.foreach(_.get)
}
- private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) {
+ private def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int,
+ startingKeyAndValueIndex: Int = 0) {
val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]()
val maxIters = numRecords * 300
var iters = 0
while (records.size < numRecords) {
for (record <- consumer.poll(50).asScala)
records.add(record)
- if(iters > maxIters)
+ if (iters > maxIters)
throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.")
iters += 1
}
@@ -574,6 +573,9 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
assertEquals(topic, record.topic())
assertEquals(part, record.partition())
assertEquals(offset.toLong, record.offset())
+ val keyAndValueIndex = startingKeyAndValueIndex + i
+ assertEquals(s"key $keyAndValueIndex", new String(record.key()))
+ assertEquals(s"value $keyAndValueIndex", new String(record.value()))
}
}
@@ -591,4 +593,4 @@ class ConsumerTest extends IntegrationTestHarness with Logging {
override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = count += 1
}
-}
\ No newline at end of file
+}
diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
similarity index 82%
rename from core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
rename to core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
index 3aef172c861c1..92c93e6550325 100644
--- a/core/src/test/scala/integration/kafka/api/ProducerSendTest.scala
+++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala
@@ -17,6 +17,7 @@
package kafka.api
+import java.io.File
import java.util.Properties
import java.util.concurrent.TimeUnit
@@ -26,20 +27,19 @@ import kafka.message.Message
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import org.apache.kafka.clients.producer._
-import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.errors.SerializationException
-import org.apache.kafka.common.serialization.ByteArraySerializer
import org.junit.Assert._
import org.junit.{After, Before, Test}
-class ProducerSendTest extends KafkaServerTestHarness {
- val numServers = 2
+abstract class BaseProducerSendTest extends KafkaServerTestHarness {
- val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
-
- def generateConfigs() =
- TestUtils.createBrokerConfigs(numServers, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps))
+ def generateConfigs = {
+ val overridingProps = new Properties()
+ val numServers = 2
+ overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
+ TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol),
+ trustStoreFile = trustStoreFile).map(KafkaConfig.fromProps(_, overridingProps))
+ }
private var consumer1: SimpleConsumer = null
private var consumer2: SimpleConsumer = null
@@ -64,6 +64,10 @@ class ProducerSendTest extends KafkaServerTestHarness {
super.tearDown()
}
+ private def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] =
+ TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile,
+ retries = retries, lingerMs = lingerMs, props = props)
+
/**
* testSendOffset checks the basic send API behavior
*
@@ -72,7 +76,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
*/
@Test
def testSendOffset() {
- var producer = TestUtils.createNewProducer(brokerList)
+ val producer = createProducer(brokerList)
val partition = new Integer(0)
object callback extends Callback {
@@ -127,60 +131,29 @@ class ProducerSendTest extends KafkaServerTestHarness {
assertEquals("Should have offset " + (numRecords + 4), numRecords + 4L, producer.send(record0, callback).get.offset)
} finally {
- if (producer != null) {
+ if (producer != null)
producer.close()
- producer = null
- }
}
}
@Test
- def testSerializer() {
+ def testWrongSerializer() {
// send a record with a wrong type should receive a serialization exception
try {
- val producer = createNewProducerWithWrongSerializer(brokerList)
- val record5 = new ProducerRecord[Array[Byte],Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes)
+ val producer = createProducerWithWrongSerializer(brokerList)
+ val record5 = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes)
producer.send(record5)
fail("Should have gotten a SerializationException")
} catch {
case se: SerializationException => // this is ok
}
-
- try {
- createNewProducerWithNoSerializer(brokerList)
- fail("Instantiating a producer without specifying a serializer should cause a ConfigException")
- } catch {
- case ce : ConfigException => // this is ok
- }
-
- // create a producer with explicit serializers should succeed
- createNewProducerWithExplicitSerializer(brokerList)
}
- private def createNewProducerWithWrongSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
- import org.apache.kafka.clients.producer.ProducerConfig
-
+ private def createProducerWithWrongSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
val producerProps = new Properties()
- producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
- return new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
- }
-
- private def createNewProducerWithNoSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
- import org.apache.kafka.clients.producer.ProducerConfig
-
- val producerProps = new Properties()
- producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
- return new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
- }
-
- private def createNewProducerWithExplicitSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
- import org.apache.kafka.clients.producer.ProducerConfig
-
- val producerProps = new Properties()
- producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
- return new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer, new ByteArraySerializer)
+ createProducer(brokerList, props = Some(producerProps))
}
/**
@@ -190,7 +163,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
*/
@Test
def testClose() {
- var producer = TestUtils.createNewProducer(brokerList)
+ val producer = createProducer(brokerList)
try {
// create topic
@@ -204,7 +177,6 @@ class ProducerSendTest extends KafkaServerTestHarness {
// close the producer
producer.close()
- producer = null
// check that all messages have been acked via offset,
// this also checks that messages with same key go to the same partition
@@ -212,10 +184,8 @@ class ProducerSendTest extends KafkaServerTestHarness {
assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset)
} finally {
- if (producer != null) {
+ if (producer != null)
producer.close()
- producer = null
- }
}
}
@@ -226,7 +196,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
*/
@Test
def testSendToPartition() {
- var producer = TestUtils.createNewProducer(brokerList)
+ val producer = createProducer(brokerList)
try {
// create topic
@@ -241,7 +211,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
for (i <- 1 to numRecords)
yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes))
val futures = responses.toList
- futures.map(_.get)
+ futures.foreach(_.get)
for (future <- futures)
assertTrue("Request should have completed", future.isDone)
@@ -267,10 +237,8 @@ class ProducerSendTest extends KafkaServerTestHarness {
assertEquals(i.toLong, messageSet1(i).offset)
}
} finally {
- if (producer != null) {
+ if (producer != null)
producer.close()
- producer = null
- }
}
}
@@ -281,7 +249,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
*/
@Test
def testAutoCreateTopic() {
- var producer = TestUtils.createNewProducer(brokerList, retries = 5)
+ val producer = createProducer(brokerList, retries = 5)
try {
// Send a message to auto-create the topic
@@ -294,7 +262,6 @@ class ProducerSendTest extends KafkaServerTestHarness {
} finally {
if (producer != null) {
producer.close()
- producer = null
}
}
}
@@ -304,7 +271,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
*/
@Test
def testFlush() {
- var producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue)
+ val producer = createProducer(brokerList, lingerMs = Long.MaxValue)
try {
TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes)
@@ -338,7 +305,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
// Test closing from caller thread.
for(i <- 0 until 50) {
- producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue)
+ producer = createProducer(brokerList, lingerMs = Long.MaxValue)
val responses = (0 until numRecords) map (i => producer.send(record0))
assertTrue("No request is complete.", responses.forall(!_.isDone()))
producer.close(0, TimeUnit.MILLISECONDS)
@@ -391,7 +358,7 @@ class ProducerSendTest extends KafkaServerTestHarness {
}
}
for(i <- 0 until 50) {
- producer = TestUtils.createNewProducer(brokerList, lingerMs = Long.MaxValue)
+ producer = createProducer(brokerList, lingerMs = Long.MaxValue)
// send message to partition 0
val responses = ((0 until numRecords) map (i => producer.send(record, new CloseCallback(producer))))
assertTrue("No request is complete.", responses.forall(!_.isDone()))
diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
index 77fcd8b5cb1ea..2ec59fbee6111 100644
--- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
@@ -23,7 +23,7 @@ import kafka.utils.TestUtils
import java.util.Properties
import org.apache.kafka.clients.consumer.KafkaConsumer
import org.apache.kafka.clients.producer.KafkaProducer
-import kafka.server.{OffsetManager, KafkaConfig}
+import kafka.server.KafkaConfig
import kafka.integration.KafkaServerTestHarness
import org.junit.{After, Before}
import scala.collection.mutable.Buffer
@@ -41,11 +41,12 @@ trait IntegrationTestHarness extends KafkaServerTestHarness {
lazy val consumerConfig = new Properties
lazy val serverConfig = new Properties
- var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
- var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]()
+ val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
+ val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]()
override def generateConfigs() = {
- val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect)
+ val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol),
+ trustStoreFile = trustStoreFile)
cfgs.foreach(_.putAll(serverConfig))
cfgs.map(KafkaConfig.fromProps)
}
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
new file mode 100644
index 0000000000000..335d585ca091b
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
@@ -0,0 +1,15 @@
+/**
+ * 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 kafka.api
+
+class PlaintextConsumerTest extends BaseConsumerTest
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala
new file mode 100644
index 0000000000000..d017d13d2cc43
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala
@@ -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 kafka.api
+
+import java.util.Properties
+
+import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer}
+import org.apache.kafka.common.config.ConfigException
+import org.apache.kafka.common.serialization.ByteArraySerializer
+import org.junit.Test
+
+class PlaintextProducerSendTest extends BaseProducerSendTest {
+
+ @Test
+ def testSerializerConstructors() {
+ try {
+ createNewProducerWithNoSerializer(brokerList)
+ fail("Instantiating a producer without specifying a serializer should cause a ConfigException")
+ } catch {
+ case ce : ConfigException => // this is ok
+ }
+
+ // create a producer with explicit serializers should succeed
+ createNewProducerWithExplicitSerializer(brokerList)
+ }
+
+ private def createNewProducerWithNoSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
+ val producerProps = new Properties()
+ producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+ return new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
+ }
+
+ private def createNewProducerWithExplicitSerializer(brokerList: String) : KafkaProducer[Array[Byte],Array[Byte]] = {
+ val producerProps = new Properties()
+ producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
+ return new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer, new ByteArraySerializer)
+ }
+
+}
diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
deleted file mode 100644
index 5dc4cbc5b8662..0000000000000
--- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala
+++ /dev/null
@@ -1,231 +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 kafka.api
-
-import java.util.Properties
-import java.io.File
-
-import org.apache.kafka.clients.producer.ProducerConfig
-import org.apache.kafka.clients.producer.ProducerRecord
-import org.apache.kafka.clients.producer.KafkaProducer
-import org.apache.kafka.clients.consumer.Consumer
-import org.apache.kafka.clients.consumer.KafkaConsumer
-import org.apache.kafka.clients.consumer.ConsumerRecord
-import org.apache.kafka.clients.consumer.ConsumerConfig
-import org.apache.kafka.common.TopicPartition
-import kafka.integration.KafkaServerTestHarness
-
-import kafka.utils.{TestUtils, Logging}
-import kafka.server.KafkaConfig
-
-import java.util.ArrayList
-import org.junit.{Test, Before, After}
-import org.junit.Assert._
-
-import scala.collection.mutable.Buffer
-import scala.collection.JavaConversions._
-import kafka.coordinator.ConsumerCoordinator
-
-
-/**
- * Integration tests for the new consumer that cover basic usage as well as server failures
- */
-class SSLConsumerTest extends KafkaServerTestHarness with Logging {
-
- val trustStoreFile = File.createTempFile("truststore", ".jks")
- val numServers = 3
- val producerCount = 1
- val consumerCount = 2
- val producerConfig = new Properties
- val consumerConfig = new Properties
-
- val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
- overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown
- overridingProps.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset
- overridingProps.put(KafkaConfig.OffsetsTopicPartitionsProp, "1")
- overridingProps.put(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout
-
- val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
- val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]()
-
- def generateConfigs() =
- TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps))
-
- val topic = "topic"
- val part = 0
- val tp = new TopicPartition(topic, part)
-
- // configure the servers and clients
- this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all")
- this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test")
- this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString)
- this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
-
- @Before
- override def setUp() {
- super.setUp()
- producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getSSLBrokerListStrFromServers(servers))
- producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
- producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer])
- consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getSSLBrokerListStrFromServers(servers))
- consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
- consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArrayDeserializer])
- consumerConfig.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range")
-
- for (i <- 0 until producerCount)
- producers += TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers),
- acks = 1,
- enableSSL=true,
- trustStoreFile=Some(trustStoreFile))
- for (i <- 0 until consumerCount)
- consumers += TestUtils.createNewConsumer(TestUtils.getSSLBrokerListStrFromServers(servers),
- groupId = "my-test",
- partitionAssignmentStrategy= "range",
- enableSSL=true,
- trustStoreFile=Some(trustStoreFile))
-
-
- // create the consumer offset topic
- TestUtils.createTopic(zkUtils, ConsumerCoordinator.OffsetsTopicName,
- overridingProps.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt,
- overridingProps.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
- servers,
- servers(0).consumerCoordinator.offsetsTopicConfigs)
-
- // create the test topic with all the brokers as replicas
- TestUtils.createTopic(zkUtils, topic, 1, numServers, this.servers)
- }
-
- @After
- override def tearDown() {
- producers.foreach(_.close())
- consumers.foreach(_.close())
- super.tearDown()
- }
-
- @Test
- def testSimpleConsumption() {
- val numRecords = 10000
- sendRecords(numRecords)
- assertEquals(0, this.consumers(0).assignment.size)
- this.consumers(0).assign(List(tp))
- assertEquals(1, this.consumers(0).assignment.size)
- this.consumers(0).seek(tp, 0)
- consumeRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0)
- }
-
- @Test
- def testAutoOffsetReset() {
- sendRecords(1)
- this.consumers(0).assign(List(tp))
- consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
- }
-
- @Test
- def testSeek() {
- val consumer = this.consumers(0)
- val totalRecords = 50L
- sendRecords(totalRecords.toInt)
- consumer.assign(List(tp))
-
- consumer.seekToEnd(tp)
- assertEquals(totalRecords, consumer.position(tp))
- assertFalse(consumer.poll(totalRecords).iterator().hasNext)
-
- consumer.seekToBeginning(tp)
- assertEquals(0, consumer.position(tp), 0)
- consumeRecords(consumer, numRecords = 1, startingOffset = 0)
-
- val mid = totalRecords / 2
- consumer.seek(tp, mid)
- assertEquals(mid, consumer.position(tp))
- consumeRecords(consumer, numRecords = 1, startingOffset = mid.toInt)
- }
-
- @Test
- def testGroupConsumption() {
- sendRecords(10)
- this.consumers(0).subscribe(List(topic))
- consumeRecords(this.consumers(0), numRecords = 1, startingOffset = 0)
- }
-
- @Test
- def testPositionAndCommit() {
- sendRecords(5)
-
- // committed() on a partition with no committed offset returns null
- assertNull(this.consumers(0).committed(new TopicPartition(topic, 15)))
-
- // position() on a partition that we aren't subscribed to throws an exception
- intercept[IllegalArgumentException] {
- this.consumers(0).position(new TopicPartition(topic, 15))
- }
-
- this.consumers(0).assign(List(tp))
-
- assertEquals("position() on a partition that we are subscribed to should reset the offset", 0L, this.consumers(0).position(tp))
- this.consumers(0).commitSync()
- assertEquals(0L, this.consumers(0).committed(tp).offset)
-
- consumeRecords(this.consumers(0), 5, 0)
- assertEquals("After consuming 5 records, position should be 5", 5L, this.consumers(0).position(tp))
- this.consumers(0).commitSync()
- assertEquals("Committed offset should be returned", 5L, this.consumers(0).committed(tp).offset)
-
- sendRecords(1)
-
- // another consumer in the same group should get the same position
- this.consumers(1).assign(List(tp))
- consumeRecords(this.consumers(1), 1, 5)
- }
-
- @Test
- def testPartitionsFor() {
- val numParts = 2
- TestUtils.createTopic(zkUtils, "part-test", numParts, 1, this.servers)
- val parts = this.consumers(0).partitionsFor("part-test")
- assertNotNull(parts)
- assertEquals(2, parts.length)
- assertNull(this.consumers(0).partitionsFor("non-exist-topic"))
- }
-
- private def sendRecords(numRecords: Int) {
- val futures = (0 until numRecords).map { i =>
- this.producers(0).send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes))
- }
- futures.map(_.get)
- }
-
- private def consumeRecords(consumer: Consumer[Array[Byte], Array[Byte]], numRecords: Int, startingOffset: Int) {
- val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]()
- val maxIters = numRecords * 300
- var iters = 0
- while (records.size < numRecords) {
- for (record <- consumer.poll(50)) {
- records.add(record)
- }
- if (iters > maxIters)
- throw new IllegalStateException("Failed to consume the expected records after " + iters + " iterations.")
- iters += 1
- }
- for (i <- 0 until numRecords) {
- val record = records.get(i)
- val offset = startingOffset + i
- assertEquals(topic, record.topic())
- assertEquals(part, record.partition())
- assertEquals(offset.toLong, record.offset())
- }
- }
-
-}
diff --git a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
deleted file mode 100644
index c22e57a8e8da7..0000000000000
--- a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala
+++ /dev/null
@@ -1,240 +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 kafka.api
-
-import java.util.Properties
-import java.util.concurrent.TimeUnit
-import java.io.File
-
-import kafka.consumer.SimpleConsumer
-import kafka.integration.KafkaServerTestHarness
-import kafka.message.Message
-import kafka.server.KafkaConfig
-import kafka.utils.TestUtils
-import org.apache.kafka.clients.producer._
-import org.apache.kafka.common.config.ConfigException
-import org.apache.kafka.common.errors.SerializationException
-import org.apache.kafka.common.serialization.ByteArraySerializer
-import org.junit.Assert._
-import org.junit.{After, Before, Test}
-
-
-class SSLProducerSendTest extends KafkaServerTestHarness {
- val numServers = 2
- val trustStoreFile = File.createTempFile("truststore", ".jks")
-
- val overridingProps = new Properties()
- overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString)
-
- def generateConfigs() =
- TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps))
-
- private var consumer1: SimpleConsumer = null
- private var consumer2: SimpleConsumer = null
-
- private val topic = "topic"
- private val numRecords = 100
-
- @Before
- override def setUp() {
- super.setUp()
-
- // TODO: we need to migrate to new consumers when 0.9 is final
- consumer1 = new SimpleConsumer("localhost", servers(0).boundPort(), 100, 1024*1024, "")
- consumer2 = new SimpleConsumer("localhost", servers(1).boundPort(), 100, 1024*1024, "")
-
- }
-
- @After
- override def tearDown() {
- consumer1.close()
- consumer2.close()
- super.tearDown()
- }
-
- /**
- * testSendOffset checks the basic send API behavior
- *
- * 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected.
- * 2. Last message of the non-blocking send should return the correct offset metadata
- */
- @Test
- def testSendOffset() {
- var sslProducer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile))
- var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers))
- val partition = new Integer(0)
-
- object callback extends Callback {
- var offset = 0L
- def onCompletion(metadata: RecordMetadata, exception: Exception) {
- if (exception == null) {
- assertEquals(offset, metadata.offset())
- assertEquals(topic, metadata.topic())
- assertEquals(partition, metadata.partition())
- offset += 1
- } else {
- fail("Send callback returns the following exception", exception)
- }
- }
- }
-
- try {
- // create topic
- TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
-
- // send a normal record
- val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes)
- assertEquals("Should have offset 0", 0L, sslProducer.send(record0, callback).get.offset)
-
- // send a record with null value should be ok
- val record1 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, null)
- assertEquals("Should have offset 1", 1L, sslProducer.send(record1, callback).get.offset)
-
- // send a record with null key should be ok
- val record2 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, "value".getBytes)
- assertEquals("Should have offset 2", 2L, sslProducer.send(record2, callback).get.offset)
-
- // send a record with null part id should be ok
- val record3 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
- assertEquals("Should have offset 3", 3L, sslProducer.send(record3, callback).get.offset)
-
- // send a record with null topic should fail
- try {
- val record4 = new ProducerRecord[Array[Byte],Array[Byte]](null, partition, "key".getBytes, "value".getBytes)
- sslProducer.send(record4, callback)
- fail("Should not allow sending a record without topic")
- } catch {
- case iae: IllegalArgumentException => // this is ok
- case e: Throwable => fail("Only expecting IllegalArgumentException", e)
- }
-
- // non-blocking send a list of records with sslProducer
- for (i <- 1 to numRecords)
- sslProducer.send(record0, callback)
- // check that all messages have been acked via offset
- assertEquals("Should have offset " + numRecords + 4L, numRecords + 4L, sslProducer.send(record0, callback).get.offset)
-
- //non-blocking send a list of records with plaintext producer
- for (i <- 1 to numRecords)
- producer.send(record0, callback)
-
- // check that all messages have been acked via offset
- assertEquals("Should have offset " + (numRecords * 2 + 5L), numRecords * 2 + 5L, producer.send(record0, callback).get.offset)
-
- } finally {
- if (sslProducer != null) {
- sslProducer.close()
- sslProducer = null
- }
- if (producer != null) {
- producer.close()
- producer = null
- }
-
- }
- }
-
- /**
- * testClose checks the closing behavior
- *
- * After close() returns, all messages should be sent with correct returned offset metadata
- */
- @Test
- def testClose() {
- var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile))
- try {
- // create topic
- TestUtils.createTopic(zkUtils, topic, 1, 2, servers)
-
- // non-blocking send a list of records
- val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, null, "key".getBytes, "value".getBytes)
- for (i <- 1 to numRecords)
- producer.send(record0)
- val response0 = producer.send(record0)
-
- // close the producer
- producer.close()
- producer = null
-
- // check that all messages have been acked via offset,
- // this also checks that messages with same key go to the same partition
- assertTrue("The last message should be acked before producer is shutdown", response0.isDone)
- assertEquals("Should have offset " + numRecords, numRecords.toLong, response0.get.offset)
-
- } finally {
- if (producer != null) {
- producer.close()
- producer = null
- }
- }
- }
-
- /**
- * testSendToPartition checks the partitioning behavior
- *
- * The specified partition-id should be respected
- */
- @Test
- def testSendToPartition() {
- var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile))
- try {
- // create topic
- val leaders = TestUtils.createTopic(zkUtils, topic, 2, 2, servers)
- val partition = 1
-
- // make sure leaders exist
- val leader1 = leaders(partition)
- assertTrue("Leader for topic \"topic\" partition 1 should exist", leader1.isDefined)
-
- val responses =
- for (i <- 1 to numRecords)
- yield producer.send(new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, null, ("value" + i).getBytes))
- val futures = responses.toList
- futures.map(_.get)
- for (future <- futures)
- assertTrue("Request should have completed", future.isDone)
-
- // make sure all of them end up in the same partition with increasing offset values
- for ((future, offset) <- futures zip (0 until numRecords)) {
- assertEquals(offset.toLong, future.get.offset)
- assertEquals(topic, future.get.topic)
- assertEquals(partition, future.get.partition)
- }
-
- // make sure the fetched messages also respect the partitioning and ordering
- val fetchResponse1 = if (leader1.get == configs(0).brokerId) {
- consumer1.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
- } else {
- consumer2.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, Int.MaxValue).build())
- }
- val messageSet1 = fetchResponse1.messageSet(topic, partition).iterator.toBuffer
- assertEquals("Should have fetched " + numRecords + " messages", numRecords, messageSet1.size)
-
- // TODO: also check topic and partition after they are added in the return messageSet
- for (i <- 0 to numRecords - 1) {
- assertEquals(new Message(bytes = ("value" + (i + 1)).getBytes), messageSet1(i).message)
- assertEquals(i.toLong, messageSet1(i).offset)
- }
- } finally {
- if (producer != null) {
- producer.close()
- producer = null
- }
- }
- }
-}
diff --git a/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala
new file mode 100644
index 0000000000000..e6f0c2b349a0e
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala
@@ -0,0 +1,19 @@
+/**
+ * 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 kafka.api
+
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarness {
+ override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+}
diff --git a/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala
new file mode 100644
index 0000000000000..4f8512ab7ecec
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala
@@ -0,0 +1,22 @@
+/**
+ * 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 kafka.api
+
+import java.io.File
+
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslSslConsumerTest extends BaseConsumerTest with SaslTestHarness {
+ override protected def securityProtocol = SecurityProtocol.SASL_SSL
+ override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+}
diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala
new file mode 100644
index 0000000000000..9575fdae6afc1
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package kafka.api
+
+import java.io.{BufferedReader, FileWriter, BufferedWriter, File}
+import javax.security.auth.login.Configuration
+
+import kafka.utils.TestUtils
+import kafka.zk.ZooKeeperTestHarness
+import org.apache.hadoop.minikdc.MiniKdc
+import org.apache.kafka.common.security.JaasUtils
+import org.junit.{After, Before}
+
+trait SaslTestHarness extends ZooKeeperTestHarness {
+ val workDir = new File(System.getProperty("test.dir", "target"))
+ val kdcConf = MiniKdc.createConf()
+ val kdc = new MiniKdc(kdcConf, workDir)
+
+ @Before
+ override def setUp() {
+ // Clean-up global configuration set by other tests
+ Configuration.setConfiguration(null)
+ val keytabFile = TestUtils.tempFile()
+ val jaasFile = TestUtils.tempFile()
+
+ val writer = new BufferedWriter(new FileWriter(jaasFile))
+ val source = io.Source.fromInputStream(
+ Thread.currentThread().getContextClassLoader.getResourceAsStream("kafka_jaas.conf"), "UTF-8")
+ if (source == null)
+ throw new IllegalStateException("Could not load `kaas_jaas.conf`, make sure it is in the classpath")
+
+ for (line <- source.getLines) {
+ val replaced = line.replaceAll("\\$keytab-location", keytabFile.getAbsolutePath)
+ writer.write(replaced)
+ writer.newLine()
+ }
+ writer.close()
+ source.close()
+
+ kdc.start()
+ kdc.createPrincipal(keytabFile, "client", "kafka/localhost")
+ System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile.getAbsolutePath)
+ super.setUp
+ }
+
+ @After
+ override def tearDown() {
+ super.tearDown
+ kdc.stop()
+ System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)
+ Configuration.setConfiguration(null)
+ }
+}
diff --git a/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala
new file mode 100644
index 0000000000000..1d13d88c311ff
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala
@@ -0,0 +1,22 @@
+/**
+ * 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 kafka.api
+
+import java.io.File
+
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SslConsumerTest extends BaseConsumerTest {
+ override protected def securityProtocol = SecurityProtocol.SSL
+ override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+}
diff --git a/core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala
new file mode 100644
index 0000000000000..4d9189c065c61
--- /dev/null
+++ b/core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala
@@ -0,0 +1,27 @@
+/**
+ * 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 kafka.api
+
+import java.io.File
+
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SslProducerSendTest extends BaseProducerSendTest {
+ override protected def securityProtocol = SecurityProtocol.SSL
+ override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+}
diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
index 3cf4dae7f50e8..05dc0bc30d2a3 100644
--- a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala
@@ -39,17 +39,23 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness {
var adHocConfigs: Seq[KafkaConfig] = null
val numConfigs: Int = 4
- /* If this is `Some`, SSL will be enabled */
+ // This should be defined if `securityProtocol` uses SSL (eg SSL, SASL_SSL)
protected def trustStoreFile: Option[File]
+ protected def securityProtocol: SecurityProtocol
@Before
override def setUp() {
super.setUp()
- val props = createBrokerConfigs(numConfigs, zkConnect, enableSSL = trustStoreFile.isDefined, trustStoreFile = trustStoreFile)
+ val props = createBrokerConfigs(numConfigs, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol),
+ trustStoreFile = trustStoreFile)
val configs: Seq[KafkaConfig] = props.map(KafkaConfig.fromProps)
adHocConfigs = configs.takeRight(configs.size - 1) // Started and stopped by individual test cases
server1 = TestUtils.createServer(configs.head)
- brokerEndPoints = Seq(new Broker(server1.config.brokerId, server1.config.hostName, server1.boundPort()).getBrokerEndPoint(SecurityProtocol.PLAINTEXT))
+ brokerEndPoints = Seq(
+ // We are using the Scala clients and they don't support SSL. Once we move to the Java ones, we should use
+ // `securityProtocol` instead of PLAINTEXT below
+ new BrokerEndPoint(server1.config.brokerId, server1.config.hostName, server1.boundPort(SecurityProtocol.PLAINTEXT))
+ )
}
@After
diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
index bca0dccf049f8..26b86f751a1bd 100755
--- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
+++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala
@@ -17,12 +17,14 @@
package kafka.integration
+import java.io.File
import java.util.Arrays
import kafka.common.KafkaException
import kafka.server._
import kafka.utils.{CoreUtils, TestUtils}
import kafka.zk.ZooKeeperTestHarness
+import org.apache.kafka.common.protocol.SecurityProtocol
import org.junit.{After, Before}
import scala.collection.mutable.Buffer
@@ -52,13 +54,16 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness {
def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",")
+ protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT
+ protected def trustStoreFile: Option[File] = None
+
@Before
override def setUp() {
super.setUp
if(configs.size <= 0)
throw new KafkaException("Must supply at least one server config.")
servers = configs.map(TestUtils.createServer(_)).toBuffer
- brokerList = TestUtils.getBrokerListStrFromServers(servers)
+ brokerList = TestUtils.getBrokerListStrFromServers(servers, securityProtocol)
alive = new Array[Boolean](servers.length)
Arrays.fill(alive, true)
}
diff --git a/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala
index 176d251993205..55c12b5c8a86c 100644
--- a/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/PlaintextTopicMetadataTest.scala
@@ -17,7 +17,10 @@
package kafka.integration
+import org.apache.kafka.common.protocol.SecurityProtocol
+
class PlaintextTopicMetadataTest extends BaseTopicMetadataTest {
+ protected def securityProtocol = SecurityProtocol.PLAINTEXT
protected def trustStoreFile = None
}
diff --git a/core/src/test/scala/unit/kafka/integration/SaslPlaintextTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/SaslPlaintextTopicMetadataTest.scala
new file mode 100644
index 0000000000000..11d6da4b48b3c
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/integration/SaslPlaintextTopicMetadataTest.scala
@@ -0,0 +1,26 @@
+/**
+ * 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 kafka.integration
+
+import kafka.api.SaslTestHarness
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslPlaintextTopicMetadataTest extends BaseTopicMetadataTest with SaslTestHarness {
+ protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+ protected def trustStoreFile = None
+}
diff --git a/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala
new file mode 100644
index 0000000000000..ea1541911fd83
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala
@@ -0,0 +1,28 @@
+/**
+ * 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 kafka.integration
+
+import java.io.File
+
+import kafka.api.SaslTestHarness
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslSslTopicMetadataTest extends BaseTopicMetadataTest with SaslTestHarness {
+ protected def securityProtocol = SecurityProtocol.SASL_SSL
+ protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+}
diff --git a/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala
index 5ff9f35b27831..ee734573a8cff 100644
--- a/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala
+++ b/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala
@@ -19,6 +19,9 @@ package kafka.integration
import java.io.File
+import org.apache.kafka.common.protocol.SecurityProtocol
+
class SslTopicMetadataTest extends BaseTopicMetadataTest {
- protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+ protected def securityProtocol = SecurityProtocol.SSL
+ protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
}
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 6f07a7a2ef93c..b0cb97ee7c608 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -221,7 +221,8 @@ class SocketServerTest extends JUnitSuite {
@Test
def testSSLSocketServer(): Unit = {
val trustStoreFile = File.createTempFile("truststore", ".jks")
- val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0, enableSSL = true, trustStoreFile = Some(trustStoreFile))
+ val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, enableSsl = true,
+ trustStoreFile = Some(trustStoreFile))
overrideProps.put("listeners", "SSL://localhost:0")
val serverMetrics = new Metrics
diff --git a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
index ade110d415b07..5ecc2c0f91dc9 100644
--- a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala
@@ -19,6 +19,7 @@ package kafka.server
import java.io.File
+import org.apache.kafka.common.protocol.SecurityProtocol
import org.junit.{Test, After, Before}
import kafka.zk.ZooKeeperTestHarness
import kafka.utils.TestUtils._
@@ -32,15 +33,16 @@ abstract class BaseReplicaFetchTest extends ZooKeeperTestHarness {
val topic1 = "foo"
val topic2 = "bar"
- /* If this is `Some`, SSL will be enabled */
+ // This should be defined if `securityProtocol` uses SSL (eg SSL, SASL_SSL)
protected def trustStoreFile: Option[File]
+ protected def securityProtocol: SecurityProtocol
@Before
override def setUp() {
super.setUp()
- brokers = createBrokerConfigs(2, zkConnect, enableControlledShutdown = false, enableSSL = trustStoreFile.isDefined, trustStoreFile = trustStoreFile)
- .map(KafkaConfig.fromProps)
- .map(TestUtils.createServer(_))
+ val props = createBrokerConfigs(2, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol),
+ trustStoreFile = trustStoreFile)
+ brokers = props.map(KafkaConfig.fromProps).map(TestUtils.createServer(_))
}
@After
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 42c1199bc7b24..c9f25400ad699 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -507,6 +507,14 @@ class KafkaConfigTest {
case KafkaConfig.SSLClientAuthProp => // ignore string
case KafkaConfig.SSLCipherSuitesProp => // ignore string
+ //Sasl Configs
+ case KafkaConfig.SaslKerberosServiceNameProp => // ignore string
+ case KafkaConfig.SaslKerberosKinitCmdProp =>
+ case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp =>
+ case KafkaConfig.SaslKerberosTicketRenewJitterProp =>
+ case KafkaConfig.SaslKerberosMinTimeBeforeReloginProp =>
+ case KafkaConfig.AuthToLocalProp => // ignore string
+
case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1")
}
})
diff --git a/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.scala
index 871e49b6f8d83..b160481ae4e33 100644
--- a/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/PlaintextReplicaFetchTest.scala
@@ -17,6 +17,9 @@
package kafka.server
+import org.apache.kafka.common.protocol.SecurityProtocol
+
class PlaintextReplicaFetchTest extends BaseReplicaFetchTest {
+ protected def securityProtocol = SecurityProtocol.PLAINTEXT
protected def trustStoreFile = None
}
diff --git a/core/src/test/scala/unit/kafka/server/SaslPlaintextReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/SaslPlaintextReplicaFetchTest.scala
new file mode 100644
index 0000000000000..740db374b01b7
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/SaslPlaintextReplicaFetchTest.scala
@@ -0,0 +1,26 @@
+/**
+ * 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 kafka.server
+
+import kafka.api.SaslTestHarness
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslPlaintextReplicaFetchTest extends BaseReplicaFetchTest with SaslTestHarness {
+ protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT
+ protected def trustStoreFile = None
+}
diff --git a/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala
new file mode 100644
index 0000000000000..1bcf8ac7d64e0
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala
@@ -0,0 +1,28 @@
+/**
+ * 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 kafka.server
+
+import java.io.File
+
+import kafka.api.SaslTestHarness
+import org.apache.kafka.common.protocol.SecurityProtocol
+
+class SaslSslReplicaFetchTest extends BaseReplicaFetchTest with SaslTestHarness {
+ protected def securityProtocol = SecurityProtocol.SASL_SSL
+ protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+}
diff --git a/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala
index 98580528fb15e..dad22851186ef 100644
--- a/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala
@@ -19,6 +19,9 @@ package kafka.server
import java.io.File
+import org.apache.kafka.common.protocol.SecurityProtocol
+
class SslReplicaFetchTest extends BaseReplicaFetchTest {
- protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
+ protected def securityProtocol = SecurityProtocol.SSL
+ protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks"))
}
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 1a0a7dc5cf6c4..46c88a31d591b 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -30,7 +30,7 @@ import kafka.security.auth.{Resource, Authorizer, Acl}
import org.apache.kafka.common.protocol.SecurityProtocol
import org.apache.kafka.common.utils.Utils._
-import collection.mutable.ListBuffer
+import scala.collection.mutable.{ArrayBuffer, ListBuffer}
import org.I0Itec.zkclient.{ZkClient, ZkConnection}
@@ -52,6 +52,7 @@ import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer}
import org.apache.kafka.common.serialization.ByteArrayDeserializer
import org.apache.kafka.clients.CommonClientConfigs
+import org.apache.kafka.common.network.Mode
import org.apache.kafka.common.security.ssl.SSLFactory
import org.apache.kafka.common.config.SSLConfigs
import org.apache.kafka.test.TestSSLUtils
@@ -137,37 +138,66 @@ object TestUtils extends Logging {
}
/**
- * Create a test config for the given node id
+ * Create a test config for the provided parameters.
+ *
+ * Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled.
*/
def createBrokerConfigs(numConfigs: Int,
zkConnect: String,
enableControlledShutdown: Boolean = true,
enableDeleteTopic: Boolean = false,
- enableSSL: Boolean = false,
- trustStoreFile: Option[File] = None): Seq[Properties] = {
- (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, enableSSL = enableSSL, trustStoreFile = trustStoreFile))
- }
-
- def getBrokerListStrFromServers(servers: Seq[KafkaServer]): String = {
- servers.map(s => formatAddress(s.config.hostName, s.boundPort())).mkString(",")
+ interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
+ trustStoreFile: Option[File] = None,
+ enablePlaintext: Boolean = true,
+ enableSsl: Boolean = false,
+ enableSaslPlaintext: Boolean = false,
+ enableSaslSsl: Boolean = false): Seq[Properties] = {
+ (0 until numConfigs).map { node =>
+ createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort,
+ interBrokerSecurityProtocol, trustStoreFile, enablePlaintext = enablePlaintext, enableSsl = enableSsl,
+ enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl)
+ }
}
- def getSSLBrokerListStrFromServers(servers: Seq[KafkaServer]): String = {
- servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.SSL))).mkString(",")
+ def getBrokerListStrFromServers(servers: Seq[KafkaServer], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): String = {
+ servers.map(s => formatAddress(s.config.hostName, s.boundPort(protocol))).mkString(",")
}
/**
- * Create a test config for the given node id
- */
+ * Create a test config for the provided parameters.
+ *
+ * Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled.
+ */
def createBrokerConfig(nodeId: Int, zkConnect: String,
enableControlledShutdown: Boolean = true,
enableDeleteTopic: Boolean = false,
- port: Int = RandomPort, enableSSL: Boolean = false, sslPort: Int = RandomPort, trustStoreFile: Option[File] = None): Properties = {
+ port: Int = RandomPort,
+ interBrokerSecurityProtocol: Option[SecurityProtocol] = None,
+ trustStoreFile: Option[File] = None,
+ enablePlaintext: Boolean = true,
+ enableSaslPlaintext: Boolean = false, saslPlaintextPort: Int = RandomPort,
+ enableSsl: Boolean = false, sslPort: Int = RandomPort,
+ enableSaslSsl: Boolean = false, saslSslPort: Int = RandomPort)
+ : Properties = {
+
+ def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol)
+
+ val protocolAndPorts = ArrayBuffer[(SecurityProtocol, Int)]()
+ if (enablePlaintext || shouldEnable(SecurityProtocol.PLAINTEXT))
+ protocolAndPorts += SecurityProtocol.PLAINTEXT -> port
+ if (enableSsl || shouldEnable(SecurityProtocol.SSL))
+ protocolAndPorts += SecurityProtocol.SSL -> sslPort
+ if (enableSaslPlaintext || shouldEnable(SecurityProtocol.SASL_PLAINTEXT))
+ protocolAndPorts += SecurityProtocol.SASL_PLAINTEXT -> saslPlaintextPort
+ if (enableSaslSsl || shouldEnable(SecurityProtocol.SASL_SSL))
+ protocolAndPorts += SecurityProtocol.SASL_SSL -> saslSslPort
+
+ val listeners = protocolAndPorts.map { case (protocol, port) =>
+ s"${protocol.name}://localhost:$port"
+ }.mkString(",")
+
val props = new Properties
- var listeners: String = "PLAINTEXT://localhost:"+port.toString
if (nodeId >= 0) props.put("broker.id", nodeId.toString)
- if (enableSSL)
- listeners = listeners + "," + "SSL://localhost:"+sslPort.toString
props.put("listeners", listeners)
props.put("log.dir", TestUtils.tempDir().getAbsolutePath)
props.put("zookeeper.connect", zkConnect)
@@ -176,9 +206,14 @@ object TestUtils extends Logging {
props.put("controlled.shutdown.enable", enableControlledShutdown.toString)
props.put("delete.topic.enable", enableDeleteTopic.toString)
props.put("controlled.shutdown.retry.backoff.ms", "100")
- if (enableSSL) {
- props.putAll(addSSLConfigs(SSLFactory.Mode.SERVER, true, trustStoreFile, "server"+nodeId))
+
+ if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) })
+ props.putAll(sslConfigs(Mode.SERVER, true, trustStoreFile, s"server$nodeId"))
+
+ interBrokerSecurityProtocol.foreach { protocol =>
+ props.put(KafkaConfig.InterBrokerSecurityProtocolProp, protocol.name)
}
+
props.put("port", port.toString)
props
}
@@ -404,28 +439,41 @@ object TestUtils extends Logging {
bufferSize: Long = 1024L * 1024L,
retries: Int = 0,
lingerMs: Long = 0,
- enableSSL: Boolean = false,
- trustStoreFile: Option[File] = None) : KafkaProducer[Array[Byte],Array[Byte]] = {
+ securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
+ trustStoreFile: Option[File] = None,
+ props: Option[Properties] = None) : KafkaProducer[Array[Byte],Array[Byte]] = {
import org.apache.kafka.clients.producer.ProducerConfig
- val producerProps = new Properties()
+ val producerProps = props.getOrElse(new Properties)
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
producerProps.put(ProducerConfig.ACKS_CONFIG, acks.toString)
producerProps.put(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, metadataFetchTimeout.toString)
producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString)
producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString)
- producerProps.put(ProducerConfig.RETRY_BACKOFF_MS_CONFIG, "100")
- producerProps.put(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG, "200")
- producerProps.put(ProducerConfig.LINGER_MS_CONFIG, lingerMs.toString)
- producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
- producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer")
- if (enableSSL) {
- producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL")
- producerProps.putAll(addSSLConfigs(SSLFactory.Mode.CLIENT, false, trustStoreFile, "producer"))
+
+ /* Only use these if not already set */
+ val defaultProps = Map(
+ ProducerConfig.RETRY_BACKOFF_MS_CONFIG -> "100",
+ ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG -> "200",
+ ProducerConfig.LINGER_MS_CONFIG -> lingerMs.toString,
+ ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer",
+ ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer"
+ )
+ defaultProps.foreach { case (key, value) =>
+ if (!producerProps.containsKey(key)) producerProps.put(key, value)
}
+
+ if (usesSslTransportLayer(securityProtocol))
+ producerProps.putAll(sslConfigs(Mode.CLIENT, false, trustStoreFile, "producer"))
+ producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name)
new KafkaProducer[Array[Byte],Array[Byte]](producerProps)
}
+ private def usesSslTransportLayer(securityProtocol: SecurityProtocol): Boolean = securityProtocol match {
+ case SecurityProtocol.SSL | SecurityProtocol.SASL_SSL => true
+ case _ => false
+ }
+
/**
* Create a new consumer with a few pre-configured properties.
*/
@@ -435,7 +483,7 @@ object TestUtils extends Logging {
partitionFetchSize: Long = 4096L,
partitionAssignmentStrategy: String = "blah",
sessionTimeout: Int = 30000,
- enableSSL: Boolean = false,
+ securityProtocol: SecurityProtocol,
trustStoreFile: Option[File] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = {
import org.apache.kafka.clients.consumer.ConsumerConfig
@@ -450,10 +498,9 @@ object TestUtils extends Logging {
consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer")
consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, partitionAssignmentStrategy)
consumerProps.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, sessionTimeout.toString)
- if (enableSSL) {
- consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL")
- consumerProps.putAll(addSSLConfigs(SSLFactory.Mode.CLIENT, false, trustStoreFile, "consumer"))
- }
+ if (usesSslTransportLayer(securityProtocol))
+ consumerProps.putAll(sslConfigs(Mode.CLIENT, false, trustStoreFile, "consumer"))
+ consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name)
new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps)
}
@@ -910,19 +957,18 @@ object TestUtils extends Logging {
new String(bytes, encoding)
}
- def addSSLConfigs(mode: SSLFactory.Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = {
- if (!trustStoreFile.isDefined) {
- throw new Exception("enableSSL set to true but no trustStoreFile provided")
+ def sslConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = {
+
+ val trustStore = trustStoreFile.getOrElse {
+ throw new Exception("SSL enabled but no trustStoreFile provided")
}
+
val sslConfigs = {
- if (mode == SSLFactory.Mode.SERVER) {
- val sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias)
- sslConfigs.put(KafkaConfig.InterBrokerSecurityProtocolProp, SecurityProtocol.SSL.name)
- sslConfigs
- }
+ if (mode == Mode.SERVER)
+ TestSSLUtils.createSSLConfig(true, true, mode, trustStore, certAlias)
else
- TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias)
+ TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStore, certAlias)
}
val sslProps = new Properties()