From 82737e5bb71f67271d90c059dede74935f8a5e56 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 31 Aug 2015 16:07:15 -0700 Subject: [PATCH 01/50] KAFKA-1686. Implement SASL/Kerberos. --- checkstyle/import-control.xml | 4 + .../kafka/common/config/SaslConfigs.java | 28 ++ .../kafka/common/network/ChannelBuilder.java | 2 + .../common/network/SaslChannelBuilder.java | 73 +++ .../common/protocol/SecurityProtocol.java | 2 + .../kafka/common/security/JaasUtils.java | 78 ++++ .../SaslClientAuthenticator.java | 261 +++++++++++ .../SaslServerAuthenticator.java | 197 +++++++++ .../SaslServerCallbackHandler.java | 83 ++++ .../security/kerberos/KerberosName.java | 378 ++++++++++++++++ .../kafka/common/security/kerberos/Login.java | 414 ++++++++++++++++++ .../security/kerberos/LoginManager.java | 68 +++ .../org/apache/kafka/common/utils/Shell.java | 335 ++++++++++++++ .../apache/kafka/common/utils/SystemTime.java | 18 + .../org/apache/kafka/common/utils/Time.java | 22 + .../apache/kafka/common/utils/MockTime.java | 21 +- .../scala/kafka/network/SocketServer.scala | 4 +- 17 files changed, 1985 insertions(+), 3 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java create mode 100644 clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java create mode 100644 clients/src/main/java/org/apache/kafka/common/utils/Shell.java diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index eb682f4eb37a0..ddeeefd8130cc 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -27,7 +27,10 @@ + + + @@ -39,6 +42,7 @@ + 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..9d299f47b75c3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -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 org.apache.kafka.common.config; + + +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 DEFAULT_KAFKA_SERVER_REALM = ""; + + +} 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..ddd7b8f045689 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 + * @returns KafkaChannel */ KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException; 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..d30a5305742f1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -0,0 +1,73 @@ +/** + * 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.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import java.util.Map; + +import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.common.security.kerberos.LoginFactory; +import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator; +import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator; +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 LoginFactory loginFactory; + private PrincipalBuilder principalBuilder; + private LoginFactory.Mode mode; + + public SaslChannelBuilder(LoginFactory.Mode mode) { + this.mode = mode; + } + + public void configure(Map configs) throws KafkaException { + try { + this.loginFactory = new LoginFactory(mode); + this.loginFactory.configure(configs); + this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); + this.principalBuilder.configure(configs); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException { + KafkaChannel channel = null; + try { + SocketChannel socketChannel = (SocketChannel) key.channel(); + TransportLayer transportLayer = new PlaintextTransportLayer(key); + Authenticator authenticator; + if (mode == LoginFactory.Mode.SERVER) + authenticator = new SaslServerAuthenticator(loginFactory.subject()); + else + authenticator = new SaslClientAuthenticator(loginFactory.subject(), loginFactory.serviceName(), socketChannel.socket().getInetAddress().getHostName()); + authenticator.configure(transportLayer, this.principalBuilder); + channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); + } catch (Exception e) { + log.info("Failed to create channel due to ", e); + throw new KafkaException(e); + } + return channel; + } + + public void close() { + this.principalBuilder.close(); + } +} 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..7db45b528f1c7 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,8 @@ public enum SecurityProtocol { PLAINTEXT(0, "PLAINTEXT"), /** SSL channel */ SSL(1, "SSL"), + /** SASL authenticated, non-encrypted channel */ + PLAINTEXTSASL(2, "PLAINTEXTSASL"), /** 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 new file mode 100644 index 0000000000000..a95d09bb1e486 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java @@ -0,0 +1,78 @@ +/** + * 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; + +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class JaasUtils { + private static final Logger LOG = LoggerFactory.getLogger(JaasUtils.class); + public static final String LOGIN_CONTEXT_SERVER = "KafkaServer"; + public static final String LOGIN_CONTEXT_CLIENT = "KafkaClient"; + public static final String SERVICE_NAME = "serviceName"; + // Oid mechanism = use Kerberos V5 as the security mechanism. + //public static final Oid KRB5_MECH_OID = new Oid("1.2.840.113554.1.2.2"); + + /** + * 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 { + 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]); + } +} 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..3864ffa40bb11 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -0,0 +1,261 @@ +/** + * 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.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.security.auth.PrincipalBuilder; +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.kerberos.KerberosName; +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.KafkaException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class SaslClientAuthenticator implements Authenticator { + + private static final Logger LOG = LoggerFactory.getLogger(SaslClientAuthenticator.class); + private SaslClient saslClient; + private Subject subject; + private String servicePrincipal; + private PrincipalBuilder principalBuilder; + private String host; + private String node = "0"; + private TransportLayer transportLayer; + private NetworkReceive netInBuffer; + private NetworkSend netOutBuffer; + private byte[] saslToken = new byte[0]; + + public enum SaslState { + INITIAL, INTERMEDIATE, COMPLETE, FAILED + } + + private SaslState saslState = SaslState.INITIAL; + + public SaslClientAuthenticator(Subject subject, String servicePrincipal, String host) throws IOException { + this.subject = subject; + this.host = host; + this.servicePrincipal = servicePrincipal; + this.saslClient = createSaslClient(); + } + + public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) throws KafkaException { + this.transportLayer = transportLayer; + this.principalBuilder = principalBuilder; + this.saslClient = createSaslClient(); + } + + private SaslClient createSaslClient() { + final Object[] principals = subject.getPrincipals().toArray(); + // determine client principal from subject. + final Principal clientPrincipal = (Principal) principals[0]; + final KerberosName clientKerberosName = new KerberosName(clientPrincipal.getName()); + // assume that server and client are in the same realm (by default; unless the config + // "kafka.server.realm" is set). + String serverRealm = System.getProperty("kafka.server.realm", clientKerberosName.realm()); + KerberosName serviceKerberosName = new KerberosName(servicePrincipal + "@" + serverRealm); + final String clientPrincipalName = clientKerberosName.toString(); + try { + saslClient = Subject.doAs(subject, new PrivilegedExceptionAction() { + public SaslClient run() throws SaslException { + LOG.debug("Client will use GSSAPI as SASL mechanism."); + String[] mechs = {"GSSAPI"}; + LOG.debug("creating sasl client: client=" + clientPrincipalName + ";service=" + servicePrincipal + ";serviceHostname=" + host); + SaslClient saslClient = Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null, new ClientCallbackHandler(null)); + return saslClient; + } + }); + return saslClient; + } catch (Exception e) { + LOG.error("Exception while trying to create SASL client", e); + throw new KafkaException("Failed to create SASL client", e); + } + } + + public void authenticate() throws IOException { + if (netOutBuffer != null && !flushNetOutBuffer()) { + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + return; + } + + if (saslClient.isComplete()) { + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + saslState = SaslState.COMPLETE; + return; + } + + byte[] serverToken = new byte[0]; + + if (saslState == SaslState.INTERMEDIATE) { + if (netInBuffer == null) netInBuffer = new NetworkReceive(node); + long readLen = netInBuffer.readFrom(transportLayer); + if (readLen != 0 && !netInBuffer.complete()) { + netInBuffer.payload().rewind(); + serverToken = new byte[netInBuffer.payload().remaining()]; + netInBuffer.payload().get(serverToken, 0, serverToken.length); + netInBuffer = null; // reset the networkReceive as we read all the data. + } + } else if (saslState == SaslState.INITIAL) { + saslState = SaslState.INTERMEDIATE; + } + + if (!(saslClient.isComplete())) { + try { + saslToken = createSaslToken(serverToken); + if (saslToken != null) { + netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken)); + if (!flushNetOutBuffer()) + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + } + } catch (SaslException se) { + saslState = SaslState.FAILED; + throw new IOException("Unable to authenticate using SASL " + se); + } + } + } + + public Principal principal() { + return new KafkaPrincipal("ANONYMOUS"); + } + + public boolean complete() { + return saslClient.isComplete() && 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 { + final byte[] retval = + Subject.doAs(subject, new PrivilegedExceptionAction() { + public byte[] run() throws SaslException { + return saslClient.evaluateChallenge(saslToken); + } + }); + return retval; + } 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."; + } + error += " Kafka Client will go to AUTH_FAILED state."; + LOG.error(error); + throw new SaslException(error); + } + } + + private boolean flushNetOutBuffer() throws IOException { + if (!netOutBuffer.completed()) { + netOutBuffer.writeTo(transportLayer); + } + return netOutBuffer.completed(); + } + + public static class ClientCallbackHandler implements CallbackHandler { + private String password = null; + + public ClientCallbackHandler(String password) { + this.password = password; + } + + 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) { + PasswordCallback pc = (PasswordCallback) callback; + if (password != null) { + pc.setPassword(this.password.toCharArray()); + } else { + 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(); + + if (authid.equals(authzid)) + ac.setAuthorized(true); + else + ac.setAuthorized(false); + + + 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..84110ab7d4b02 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -0,0 +1,197 @@ +/** + * 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.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.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 SaslServer saslServer; + private Subject subject; + private TransportLayer transportLayer; + private String node = "0"; + private NetworkReceive netInBuffer; + private NetworkSend netOutBuffer; + private SaslServerCallbackHandler saslServerCallbackHandler; + private PrincipalBuilder principalBuilder; + + public enum SaslState { + INITIAL, INTERMEDIATE, COMPLETE, FAILED + } + + private SaslState saslState = SaslState.INITIAL; + + public SaslServerAuthenticator(final Subject subject) throws IOException { + this.transportLayer = transportLayer; + this.subject = subject; + saslServer = createSaslServer(); + } + + public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) { + this.transportLayer = transportLayer; + this.principalBuilder = principalBuilder; + } + + private SaslServer createSaslServer() throws IOException { + if (subject != null) { + // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject. + if (subject.getPrincipals().size() > 0) { + try { + saslServerCallbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration()); + final Object[] principals = subject.getPrincipals().toArray(); + final Principal servicePrincipal = (Principal) principals[0]; + + final String servicePrincipalNameAndHostname = servicePrincipal.getName(); + int indexOf = servicePrincipalNameAndHostname.indexOf("/"); + final String servicePrincipalName = servicePrincipalNameAndHostname.substring(0, indexOf); + final String serviceHostnameAndKerbDomain = servicePrincipalNameAndHostname.substring(indexOf + 1, servicePrincipalNameAndHostname.length()); + indexOf = serviceHostnameAndKerbDomain.indexOf("@"); + final String serviceHostname = serviceHostnameAndKerbDomain.substring(0, indexOf); + final String mech = "GSSAPI"; + + LOG.debug("serviceHostname is '" + serviceHostname + "'"); + LOG.debug("servicePrincipalName is '" + servicePrincipalName + "'"); + LOG.debug("SASL mechanism(mech) is '" + mech + "'"); + boolean usingNativeJgss = Boolean.getBoolean("sun.security.jgss.native"); + if (usingNativeJgss) { + try { + GSSManager manager = GSSManager.getInstance(); + 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 { + SaslServer saslServer; + saslServer = Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler); + return saslServer; + } catch (SaslException e) { + LOG.error("Kafka Server failed to create a SaslServer to interact with a client during session authentication: " + e); + return null; + } + } + }); + } catch (PrivilegedActionException e) { + LOG.error("KafkaBroker experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context:" + e); + } + } catch (IndexOutOfBoundsException e) { + LOG.error("Kafka Server principal name/hostname determination error: ", e); + } + } + } + return null; + } + + public void authenticate() throws IOException { + if (netOutBuffer != null && !flushNetOutBuffer()) { + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + return; + } + + if (saslServer.isComplete()) return; + + byte[] clientToken = new byte[0]; + + if (netInBuffer == null) netInBuffer = new NetworkReceive(node); + + long readLen = netInBuffer.readFrom(transportLayer); + if (readLen == 0 || !netInBuffer.complete()) { + return; + } else { + netInBuffer.payload().rewind(); + 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; + response = saslServer.evaluateResponse(clientToken); + if (response != null) { + netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); + if (!flushNetOutBuffer()) { + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + return; + } + } + } catch (Exception e) { + throw new IOException(e); + } + + if (saslServer.isComplete() && !flushNetOutBuffer()) + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + } + + + + public Principal principal() { + return new KafkaPrincipal(saslServer.getAuthorizationID()); + } + + public boolean complete() { + return saslServer.isComplete(); + } + + public void close() throws IOException { + saslServer.dispose(); + } + + 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..e9a6c1f94e374 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java @@ -0,0 +1,83 @@ +/** + * 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.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); + + public SaslServerCallbackHandler(Configuration configuration) throws IOException { + AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(JaasUtils.LOGIN_CONTEXT_SERVER); + + if (configurationEntries == null) { + String errorMessage = "Could not find a 'KafkaServer' entry in this configuration: Kafka Server cannot start."; + LOG.error(errorMessage); + throw new IOException(errorMessage); + } + } + + 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=" + authenticationID + + "; authorizationID=" + authorizationID + "."); + ac.setAuthorized(true); + + KerberosName kerberosName = new KerberosName(authenticationID); + try { + StringBuilder userNameBuilder = new StringBuilder(kerberosName.shortName()); + LOG.info("Setting authorizedID: " + userNameBuilder); + ac.setAuthorizedID(userNameBuilder.toString()); + } 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..953394ab08aca --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosName.java @@ -0,0 +1,378 @@ +/** + * 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; + +import org.apache.kafka.common.security.JaasUtils; + + +/** + * 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 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; + + /** + * A pattern that matches a Kerberos name with at most 2 components. + */ + private static final Pattern NAMEPARSER = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)"); + + /** + * A pattern that matches a string with out '$' and then a single + * parameter with $n. + */ + private static final Pattern PARAMETERPATTERN = Pattern.compile("([^$]*)(\\$(\\d*))?"); + + /** + * A pattern for parsing a auth_to_local rule. + */ + private static final Pattern RULEPARSER = Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?" + + "(s/([^/]*)/([^/]*)/(g)?)?))"); + + /** + * A pattern that recognizes simple/non-simple names. + */ + private static final Pattern NONSIMPLEPATTERN = Pattern.compile("[/@]"); + + /** + * The list of translation rules. + */ + private static List rules; + + private static String defaultRealm; + + static { + try { + defaultRealm = JaasUtils.defaultRealm(); + } catch (Exception ke) { + throw new IllegalArgumentException("Could not configure Kerberos principal name mapping."); + } + } + + /** + * Create a name from the full Kerberos principal name. + * @param name + */ + public KerberosName(String name) { + Matcher match = NAMEPARSER.matcher(name); + if (!match.matches()) { + if (name.contains("@")) { + throw new IllegalArgumentException("Malformed Kerberos name: " + name); + } else { + serviceName = name; + hostName = null; + realm = null; + } + } else { + serviceName = match.group(1); + hostName = match.group(3); + realm = match.group(4); + } + } + + /** + * Get the configured default realm. + * @return the default realm from the krb5.conf + */ + public String getDefaultRealm() { + return defaultRealm; + } + + /** + * 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; + } + + /** + * An encoding of a rule for translating kerberos names. + */ + private static class Rule { + 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; + + Rule() { + isDefault = true; + numOfComponents = 0; + format = null; + match = null; + fromPattern = null; + toPattern = null; + repeat = false; + } + + Rule(int numOfComponents, String format, String match, String fromPattern, + String toPattern, boolean repeat) { + 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 BadFormatString + */ + static String replaceParameters(String format, + String[] params) throws BadFormatString { + Matcher match = PARAMETERPATTERN.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 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 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 && NONSIMPLEPATTERN.matcher(result).find()) { + throw new NoMatchingRule("Non-simple name " + result + + " after auth_to_local rule " + this); + } + return result; + } + } + + static List parseRules(String rules) { + List result = new ArrayList(); + String remaining = rules.trim(); + while (remaining.length() > 0) { + Matcher matcher = RULEPARSER.matcher(remaining); + if (!matcher.lookingAt()) { + throw new IllegalArgumentException("Invalid rule: " + remaining); + } + if (matcher.group(2) != null) { + result.add(new Rule()); + } else { + result.add(new Rule(Integer.parseInt(matcher.group(4)), + matcher.group(5), + matcher.group(7), + matcher.group(9), + matcher.group(10), + "g".equals(matcher.group(11)))); + } + remaining = remaining.substring(matcher.end()); + } + return result; + } + + /** + * Set the static configuration to get the rules. + * @param conf the new configuration + * @throws IOException + */ + public static void setConfiguration() throws IOException { + String ruleString = System.getProperty("kafka.security.auth_to_local", "DEFAULT"); + rules = parseRules(ruleString); + } + + @SuppressWarnings("serial") + public static class BadFormatString extends IOException { + BadFormatString(String msg) { + super(msg); + } + BadFormatString(String msg, Throwable err) { + super(msg, err); + } + } + + @SuppressWarnings("serial") + public static class NoMatchingRule extends IOException { + NoMatchingRule(String msg) { + super(msg); + } + } + + /** + * 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 (Rule r: rules) { + 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/Login.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java new file mode 100644 index 0000000000000..f23dfd3bb67ba --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java @@ -0,0 +1,414 @@ +/** + * 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; + +/** + * This class is responsible for refreshing Kerberos credentials for + * logins for both Kafka client and server. + */ + +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.utils.Shell; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.SystemTime; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.security.NoSuchAlgorithmException; +import java.security.URIParameter; +import java.util.Date; +import java.util.Random; +import java.util.Set; + +public class Login { + private static final Logger log = LoggerFactory.getLogger(Login.class); + + // 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 static final float TICKET_RENEW_WINDOW = 0.80f; + + /** + * Percentage of random jitter added to the renewal time + */ + private static final float TICKET_RENEW_JITTER = 0.05f; + + // Regardless of TICKET_RENEW_WINDOW 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 static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L; + + private volatile Subject subject = null; + private Thread t = null; + private boolean isKrbTicket = false; + private boolean isUsingTicketCache = false; + + /** Random number generator */ + private static Random rng = new Random(); + + private LoginContext login = null; + private String loginContextName = null; + private String jaasConfigFilePath = null; + private String principal = null; + private Configuration jaasConfig = null; + private Time time = new SystemTime(); + // Initialize 'lastLogin' to do a login at first time + private long lastLogin = time.currentElapsedTime() - MIN_TIME_BEFORE_RELOGIN; + public CallbackHandler callbackHandler = new ClientCallbackHandler(null); + + /** + * 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(). + * @throws javax.security.auth.login.LoginException + * Thrown if authentication fails. + */ + public Login(final String loginContextName) + throws LoginException { + this.loginContextName = loginContextName; + this.jaasConfigFilePath = jaasConfigFilePath; + login = login(loginContextName); + subject = login.getSubject(); + isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); + AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); + for (AppConfigurationEntry entry: entries) { + // there will only be a single entry, so this for() loop will only be iterated through once. + if (entry.getOptions().get("useTicketCache") != null) { + String val = (String) entry.getOptions().get("useTicketCache"); + if (val.equals("true")) + isUsingTicketCache = true; + } + if (entry.getOptions().get("principal") != null) + principal = (String) entry.getOptions().get("principal"); + break; + } + + log.debug("checking if its isKrbTicket"); + if (!isKrbTicket) { + // if no TGT, do not bother with ticket management. + return; + } + log.debug("its a krb5ticket"); + + // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the + // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. 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 = new 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 = time.currentWallTime(); + long nextRefresh; + Date nextRefreshDate; + if (tgt == null) { + nextRefresh = now + MIN_TIME_BEFORE_RELOGIN; + 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.getEndTime().equals(tgt.getRenewTill()))) { + log.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." + + "This process will not be able to authenticate new SASL connections after that " + + "time (for example, it will not be authenticate a new connection with a Kafka " + + "Broker). Ask your system administrator to either increase the " + + "'renew until' time by doing : 'modprinc -maxrenewlife " + principal + "' within " + + "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " + + "expiry cannot be further extended by refreshing, exiting refresh thread now."); + 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 + // MIN_TIME_BEFORE_RELOGIN. Will not sleep less than MIN_TIME_BEFORE_RELOGIN, unless doing so + // would cause ticket expiration. + if ((nextRefresh > expiry) || + ((now + MIN_TIME_BEFORE_RELOGIN) > 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 + MIN_TIME_BEFORE_RELOGIN)) { + // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN). + Date until = new Date(nextRefresh); + Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN); + log.warn("TGT refresh thread time adjusted from : " + until + " to : " + newuntil + " since " + + "the former is sooner than the minimum refresh interval (" + + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now."); + } + nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN); + } + nextRefreshDate = new Date(nextRefresh); + if (nextRefresh > expiry) { + log.error("next refresh: " + nextRefreshDate + " is later than expiry " + expiryDate + + ". This may indicate a clock skew problem. Check that this host and the KDC's " + + "hosts' clocks are in sync. Exiting refresh thread."); + return; + } + } + if (now < nextRefresh) { + Date until = new Date(nextRefresh); + log.info("TGT refresh sleeping until: " + until.toString()); + try { + Thread.sleep(nextRefresh - now); + } catch (InterruptedException ie) { + log.warn("TGT renewal thread has been interrupted and will exit."); + break; + } + } else { + log.error("nextRefresh:" + nextRefreshDate + " 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."); + return; + } + if (isUsingTicketCache) { + String cmd = "/usr/bin/kinit"; + if (System.getProperty("kafka.kinit") != null) { + cmd = System.getProperty("kafka.kinit"); + } + String kinitArgs = "-R"; + int retry = 1; + while (retry >= 0) { + try { + log.debug("running ticket cache refresh command: " + cmd + " " + kinitArgs); + Shell.execCommand(cmd, 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: '" + cmd + + " " + 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); + break; + } + } + } + }); + t.setDaemon(true); + } + + 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); + } + } + } + + public Subject subject() { + return subject; + } + + public String loginContextName() { + return loginContextName; + } + + private synchronized LoginContext login(final String loginContextName) throws LoginException { + if (loginContextName == null) { + throw new LoginException("loginContext name (JAAS file section header) was null. " + + "Please check your java.security.login.auth.config (=" + + System.getProperty("java.security.login.auth.config") + + ") and your " + JaasUtils.LOGIN_CONTEXT_SERVER + "(=" + + System.getProperty(JaasUtils.LOGIN_CONTEXT_CLIENT, "Client") + ")"); + } + + if (System.getProperty("java.security.auth.login.config") == null) { + throw new IllegalArgumentException("You must pass java.security.auth.login.config in secure mode."); + } + + File configFile = new File(System.getProperty("java.security.auth.login.config")); + Configuration loginConf = null; + try { + loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configFile.toURI())); + Configuration.setConfiguration(loginConf); + } catch (NoSuchAlgorithmException e) { + throw new RuntimeException(e); + } + + 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().toString()); + log.info("TGT expires: " + tgt.getEndTime().toString()); + long proposedRefresh = start + (long) ((expires - start) * + (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble()))); + + if (proposedRefresh > expires) + // proposedRefresh is too far in the future: it's after ticket expires: simply return now. + return time.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 = time.currentElapsedTime(); + if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN) { + log.warn("Not attempting to re-login since the last re-login was " + + "attempted less than " + (MIN_TIME_BEFORE_RELOGIN / 1000) + " seconds" + + " before."); + return false; + } + // register most recent relogin attempt + setLastLogin(now); + return true; + } + + /** + * Returns login object + * @return login + */ + private LoginContext getLogin() { + return login; + } + + /** + * Set the login object + * @param login + */ + private void setLogin(LoginContext login) { + this.login = login; + } + + /** + * Set the last login time. + * @param time the number of milliseconds since the beginning of time + */ + private void setLastLogin(long time) { + lastLogin = time; + } + + /** + * Get the time of the last login. + * @return the number of milliseconds since the beginning of time. + */ + private long getLastLogin() { + return lastLogin; + } + + /** + * 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; + } + LoginContext login = getLogin(); + if (login == null) { + throw new LoginException("login must be done first"); + } + if (!hasSufficientTimeElapsed()) { + return; + } + log.info("Initiating logout for " + principal); + synchronized (Login.class) { + //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(); + setLogin(login); + } + } +} 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..2b34367109ac9 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java @@ -0,0 +1,68 @@ +/** + * 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.Map; + +import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.Configurable; + + +public class LoginManager implements Configurable { + public enum Mode { CLIENT, SERVER }; + private Login login; + private final String serviceName; + private final String loginContext; + private final Mode mode; + + public LoginManager(Mode mode) throws IOException, LoginException { + this.mode = mode; + if (mode == Mode.SERVER) + this.loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; + else + this.loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; + this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); + } + + @Override + public void configure(Map configs) throws KafkaException { + try { + login = new Login(loginContext); + login.startThreadIfNeeded(); + } catch (Exception e) { + throw new KafkaException(e); + } + } + + public Subject subject() { + return login.subject(); + } + + public String serviceName() { + return serviceName; + } + + public void close() { + login.shutdown(); + } +} 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..aaa64e1493573 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java @@ -0,0 +1,335 @@ +/** + * 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); + + private int exitCode; + + /** return an array containing the command name & its parameters */ + protected abstract String[] execString(); + + /** Parse the execution result */ + protected abstract void parseExecResult(BufferedReader lines) + throws IOException; + + /**Time after which the executing script would be timedout*/ + protected long timeOutInterval = 0L; + + private long interval; // refresh interval in msec + private long lastTime; // last time the command was performed + private Process process; // sub process used to execute the command + + /**If or not script finished executing*/ + private volatile AtomicBoolean completed; + + public static final Time TIME = new SystemTime(); + + public Shell() { + this(0L); + } + + /** + * @param interval the minimum duration to wait before re-executing the + * command. + */ + public Shell(long interval) { + this.interval = interval; + this.lastTime = (interval < 0) ? 0 : -interval; + } + + /** 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; + } + + /** check to see if a command needs to be executed and execute if needed */ + protected void run() throws IOException { + if (lastTime + interval > TIME.currentElapsedTime()) + return; + exitCode = 0; // reset for next run + runCommand(); + } + + /** Run a command */ + private void runCommand() throws IOException { + ProcessBuilder builder = new ProcessBuilder(execString()); + Timer timeOutTimer = null; + ShellTimeoutTimerTask timeoutTimerTask = null; + completed = new AtomicBoolean(false); + + process = builder.start(); + if (timeOutInterval > 0) { + timeOutTimer = new Timer(); + timeoutTimerTask = new ShellTimeoutTimerTask( + this); + //One time scheduling. + timeOutTimer.schedule(timeoutTimerTask, timeOutInterval); + } + 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 = new Thread() { + @Override + public void run() { + try { + String line = errReader.readLine(); + while ((line != null) && !isInterrupted()) { + errMsg.append(line); + errMsg.append(System.getProperty("line.separator")); + line = errReader.readLine(); + } + } catch (IOException ioe) { + LOG.warn("Error reading the error stream", ioe); + } + } + }; + try { + errThread.start(); + } catch (IllegalStateException ise) { } + try { + parseExecResult(inReader); // parse the output + // clear the input stream buffer + String line = inReader.readLine(); + 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(); + lastTime = TIME.currentElapsedTime(); + } + } + + + /** + * 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 String[] command; + private StringBuffer output; + + public ShellCommandExecutor(String[] execString) { + this(execString, 0L); + } + + + /** + * 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 and the status marked as timedout. + * If 0, the command will not be timed out. + */ + + public ShellCommandExecutor(String[] execString, long timeout) { + command = execString.clone(); + timeOutInterval = timeout; + } + + + /** Execute the shell command. */ + public void execute() throws IOException { + this.run(); + } + + protected String[] execString() { + return command; + } + + protected void parseExecResult(BufferedReader lines) throws IOException { + output = new StringBuffer(); + char[] buf = new char[512]; + int nRead; + while ((nRead = lines.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, 0L); + } + + /** + * 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 marked 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 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/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index 18725de836838..348b7a8386de6 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.common.utils; +import java.util.Date; + /** * A time implementation that uses the system clock and sleep call */ @@ -40,4 +42,20 @@ public void sleep(long ms) { } } + @Override + public long currentElapsedTime() { + return System.nanoTime() / 1000000; + } + + @Override + public long currentWallTime() { + return System.currentTimeMillis(); + } + + @Override + public Date elapsedTimeToDate(long elapsedTime) { + long wallTime = currentWallTime() + elapsedTime - currentElapsedTime(); + return new Date(wallTime); + } + } 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..6994fd56809d2 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.common.utils; +import java.util.Date; /** * An interface abstracting the clock to use in unit testing classes that make use of clock time */ @@ -31,9 +32,30 @@ public interface Time { */ public long nanoseconds(); + /** + * Returns time in milliseconds as does System.currentTimeMillis(), + * but uses elapsed time from an arbitrary epoch more like System.nanoTime(). + * The difference is that if somebody changes the system clock, + * Time.currentElapsedTime will change but nanoTime won't. + * @return The time in milliseconds from some arbitrary point in time. + */ + public long currentElapsedTime(); + + /** + * Explicitly returns system dependent current wall time. + * @return Current time in msec. + */ + public long currentWallTime(); + /** * Sleep for the given number of milliseconds */ public void sleep(long ms); + /** + * This is to convert the elapsedTime to a Date. + * @return A date object indicated by the elapsedTime. + */ + public Date elapsedTimeToDate(long elapsedTime); + } 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..9b83c4ab6d007 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. @@ -13,6 +13,7 @@ package org.apache.kafka.common.utils; import java.util.concurrent.TimeUnit; +import java.util.Date; /** * A clock that you can manually advance by calling sleep @@ -40,4 +41,20 @@ public void sleep(long ms) { this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); } + @Override + public long currentElapsedTime() { + return 0; + } + + @Override + public long currentWallTime() { + return 0; + } + + @Override + public Date elapsedTimeToDate(long elapsedTime) { + return null; + } + + } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index d46603b32d922..deb7de46ce8ab 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -32,8 +32,10 @@ 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.{ChannelBuilders, InvalidReceiveException, ChannelBuilder, PlaintextChannelBuilder, SSLChannelBuilder} +import org.apache.kafka.common.network.{InvalidReceiveException, ChannelBuilder, + PlaintextChannelBuilder, SSLChannelBuilder} import org.apache.kafka.common.security.ssl.SSLFactory +import org.apache.kafka.common.security.kerberos.LoginFactory import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.protocol.types.SchemaException import org.apache.kafka.common.utils.{SystemTime, Time, Utils} From a3417d7f2c558c0082799b117a3c62c706ad519d Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 2 Sep 2015 20:31:34 -0700 Subject: [PATCH 02/50] KAFKA-1686. Implement SASL/Kerberos. --- build.gradle | 3 +- .../org/apache/kafka/clients/ClientUtils.java | 2 +- .../kafka/common/network/KafkaChannel.java | 4 +- .../network/PlaintextTransportLayer.java | 8 +- .../common/network/SaslChannelBuilder.java | 19 +-- .../apache/kafka/common/network/Selector.java | 6 +- .../SaslClientAuthenticator.java | 53 +++---- .../SaslServerAuthenticator.java | 45 +++--- .../security/kerberos/KerberosName.java | 5 + .../kafka/common/security/kerberos/Login.java | 1 - .../security/kerberos/LoginManager.java | 23 +++- .../apache/kafka/copycat/util/MockTime.java | 16 +++ .../scala/kafka/network/SocketServer.scala | 4 +- core/src/test/resources/kafka_jaas.conf | 18 +++ .../kafka/api/SaslProducerTest.scala | 130 ++++++++++++++++++ .../kafka/api/SaslTestHarness.scala | 48 +++++++ .../scala/unit/kafka/utils/TestUtils.scala | 13 +- gradle.properties | 2 +- 18 files changed, 319 insertions(+), 81 deletions(-) create mode 100644 core/src/test/resources/kafka_jaas.conf create mode 100755 core/src/test/scala/integration/kafka/api/SaslProducerTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SaslTestHarness.scala diff --git a/build.gradle b/build.gradle index fecc3eb3b6918..296f764e9e965 100644 --- a/build.gradle +++ b/build.gradle @@ -261,8 +261,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.0' testRuntime "$slf4jlog4j" - zinc 'com.typesafe.zinc:zinc:0.3.7' } @@ -275,6 +275,7 @@ project(':core') { compile.exclude module: 'jmxtools' compile.exclude module: 'mail' compile.exclude module: 'netty' + testCompile.exclude module: 'api-ldap-schema-data' } tasks.create(name: "copyDependantLibs", type: Copy) { 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..c42dc6c06e6bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -74,7 +74,7 @@ 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) + if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.PLAINTEXTSASL) throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); return ChannelBuilders.create(securityProtocol, SSLFactory.Mode.CLIENT, configs); } 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 28a4f4198f9ee..79c0de8dbb6d8 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 @@ -63,11 +63,9 @@ public Principal principal() throws IOException { * 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()) + else if (transportLayer.ready() && !authenticator.complete()) authenticator.authenticate(); } 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 9bfa3a1c5b15e..1bb171fe1e887 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 @@ -34,7 +34,7 @@ public class PlaintextTransportLayer implements TransportLayer { private static final Logger log = LoggerFactory.getLogger(PlaintextTransportLayer.class); - private final SelectionKey key; + public final SelectionKey key; private final SocketChannel socketChannel; private final Principal principal = KafkaPrincipal.ANONYMOUS; @@ -51,10 +51,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 @@ -213,4 +210,5 @@ public void removeInterestOps(int ops) { public boolean isMute() { return key.isValid() && (key.interestOps() & SelectionKey.OP_READ) == 0; } + } 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 index d30a5305742f1..a021ac34e28a8 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -17,7 +17,7 @@ import java.util.Map; import org.apache.kafka.common.security.auth.PrincipalBuilder; -import org.apache.kafka.common.security.kerberos.LoginFactory; +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.config.SSLConfigs; @@ -29,18 +29,18 @@ public class SaslChannelBuilder implements ChannelBuilder { private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class); - private LoginFactory loginFactory; + private LoginManager loginManager; private PrincipalBuilder principalBuilder; - private LoginFactory.Mode mode; + private LoginManager.Mode mode; - public SaslChannelBuilder(LoginFactory.Mode mode) { + public SaslChannelBuilder(LoginManager.Mode mode) { this.mode = mode; } public void configure(Map configs) throws KafkaException { try { - this.loginFactory = new LoginFactory(mode); - this.loginFactory.configure(configs); + this.loginManager = LoginManager.getLoginManager(mode); + this.loginManager.configure(configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); } catch (Exception e) { @@ -54,10 +54,10 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize SocketChannel socketChannel = (SocketChannel) key.channel(); TransportLayer transportLayer = new PlaintextTransportLayer(key); Authenticator authenticator; - if (mode == LoginFactory.Mode.SERVER) - authenticator = new SaslServerAuthenticator(loginFactory.subject()); + if (mode == LoginManager.Mode.SERVER) + authenticator = new SaslServerAuthenticator(id, loginManager.subject()); else - authenticator = new SaslClientAuthenticator(loginFactory.subject(), loginFactory.serviceName(), socketChannel.socket().getInetAddress().getHostName()); + authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(), socketChannel.socket().getInetAddress().getHostName()); authenticator.configure(transportLayer, this.principalBuilder); channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); } catch (Exception e) { @@ -69,5 +69,6 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize public void close() { this.principalBuilder.close(); + this.loginManager.close(); } } 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 4aa5cbb86ce6e..bdc06ec43cf1b 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 @@ -288,8 +288,12 @@ public void poll(long timeout) throws IOException { } /* if channel is not ready finish prepare */ - if (channel.isConnected() && !channel.ready()) + if (channel.isConnected() && !channel.ready()) { channel.prepare(); + if (channel.id().equals("-1")) { + + } + } /* if channel is ready read from any connections that have readable data */ if (channel.ready() && key.isReadable() && !hasStagedReceive(channel)) { 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 index 3864ffa40bb11..f1f6fd4d1a044 100644 --- 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 @@ -59,7 +59,7 @@ public class SaslClientAuthenticator implements Authenticator { private String servicePrincipal; private PrincipalBuilder principalBuilder; private String host; - private String node = "0"; + private String node; private TransportLayer transportLayer; private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; @@ -71,7 +71,8 @@ public enum SaslState { private SaslState saslState = SaslState.INITIAL; - public SaslClientAuthenticator(Subject subject, String servicePrincipal, String host) throws IOException { + public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host) throws IOException { + this.node = node; this.subject = subject; this.host = host; this.servicePrincipal = servicePrincipal; @@ -111,34 +112,40 @@ public SaslClient run() throws SaslException { } } + public void authenticate() throws IOException { if (netOutBuffer != null && !flushNetOutBuffer()) { transportLayer.addInterestOps(SelectionKey.OP_WRITE); return; } - - if (saslClient.isComplete()) { - transportLayer.removeInterestOps(SelectionKey.OP_WRITE); - saslState = SaslState.COMPLETE; - return; - } - byte[] serverToken = new byte[0]; - - if (saslState == SaslState.INTERMEDIATE) { - if (netInBuffer == null) netInBuffer = new NetworkReceive(node); - long readLen = netInBuffer.readFrom(transportLayer); - if (readLen != 0 && !netInBuffer.complete()) { - netInBuffer.payload().rewind(); - serverToken = new byte[netInBuffer.payload().remaining()]; - netInBuffer.payload().get(serverToken, 0, serverToken.length); - netInBuffer = null; // reset the networkReceive as we read all the data. - } - } else if (saslState == SaslState.INITIAL) { - saslState = SaslState.INTERMEDIATE; + switch (saslState) { + case INITIAL: + sendSaslToken(serverToken); + saslState = SaslState.INTERMEDIATE; + break; + case INTERMEDIATE: + if (netInBuffer == null) netInBuffer = new NetworkReceive(node); + long readLen = netInBuffer.readFrom(transportLayer); + if (netInBuffer.complete()) { + netInBuffer.payload().rewind(); + 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; + break; + case COMPLETE: + break; + case FAILED: + throw new IOException("SASL handshake failed"); } + } - if (!(saslClient.isComplete())) { + private void sendSaslToken(byte[] serverToken) throws IOException { + if (!saslClient.isComplete()) { try { saslToken = createSaslToken(serverToken); if (saslToken != null) { @@ -148,7 +155,7 @@ public void authenticate() throws IOException { } } catch (SaslException se) { saslState = SaslState.FAILED; - throw new IOException("Unable to authenticate using SASL " + se); + throw new IOException("Failed to authenticate using SASL " + se); } } } 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 index 84110ab7d4b02..5e505cda85f92 100644 --- 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 @@ -61,13 +61,8 @@ public class SaslServerAuthenticator implements Authenticator { private SaslServerCallbackHandler saslServerCallbackHandler; private PrincipalBuilder principalBuilder; - public enum SaslState { - INITIAL, INTERMEDIATE, COMPLETE, FAILED - } - - private SaslState saslState = SaslState.INITIAL; - - public SaslServerAuthenticator(final Subject subject) throws IOException { + public SaslServerAuthenticator(String node, final Subject subject) throws IOException { + this.node = node; this.transportLayer = transportLayer; this.subject = subject; saslServer = createSaslServer(); @@ -141,38 +136,36 @@ public void authenticate() throws IOException { return; } - if (saslServer.isComplete()) return; + if (saslServer.isComplete()) { + transportLayer.addInterestOps(SelectionKey.OP_READ); + return; + } byte[] clientToken = new byte[0]; if (netInBuffer == null) netInBuffer = new NetworkReceive(node); long readLen = netInBuffer.readFrom(transportLayer); - if (readLen == 0 || !netInBuffer.complete()) { - return; - } else { + + if (netInBuffer.complete()) { netInBuffer.payload().rewind(); 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; - response = saslServer.evaluateResponse(clientToken); - if (response != null) { - netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); - if (!flushNetOutBuffer()) { - transportLayer.addInterestOps(SelectionKey.OP_WRITE); - return; + try { + byte[] response; + response = saslServer.evaluateResponse(clientToken); + if (response != null) { + netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); + if (!flushNetOutBuffer()) { + transportLayer.addInterestOps(SelectionKey.OP_WRITE); + return; + } } + } catch (Exception e) { + throw new IOException(e); } - } catch (Exception e) { - throw new IOException(e); } - - if (saslServer.isComplete() && !flushNetOutBuffer()) - transportLayer.addInterestOps(SelectionKey.OP_WRITE); } 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 index 953394ab08aca..52445d2f3eee6 100644 --- 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 @@ -74,6 +74,11 @@ public class KerberosName { try { defaultRealm = JaasUtils.defaultRealm(); } catch (Exception ke) { + defaultRealm = ""; + } + try { + setConfiguration(); + } catch (IOException e) { throw new IllegalArgumentException("Could not configure Kerberos principal name mapping."); } } 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 index f23dfd3bb67ba..3684d9fe94b06 100644 --- 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 @@ -119,7 +119,6 @@ public Login(final String loginContextName) return; } log.debug("its a krb5ticket"); - // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. For testing and development, // you can decrease the interval of expiration of tickets (for example, to 3 minutes) by running : 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 index 2b34367109ac9..95d8a9c0529d6 100644 --- 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 @@ -34,24 +34,33 @@ public enum Mode { CLIENT, SERVER }; private final String serviceName; private final String loginContext; private final Mode mode; + volatile private static LoginManager instance; - public LoginManager(Mode mode) throws IOException, LoginException { + private LoginManager(Mode mode) throws IOException, LoginException { this.mode = mode; if (mode == Mode.SERVER) this.loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; else this.loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); + login = new Login(loginContext); + login.startThreadIfNeeded(); + } + + public static final LoginManager getLoginManager(Mode mode) throws IOException, LoginException { + if (instance != null) { + return instance; + } else { + synchronized (LoginManager.class) { + if (instance == null) + instance = new LoginManager(mode); + } + } + return instance; } @Override public void configure(Map configs) throws KafkaException { - try { - login = new Login(loginContext); - login.startThreadIfNeeded(); - } catch (Exception e) { - throw new KafkaException(e); - } } public Subject subject() { diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java index 53149db016f84..2f5fcde9e7745 100644 --- a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.utils.Time; import java.util.concurrent.TimeUnit; +import java.util.Date; /** * A clock that you can manually advance by calling sleep @@ -46,4 +47,19 @@ public void sleep(long ms) { this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); } + @Override + public long currentElapsedTime() { + return 0; + } + + @Override + public long currentWallTime() { + return 0; + } + + @Override + public Date elapsedTimeToDate(long elapsedTime) { + return null; + } + } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index deb7de46ce8ab..da55a828969ab 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -33,9 +33,9 @@ import kafka.utils._ import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ import org.apache.kafka.common.network.{InvalidReceiveException, ChannelBuilder, - PlaintextChannelBuilder, SSLChannelBuilder} + PlaintextChannelBuilder, SSLChannelBuilder, SaslChannelBuilder} import org.apache.kafka.common.security.ssl.SSLFactory -import org.apache.kafka.common.security.kerberos.LoginFactory +import org.apache.kafka.common.security.kerberos.LoginManager import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.protocol.types.SchemaException import org.apache.kafka.common.utils.{SystemTime, Time, Utils} diff --git a/core/src/test/resources/kafka_jaas.conf b/core/src/test/resources/kafka_jaas.conf new file mode 100644 index 0000000000000..c7e1a3eb7d9b7 --- /dev/null +++ b/core/src/test/resources/kafka_jaas.conf @@ -0,0 +1,18 @@ +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/SaslProducerTest.scala b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala new file mode 100755 index 0000000000000..a0eb2203c8507 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala @@ -0,0 +1,130 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.api + +import java.util +import java.util.Properties + +import kafka.admin.AdminUtils +import kafka.common.{ErrorMapping, FailedToSendMessageException} +import kafka.consumer.SimpleConsumer +import kafka.message.Message +import kafka.producer.KeyedMessage +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} +import kafka.utils._ +import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.protocol.SecurityProtocol +import org.apache.kafka.clients.producer._ + +import org.junit.Assert._ +import org.junit.{After, Before, Test} + + +class SaslProducerTest extends SaslTestHarness { + private val brokerId1 = 0 + private var server1: KafkaServer = null + private var consumer1: SimpleConsumer = null + private var servers = List.empty[KafkaServer] + + @Before + override def setUp() { + super.setUp() + // set up 2 brokers with 4 partitions each + val props1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, false, enableSasl = true) + props1.put("num.partitions", "1") + props1.put(KafkaConfig.DefaultReplicationFactorProp, "1") + val config1 = KafkaConfig.fromProps(props1) + + server1 = TestUtils.createServer(config1) + servers = List(server1) + consumer1 = new SimpleConsumer("localhost", server1.boundPort(SecurityProtocol.PLAINTEXT), 1000000, 64*1024, "") + } + + @After + override def tearDown() { + consumer1.close() + server1.shutdown + CoreUtils.rm(server1.config.logDirs) + super.tearDown() + } + + @Test + def testProduceAndConsume() { + val topic = "new-topic" + val producer = TestUtils.createNewProducer(TestUtils.getSaslBrokerListStrFromServers(servers), enableSasl=true) + val partition = new Integer(0) + val numRecords = 100 + + 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 with 1 partition and await leadership + TestUtils.createTopic(zkClient, topic, 1, 1, servers) + + val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes) + assertEquals("Should have offset 0", 0L, producer.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, producer.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, producer.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, producer.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) + producer.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 producer + for (i <- 1 to numRecords) + producer.send(record0, callback) + // check that all messages have been acked via offset + assertEquals("Should have offset " + numRecords + 4L, numRecords + 4L, producer.send(record0, callback).get.offset) + + + } finally { + producer.close() + } + } +} 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..36de671335af6 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -0,0 +1,48 @@ +package kafka.api + +import java.io.{FileWriter, BufferedWriter, File} +import java.net.URL +import javax.security.auth.login.Configuration + +import kafka.utils.TestUtils +import kafka.zk.ZooKeeperTestHarness +import org.apache.hadoop.minikdc.MiniKdc +import org.junit.{After, Before} +import org.scalatest.junit.JUnitSuite + + +trait SaslTestHarness extends ZooKeeperTestHarness { + val WorkDir = new File(System.getProperty("test.dir", "target")); + val KdcConf = MiniKdc.createConf(); + val Kdc: MiniKdc = new MiniKdc(KdcConf, WorkDir); + + @Before + override def setUp() { + val keytabFile: File = TestUtils.tempFile() + val jaasFile: File = TestUtils.tempFile() + + val writer: BufferedWriter = new BufferedWriter(new FileWriter(jaasFile)) + + val path: String = Thread.currentThread().getContextClassLoader.getResource("kafka_jaas.conf").getPath + for(line <- io.Source.fromFile(path).getLines()) { + val s: String = "\\$keytab-location" + val replaced = line.replaceAll("\\$keytab-location",keytabFile.getAbsolutePath) + writer.write(replaced) + writer.newLine() + } + writer.close() + + Kdc.start() + Kdc.createPrincipal(keytabFile, "client", "kafka/localhost") + System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath) + super.setUp + } + + @After + override def tearDown() { + super.tearDown + Kdc.stop() + System.clearProperty("java.security.auth.login.config") + Configuration.setConfiguration(null) + } +} diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 09b8444c2add8..eb8c5b06e79a4 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -154,18 +154,26 @@ object TestUtils extends Logging { servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.SSL))).mkString(",") } + def getSaslBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { + servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.PLAINTEXTSASL))).mkString(",") + } + /** * Create a test config for the given node id */ 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, enableSasl:Boolean = false, saslPort:Int = RandomPort, enableSSL: Boolean = false, + sslPort: Int = RandomPort, trustStoreFile: Option[File] = None): Properties = { + 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 + if (enableSasl) + listeners = listeners + "," + "PLAINTEXTSASL://localhost:" + saslPort.toString props.put("listeners", listeners) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", zkConnect) @@ -402,6 +410,7 @@ object TestUtils extends Logging { bufferSize: Long = 1024L * 1024L, retries: Int = 0, lingerMs: Long = 0, + enableSasl: Boolean = false, enableSSL: Boolean = false, trustStoreFile: Option[File] = None) : KafkaProducer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.producer.ProducerConfig @@ -421,6 +430,8 @@ object TestUtils extends Logging { if (enableSSL) { producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") producerProps.putAll(addSSLConfigs(SSLFactory.Mode.CLIENT, false, trustStoreFile, "producer")) + } else if (enableSasl) { + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXTSASL") } new KafkaProducer[Array[Byte],Array[Byte]](producerProps) } diff --git a/gradle.properties b/gradle.properties index 6f18a50902ea9..3abfef3203445 100644 --- a/gradle.properties +++ b/gradle.properties @@ -17,4 +17,4 @@ group=org.apache.kafka version=0.8.3-SNAPSHOT scalaVersion=2.10.5 task=build -org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m +org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m -XX:-MaxFDLimit From 8f718ce6b03a9c86712dc8f960af2b739b8ed510 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Wed, 2 Sep 2015 21:10:40 -0700 Subject: [PATCH 03/50] KAFKA-1686. Implement SASL/Kerberos. --- .../org/apache/kafka/clients/ClientUtils.java | 3 ++- .../kafka/common/network/ChannelBuilders.java | 7 ++++++- .../org/apache/kafka/common/network/Mode.java | 19 +++++++++++++++++++ .../common/network/SSLChannelBuilder.java | 5 +++-- .../common/network/SaslChannelBuilder.java | 7 ++++--- .../SaslClientAuthenticator.java | 2 +- .../SaslServerAuthenticator.java | 2 +- .../security/kerberos/LoginManager.java | 2 +- .../kafka/common/security/ssl/SSLFactory.java | 2 +- .../kafka/common/network/EchoServer.java | 2 +- .../kafka/common/network/SSLSelectorTest.java | 6 +++--- .../common/security/ssl/SSLFactoryTest.java | 9 +++++---- .../org/apache/kafka/test/TestSSLUtils.java | 9 +++++---- .../controller/ControllerChannelManager.scala | 5 ++--- .../scala/kafka/network/SocketServer.scala | 4 ++-- .../main/scala/kafka/server/KafkaServer.scala | 6 +++--- .../scala/unit/kafka/utils/TestUtils.scala | 11 ++++++----- 17 files changed, 65 insertions(+), 36 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/network/Mode.java 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 c42dc6c06e6bc..fd40537ad36ce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -20,6 +20,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.network.ChannelBuilders; +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; @@ -76,7 +77,7 @@ public static ChannelBuilder createChannelBuilder(Map configs) { SecurityProtocol securityProtocol = SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.PLAINTEXTSASL) throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); - return ChannelBuilders.create(securityProtocol, SSLFactory.Mode.CLIENT, configs); + return ChannelBuilders.create(securityProtocol, Mode.CLIENT, configs); } } 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..2a1e14762aea2 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 @@ -29,7 +29,7 @@ private ChannelBuilders() { } * @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) { + public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode, Map configs) { ChannelBuilder channelBuilder = null; switch (securityProtocol) { @@ -38,6 +38,11 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, SSLFactor throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `SSL`"); channelBuilder = new SSLChannelBuilder(mode); break; + case PLAINTEXTSASL: + if (mode == null) + throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `PLAINTEXTSASL`"); + channelBuilder = new SaslChannelBuilder(mode); + break; case PLAINTEXT: case TRACE: channelBuilder = new PlaintextChannelBuilder(); 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/SSLChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java index 88c218b04ee85..4ac0340dfda9e 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 @@ -16,6 +16,7 @@ import java.nio.channels.SocketChannel; import java.util.Map; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.security.auth.PrincipalBuilder; import org.apache.kafka.common.security.ssl.SSLFactory; import org.apache.kafka.common.config.SSLConfigs; @@ -28,9 +29,9 @@ 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; - public SSLChannelBuilder(SSLFactory.Mode mode) { + public SSLChannelBuilder(Mode mode) { this.mode = mode; } 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 index a021ac34e28a8..944b9c2293e22 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator; import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator; import org.apache.kafka.common.config.SSLConfigs; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.KafkaException; @@ -31,9 +32,9 @@ public class SaslChannelBuilder implements ChannelBuilder { private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class); private LoginManager loginManager; private PrincipalBuilder principalBuilder; - private LoginManager.Mode mode; + private Mode mode; - public SaslChannelBuilder(LoginManager.Mode mode) { + public SaslChannelBuilder(Mode mode) { this.mode = mode; } @@ -54,7 +55,7 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize SocketChannel socketChannel = (SocketChannel) key.channel(); TransportLayer transportLayer = new PlaintextTransportLayer(key); Authenticator authenticator; - if (mode == LoginManager.Mode.SERVER) + if (mode == Mode.SERVER) authenticator = new SaslServerAuthenticator(id, loginManager.subject()); else authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(), socketChannel.socket().getInetAddress().getHostName()); 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 index f1f6fd4d1a044..02e8c6e039f8f 100644 --- 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 @@ -161,7 +161,7 @@ private void sendSaslToken(byte[] serverToken) throws IOException { } public Principal principal() { - return new KafkaPrincipal("ANONYMOUS"); + return KafkaPrincipal.ANONYMOUS; } public boolean complete() { 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 index 5e505cda85f92..6fc0300949ef7 100644 --- 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 @@ -171,7 +171,7 @@ public void authenticate() throws IOException { public Principal principal() { - return new KafkaPrincipal(saslServer.getAuthorizationID()); + return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, saslServer.getAuthorizationID()); } public boolean complete() { 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 index 95d8a9c0529d6..a62b390202dd8 100644 --- 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 @@ -23,13 +23,13 @@ import java.io.IOException; import java.util.Map; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.security.JaasUtils; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Configurable; public class LoginManager implements Configurable { - public enum Mode { CLIENT, SERVER }; private Login login; private final String serviceName; private final String loginContext; 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 f79b65cdfd014..8e0241124a4c4 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 @@ -25,6 +25,7 @@ import javax.net.ssl.*; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.config.SSLConfigs; @@ -32,7 +33,6 @@ public class SSLFactory implements Configurable { - public enum Mode { CLIENT, SERVER }; private String protocol; private String provider; private String kmfAlgorithm; 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 df1205c935bee..c28c2a0b75cb6 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 @@ -50,14 +50,14 @@ public class SSLSelectorTest { 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(); - 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.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", new LinkedHashMap(), channelBuilder); } 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/test/TestSSLUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java index c01cf3748ff16..8b7304d55dc01 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java @@ -19,6 +19,7 @@ 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 +178,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 +204,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 da1cff07f7f76..0a39facdab0b7 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.{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, 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 da55a828969ab..6b9e222ef27af 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -32,7 +32,7 @@ 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.{InvalidReceiveException, ChannelBuilder, +import org.apache.kafka.common.network.{Mode, InvalidReceiveException, ChannelBuilders, PlaintextChannelBuilder, SSLChannelBuilder, SaslChannelBuilder} import org.apache.kafka.common.security.ssl.SSLFactory import org.apache.kafka.common.security.kerberos.LoginManager @@ -369,7 +369,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, channelConfigs) private val metricTags = new util.HashMap[String, String]() metricTags.put("networkProcessor", id.toString) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 30406ce809caa..5e2b9d9f4339f 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -34,7 +34,7 @@ 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.{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} @@ -300,7 +300,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg "kafka-server-controlled-shutdown", Map.empty.asJava, false, - ChannelBuilders.create(config.interBrokerSecurityProtocol, SSLFactory.Mode.CLIENT, config.channelConfigs) + ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, config.channelConfigs) ) new NetworkClient( selector, @@ -482,7 +482,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if (config.interBrokerProtocolVersion.onOrAfter(KAFKA_083)) 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/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index eb8c5b06e79a4..4310897bcec0a 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -50,6 +50,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 @@ -183,7 +184,7 @@ object TestUtils extends Logging { 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)) + props.putAll(addSSLConfigs(Mode.SERVER, true, trustStoreFile, "server"+nodeId)) } props.put("port", port.toString) props @@ -429,7 +430,7 @@ object TestUtils extends Logging { 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")) + producerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "producer")) } else if (enableSasl) { producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXTSASL") } @@ -462,7 +463,7 @@ object TestUtils extends Logging { 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")) + consumerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "consumer")) } new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } @@ -920,12 +921,12 @@ object TestUtils extends Logging { new String(bytes, encoding) } - def addSSLConfigs(mode: SSLFactory.Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { + def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { var sslConfigs: java.util.Map[String, Object] = new java.util.HashMap[String, Object]() if (!trustStoreFile.isDefined) { throw new Exception("enableSSL set to true but no trustStoreFile provided") } - if (mode == SSLFactory.Mode.SERVER) + if (mode == Mode.SERVER) sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias) else sslConfigs = TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) From aa928952305a31c5b6e2bac705d350f94c9f7501 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 3 Sep 2015 06:48:47 -0700 Subject: [PATCH 04/50] Added licesense. --- core/src/test/resources/kafka_jaas.conf | 13 ++++++++++++- .../integration/kafka/api/SaslTestHarness.scala | 12 ++++++++++++ 2 files changed, 24 insertions(+), 1 deletion(-) diff --git a/core/src/test/resources/kafka_jaas.conf b/core/src/test/resources/kafka_jaas.conf index c7e1a3eb7d9b7..b097e260b7a82 100644 --- a/core/src/test/resources/kafka_jaas.conf +++ b/core/src/test/resources/kafka_jaas.conf @@ -1,3 +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. + */ KafkaClient { com.sun.security.auth.module.Krb5LoginModule required debug=true useKeyTab=true @@ -15,4 +27,3 @@ KafkaServer { keyTab="$keytab-location" principal="kafka/localhost@EXAMPLE.COM"; }; - diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 36de671335af6..5cd01bbaafeee 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -1,3 +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 import java.io.{FileWriter, BufferedWriter, File} From f178107b516af414162634fc7253cedd2a6a3bf5 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 3 Sep 2015 06:57:57 -0700 Subject: [PATCH 05/50] KAFKA-1686. Implement SASL/Kerberos. --- clients/src/main/java/org/apache/kafka/clients/ClientUtils.java | 1 - .../java/org/apache/kafka/common/network/ChannelBuilders.java | 1 - .../java/org/apache/kafka/common/network/SSLChannelBuilder.java | 1 - .../java/org/apache/kafka/common/network/SaslChannelBuilder.java | 1 - .../java/org/apache/kafka/common/network/SSLSelectorTest.java | 1 - clients/src/test/java/org/apache/kafka/test/TestSSLUtils.java | 1 - 6 files changed, 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index fd40537ad36ce..2d6b3e397e9b2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -23,7 +23,6 @@ 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; 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 2a1e14762aea2..7c20221f73525 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; 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 4ac0340dfda9e..a12e2b9ae4864 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 @@ -16,7 +16,6 @@ import java.nio.channels.SocketChannel; import java.util.Map; -import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.security.auth.PrincipalBuilder; import org.apache.kafka.common.security.ssl.SSLFactory; import org.apache.kafka.common.config.SSLConfigs; 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 index 944b9c2293e22..e0b79414f0918 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator; import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator; import org.apache.kafka.common.config.SSLConfigs; -import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.KafkaException; 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 c28c2a0b75cb6..167d1e7fa57b5 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 @@ -24,7 +24,6 @@ import java.nio.ByteBuffer; import org.apache.kafka.common.config.SSLConfigs; -import org.apache.kafka.common.security.ssl.SSLFactory; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Utils; 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 8b7304d55dc01..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,6 @@ 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; From 9d260c67472296d752f74bc04eefb1e95b6b9746 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Sun, 4 Oct 2015 11:36:52 -0700 Subject: [PATCH 06/50] KAFKA-1686. Fixes after the merge. --- .../main/scala/kafka/network/SocketServer.scala | 2 +- .../kafka/server/ReplicaFetcherThread.scala | 4 ++-- .../test/scala/unit/kafka/utils/TestUtils.scala | 17 ----------------- 3 files changed, 3 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 6b72be4bb2895..4922c6ab924d4 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -32,7 +32,7 @@ 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.{Mode, InvalidReceiveException, ChannelBuilder, +import org.apache.kafka.common.network.{Mode, InvalidReceiveException, ChannelBuilders, PlaintextChannelBuilder, SSLChannelBuilder, SaslChannelBuilder} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.ssl.SSLFactory diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 82a60018acdd3..7123ded20a788 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.{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, brokerConfig.channelConfigs) ) new NetworkClient( selector, diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index d64197dcaf327..9174d8b6ea465 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -920,7 +920,6 @@ object TestUtils extends Logging { new String(bytes, encoding) } -<<<<<<< HEAD def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { var sslConfigs: java.util.Map[String, Object] = new java.util.HashMap[String, Object]() if (!trustStoreFile.isDefined) { @@ -930,22 +929,6 @@ object TestUtils extends Logging { sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias) else sslConfigs = TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) -======= - 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") - } - - 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 - } - else - TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) - } ->>>>>>> refs/remotes/origin/trunk val sslProps = new Properties() sslConfigs.foreach { case (k, v) => sslProps.put(k, v) } From 5723dd2a392a307cfd6484c1f3f7c32cc8891940 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 8 Oct 2015 23:43:51 -0700 Subject: [PATCH 07/50] KAFKA-1686. Addressing comments. --- build.gradle | 2 +- .../clients/consumer/ConsumerConfig.java | 3 + .../clients/producer/ProducerConfig.java | 3 + .../kafka/common/config/SaslConfigs.java | 6 + .../kafka/common/network/Authenticator.java | 3 +- .../kafka/common/network/ChannelBuilders.java | 2 +- .../common/network/DefaultAuthenticator.java | 5 +- .../network/PlaintextChannelBuilder.java | 6 +- .../common/network/SSLChannelBuilder.java | 8 +- .../common/network/SaslChannelBuilder.java | 26 +++- .../common/protocol/SecurityProtocol.java | 2 + .../SaslClientAuthenticator.java | 111 ++++++++++-------- .../SaslServerAuthenticator.java | 3 +- .../kafka/api/SSLConsumerTest.scala | 9 +- .../kafka/api/SSLProducerSendTest.scala | 7 +- .../kafka/api/SaslProducerTest.scala | 2 +- .../scala/unit/kafka/utils/TestUtils.scala | 12 +- 17 files changed, 126 insertions(+), 84 deletions(-) diff --git a/build.gradle b/build.gradle index 42aaf7fb0a703..3dfb287c984f8 100644 --- a/build.gradle +++ b/build.gradle @@ -258,7 +258,7 @@ 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.0' + testCompile 'org.apache.hadoop:hadoop-minikdc:2.7.1' testRuntime "$slf4jlog4j" zinc 'com.typesafe.zinc:zinc:0.3.7' } 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 bcc143ce6689e..8b78f86032e22 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,8 @@ 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_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(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 c3dbd10479582..37a3667d5b96e 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,8 @@ 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_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) /* 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 index 9d299f47b75c3..a9c6c9f6843b1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -24,5 +24,11 @@ public class SaslConfigs { + "Default will be from kafka jaas config"; public static final String DEFAULT_KAFKA_SERVER_REALM = ""; + 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"; + + } 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..dc9c7919b6450 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; @@ -34,7 +35,7 @@ public interface Authenticator { * @param TransportLayer transportLayer * @param PrincipalBuilder principalBuilder */ - 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/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java index 7c20221f73525..52d716a218c11 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 @@ -40,7 +40,7 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode case PLAINTEXTSASL: if (mode == null) throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `PLAINTEXTSASL`"); - channelBuilder = new SaslChannelBuilder(mode); + channelBuilder = new SaslChannelBuilder(mode, securityProtocol); break; case PLAINTEXT: case TRACE: 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..7d5b91363ec5a 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; @@ -27,11 +28,13 @@ public class DefaultAuthenticator implements Authenticator { private TransportLayer transportLayer; private PrincipalBuilder principalBuilder; + private Map configs; 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; + this.configs = configs; } /** 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/SSLChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SSLChannelBuilder.java index a12e2b9ae4864..b79658b656d6b 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,6 +29,7 @@ public class SSLChannelBuilder implements ChannelBuilder { private SSLFactory sslFactory; private PrincipalBuilder principalBuilder; private Mode mode; + private Map configs; public SSLChannelBuilder(Mode mode) { this.mode = mode; @@ -36,10 +37,11 @@ public SSLChannelBuilder(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); } @@ -53,7 +55,7 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), socketChannel.socket().getPort())); 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); 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 index e0b79414f0918..ad84b8eec0d8b 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -20,6 +20,8 @@ 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; @@ -31,18 +33,27 @@ public class SaslChannelBuilder implements ChannelBuilder { private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class); private LoginManager loginManager; private PrincipalBuilder principalBuilder; + private SecurityProtocol securityProtocol; + private SSLFactory sslFactory; private Mode mode; + private Map configs; - public SaslChannelBuilder(Mode mode) { + public SaslChannelBuilder(Mode mode, SecurityProtocol securityProtocol) { this.mode = mode; + this.securityProtocol = securityProtocol; } public void configure(Map configs) throws KafkaException { try { + this.configs = configs; this.loginManager = LoginManager.getLoginManager(mode); - this.loginManager.configure(configs); + this.loginManager.configure(this.configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); + if (this.securityProtocol == SecurityProtocol.SSLSASL) { + this.sslFactory = new SSLFactory(mode); + this.sslFactory.configure(this.configs); + } } catch (Exception e) { throw new KafkaException(e); } @@ -52,13 +63,20 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize KafkaChannel channel = null; try { SocketChannel socketChannel = (SocketChannel) key.channel(); - TransportLayer transportLayer = new PlaintextTransportLayer(key); + TransportLayer transportLayer; + if (this.securityProtocol == SecurityProtocol.SSLSASL) { + transportLayer = new SSLTransportLayer(id, key, + sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), + socketChannel.socket().getPort())); + } else { + transportLayer = new PlaintextTransportLayer(key); + } Authenticator authenticator; if (mode == Mode.SERVER) authenticator = new SaslServerAuthenticator(id, loginManager.subject()); else authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(), socketChannel.socket().getInetAddress().getHostName()); - 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); 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 7db45b528f1c7..cc15472262fb3 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 @@ -28,6 +28,8 @@ public enum SecurityProtocol { SSL(1, "SSL"), /** SASL authenticated, non-encrypted channel */ PLAINTEXTSASL(2, "PLAINTEXTSASL"), + /** SASL authenticated, SSL channel */ + SSLSASL(2, "SASL"), /** 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/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 02e8c6e039f8f..fd728a45936a8 100644 --- 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 @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.SelectionKey; +import java.util.Map; import java.security.Principal; import java.security.PrivilegedActionException; @@ -39,12 +40,13 @@ import javax.security.sasl.SaslException; import org.apache.kafka.common.network.Authenticator; -import org.apache.kafka.common.security.auth.PrincipalBuilder; -import org.apache.kafka.common.security.auth.KafkaPrincipal; -import org.apache.kafka.common.security.kerberos.KerberosName; 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; +import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.kerberos.KerberosName; +import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.KafkaException; import org.slf4j.Logger; @@ -60,6 +62,12 @@ public class SaslClientAuthenticator implements Authenticator { private PrincipalBuilder principalBuilder; private String host; private String node; + private String serverRealm; + private Object[] principals; + private Principal clientPrincipal; + private KerberosName clientKerberosName; + private KerberosName serviceKerberosName; + private String clientPrincipalName; private TransportLayer transportLayer; private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; @@ -76,25 +84,32 @@ public SaslClientAuthenticator(String node, Subject subject, String servicePrinc this.subject = subject; this.host = host; this.servicePrincipal = servicePrincipal; - this.saslClient = createSaslClient(); } - public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) throws KafkaException { - this.transportLayer = transportLayer; - this.principalBuilder = principalBuilder; - this.saslClient = createSaslClient(); + public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) throws KafkaException { + try { + this.transportLayer = transportLayer; + this.principalBuilder = principalBuilder; + + this.principals = subject.getPrincipals().toArray(); + // determine client principal from subject. + this.clientPrincipal = (Principal) principals[0]; + this.clientKerberosName = new KerberosName(clientPrincipal.getName()); + // assume that server and client are in the same realm (by default; unless the config + // "kafka.server.realm" is set). + if (configs.containsKey(SaslConfigs.SASL_KAFKA_SERVER_REALM)) + this.serverRealm = (String) configs.get(SaslConfigs.SASL_KAFKA_SERVER_REALM); + else + this.serverRealm = clientKerberosName.realm(); + this.serviceKerberosName = new KerberosName(servicePrincipal + "@" + serverRealm); + this.clientPrincipalName = clientKerberosName.toString(); + this.saslClient = createSaslClient(); + } catch (Exception e) { + throw new KafkaException("Failed to configure SaslClientAuthenticator", e); + } } private SaslClient createSaslClient() { - final Object[] principals = subject.getPrincipals().toArray(); - // determine client principal from subject. - final Principal clientPrincipal = (Principal) principals[0]; - final KerberosName clientKerberosName = new KerberosName(clientPrincipal.getName()); - // assume that server and client are in the same realm (by default; unless the config - // "kafka.server.realm" is set). - String serverRealm = System.getProperty("kafka.server.realm", clientKerberosName.realm()); - KerberosName serviceKerberosName = new KerberosName(servicePrincipal + "@" + serverRealm); - final String clientPrincipalName = clientKerberosName.toString(); try { saslClient = Subject.doAs(subject, new PrivilegedExceptionAction() { public SaslClient run() throws SaslException { @@ -218,51 +233,43 @@ public ClientCallbackHandler(String password) { this.password = password; } - public void handle(Callback[] callbacks) throws - UnsupportedCallbackException { + 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) { - PasswordCallback pc = (PasswordCallback) callback; - if (password != null) { - pc.setPassword(this.password.toCharArray()); - } else { - 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 PasswordCallback) { + PasswordCallback pc = (PasswordCallback) callback; + if (password != null) { + pc.setPassword(this.password.toCharArray()); } 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(); + 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(); - if (authid.equals(authzid)) - ac.setAuthorized(true); - else - ac.setAuthorized(false); + if (authid.equals(authzid)) + ac.setAuthorized(true); + else + ac.setAuthorized(false); - if (ac.isAuthorized()) - ac.setAuthorizedID(authzid); - } else { - throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback"); - } - } - } + 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 index 6fc0300949ef7..b7c98a18dfa4c 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.kafka.common.security.authenticator; import java.io.IOException; +import java.util.Map; import java.nio.ByteBuffer; import java.nio.channels.SelectionKey; @@ -68,7 +69,7 @@ public SaslServerAuthenticator(String node, final Subject subject) throws IOExce saslServer = createSaslServer(); } - public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder) { + public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) { this.transportLayer = transportLayer; this.principalBuilder = principalBuilder; } diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala index 2f72c782be4df..69b2958b5cde1 100644 --- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala @@ -23,6 +23,7 @@ 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 org.apache.kafka.common.protocol.SecurityProtocol import kafka.integration.KafkaServerTestHarness import kafka.utils.{TestUtils, Logging} @@ -75,21 +76,21 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging { @Before override def setUp() { super.setUp() - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getSSLBrokerListStrFromServers(servers)) + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL)) 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.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL)) 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), + producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), acks = 1, enableSSL=true, trustStoreFile=Some(trustStoreFile)) for (i <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getSSLBrokerListStrFromServers(servers), + consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), groupId = "my-test", partitionAssignmentStrategy= "range", enableSSL=true, diff --git a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala index 0f706248625d4..967c9f443a612 100644 --- a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala @@ -30,6 +30,7 @@ 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.apache.kafka.common.protocol.SecurityProtocol import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -75,7 +76,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness { */ @Test def testSendOffset() { - var sslProducer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + var sslProducer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), enableSSL=true, trustStoreFile=Some(trustStoreFile)) var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers)) val partition = new Integer(0) @@ -156,7 +157,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness { */ @Test def testClose() { - var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), enableSSL=true, trustStoreFile=Some(trustStoreFile)) try { // create topic TestUtils.createTopic(zkClient, topic, 1, 2, servers) @@ -191,7 +192,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness { */ @Test def testSendToPartition() { - var producer = TestUtils.createNewProducer(TestUtils.getSSLBrokerListStrFromServers(servers), enableSSL=true, trustStoreFile=Some(trustStoreFile)) + var producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), enableSSL=true, trustStoreFile=Some(trustStoreFile)) try { // create topic val leaders = TestUtils.createTopic(zkClient, topic, 2, 2, servers) diff --git a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala index a0eb2203c8507..5ff5a3ad3f51b 100755 --- a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala @@ -67,7 +67,7 @@ class SaslProducerTest extends SaslTestHarness { @Test def testProduceAndConsume() { val topic = "new-topic" - val producer = TestUtils.createNewProducer(TestUtils.getSaslBrokerListStrFromServers(servers), enableSasl=true) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), enableSasl=true) val partition = new Integer(0) val numRecords = 100 diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 9174d8b6ea465..b255b3f913417 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -147,16 +147,8 @@ object TestUtils extends Logging { (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(",") - } - - def getSSLBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { - servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.SSL))).mkString(",") - } - - def getSaslBrokerListStrFromServers(servers: Seq[KafkaServer]): String = { - servers.map(s => formatAddress(s.config.hostName, s.boundPort(SecurityProtocol.PLAINTEXTSASL))).mkString(",") + def getBrokerListStrFromServers(servers: Seq[KafkaServer], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): String = { + servers.map(s => formatAddress(s.config.hostName, s.boundPort(protocol))).mkString(",") } /** From 2596c4a668f7095f4cfce36b34504c50f4603631 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 9 Oct 2015 13:21:05 +0100 Subject: [PATCH 08/50] Remove unused code, fix formatting and minor javadoc tweaks --- checkstyle/import-control.xml | 69 +++++++++---------- .../kafka/common/config/SaslConfigs.java | 7 +- .../kafka/common/network/Authenticator.java | 6 +- .../kafka/common/network/ChannelBuilder.java | 2 +- .../common/network/DefaultAuthenticator.java | 2 - .../apache/kafka/common/network/Selector.java | 6 +- .../kafka/common/security/kerberos/Login.java | 5 +- .../apache/kafka/common/utils/SystemTime.java | 8 --- .../org/apache/kafka/common/utils/Time.java | 7 -- .../apache/kafka/common/utils/MockTime.java | 7 -- .../scala/kafka/network/SocketServer.scala | 7 +- 11 files changed, 44 insertions(+), 82 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 533737ce3be72..6474865ef6bf6 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -22,47 +22,46 @@ - - - - - + + + + + - - + - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + + + + + + + + + + + + + + + + + + + + + + + + + + + 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 index a9c6c9f6843b1..1fce0feb47cfc 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -20,15 +20,12 @@ public class SaslConfigs { */ 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." + public static final String SASL_KAFKA_SERVER_DOC = "The sasl kafka server realm. " + "Default will be from kafka jaas config"; - public static final String DEFAULT_KAFKA_SERVER_REALM = ""; 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." + 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"; - - } 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 dc9c7919b6450..5a9af8ff52995 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 @@ -31,9 +31,9 @@ public interface Authenticator { /** - * configures Authenticator using principalbuilder and transportLayer. - * @param TransportLayer transportLayer - * @param PrincipalBuilder principalBuilder + * Configures Authenticator using principalBuilder and transportLayer. + * @param transportLayer + * @param principalBuilder */ void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs); 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 ddd7b8f045689..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 @@ -33,7 +33,7 @@ public interface ChannelBuilder { * @param id channel id * @param key SelectionKey * @param maxReceiveSize - * @returns KafkaChannel + * @return KafkaChannel */ KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException; 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 7d5b91363ec5a..74958959b3437 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 @@ -28,13 +28,11 @@ public class DefaultAuthenticator implements Authenticator { private TransportLayer transportLayer; private PrincipalBuilder principalBuilder; - private Map configs; private Principal principal; public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) { this.transportLayer = transportLayer; this.principalBuilder = principalBuilder; - this.configs = configs; } /** 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 6d5495d589252..7cdc1674a32ac 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 @@ -279,12 +279,8 @@ public void poll(long timeout) throws IOException { } /* if channel is not ready finish prepare */ - if (channel.isConnected() && !channel.ready()) { + if (channel.isConnected() && !channel.ready()) channel.prepare(); - if (channel.id().equals("-1")) { - - } - } /* if channel is ready read from any connections that have readable data */ if (channel.ready() && key.isReadable() && !hasStagedReceive(channel)) { 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 index 3684d9fe94b06..a75ded443842f 100644 --- 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 @@ -76,7 +76,6 @@ public class Login { private LoginContext login = null; private String loginContextName = null; - private String jaasConfigFilePath = null; private String principal = null; private Configuration jaasConfig = null; private Time time = new SystemTime(); @@ -93,10 +92,8 @@ public class Login { * @throws javax.security.auth.login.LoginException * Thrown if authentication fails. */ - public Login(final String loginContextName) - throws LoginException { + public Login(final String loginContextName) throws LoginException { this.loginContextName = loginContextName; - this.jaasConfigFilePath = jaasConfigFilePath; login = login(loginContextName); subject = login.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index 348b7a8386de6..cd75d209091a2 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.common.utils; -import java.util.Date; - /** * A time implementation that uses the system clock and sleep call */ @@ -52,10 +50,4 @@ public long currentWallTime() { return System.currentTimeMillis(); } - @Override - public Date elapsedTimeToDate(long elapsedTime) { - long wallTime = currentWallTime() + elapsedTime - currentElapsedTime(); - return new Date(wallTime); - } - } 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 6994fd56809d2..93434c2b2ad79 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.common.utils; -import java.util.Date; /** * An interface abstracting the clock to use in unit testing classes that make use of clock time */ @@ -52,10 +51,4 @@ public interface Time { */ public void sleep(long ms); - /** - * This is to convert the elapsedTime to a Date. - * @return A date object indicated by the elapsedTime. - */ - public Date elapsedTimeToDate(long elapsedTime); - } 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 9b83c4ab6d007..e92116d0f0536 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 @@ -13,7 +13,6 @@ package org.apache.kafka.common.utils; import java.util.concurrent.TimeUnit; -import java.util.Date; /** * A clock that you can manually advance by calling sleep @@ -51,10 +50,4 @@ public long currentWallTime() { return 0; } - @Override - public Date elapsedTimeToDate(long elapsedTime) { - return null; - } - - } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 4922c6ab924d4..2043e235adf84 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -32,14 +32,11 @@ 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.{Mode, InvalidReceiveException, ChannelBuilders, - PlaintextChannelBuilder, SSLChannelBuilder, SaslChannelBuilder} +import org.apache.kafka.common.network.{Mode, InvalidReceiveException, ChannelBuilders} import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.common.security.ssl.SSLFactory -import org.apache.kafka.common.security.kerberos.LoginManager import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.protocol.types.SchemaException -import org.apache.kafka.common.utils.{SystemTime, Time, Utils} +import org.apache.kafka.common.utils.{Time, Utils} import scala.collection._ import scala.util.control.{NonFatal, ControlThrowable} From 2919bc3ae474b3e27ca5cb0c75e4cff0fee9ca93 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 9 Oct 2015 13:23:17 +0100 Subject: [PATCH 09/50] Fix bad merge in `TestUtils` --- .../test/scala/unit/kafka/utils/TestUtils.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 26a7261a5e328..893b46e0bfa20 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -914,14 +914,19 @@ object TestUtils extends Logging { } def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { - var sslConfigs: java.util.Map[String, Object] = new java.util.HashMap[String, Object]() if (!trustStoreFile.isDefined) { throw new Exception("enableSSL set to true but no trustStoreFile provided") } - if (mode == Mode.SERVER) - sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias) - else - sslConfigs = TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) + + val sslConfigs = { + if (mode == Mode.SERVER) { + val sslConfigs = TestSSLUtils.createSSLConfig(true, true, mode, trustStoreFile.get, certAlias) + sslConfigs.put(KafkaConfig.InterBrokerSecurityProtocolProp, SecurityProtocol.SSL.name) + sslConfigs + } + else + TestSSLUtils.createSSLConfig(clientCert, false, mode, trustStoreFile.get, certAlias) + } val sslProps = new Properties() sslConfigs.foreach { case (k, v) => sslProps.put(k, v) } From 9ed1a2635d97c290e42b723ce8db2bf60c1c6440 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 9 Oct 2015 13:23:46 +0100 Subject: [PATCH 10/50] Remove -XX:-MaxFDLimit from `gradle.properties` --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index c63cdcef1d547..faeaf885efed0 100644 --- a/gradle.properties +++ b/gradle.properties @@ -17,4 +17,4 @@ group=org.apache.kafka version=0.9.0.0-SNAPSHOT scalaVersion=2.10.5 task=build -org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m -XX:-MaxFDLimit +org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m From 2d2fcecb7bda62519d36d4f71a955cf55c8bbd2a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 9 Oct 2015 13:36:06 +0100 Subject: [PATCH 11/50] Support `SSLSASL` in `ChannelBuilders`, reduce duplication in `TestUtils` and clean-up `SaslTestHarness` --- .../kafka/common/network/ChannelBuilders.java | 13 ++++--- .../kafka/api/SaslTestHarness.scala | 35 +++++++++---------- .../scala/unit/kafka/utils/TestUtils.scala | 25 +++++++------ 3 files changed, 40 insertions(+), 33 deletions(-) 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 52d716a218c11..667c6f2e57930 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 @@ -33,13 +33,12 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode 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 SSLSASL: case PLAINTEXTSASL: - if (mode == null) - throw new IllegalArgumentException("`mode` must be non-null if `securityProtocol` is `PLAINTEXTSASL`"); + requireNonNullMode(mode, securityProtocol); channelBuilder = new SaslChannelBuilder(mode, securityProtocol); break; case PLAINTEXT: @@ -53,4 +52,10 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode 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/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 5cd01bbaafeee..27dce9f04eb24 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -12,40 +12,37 @@ */ package kafka.api -import java.io.{FileWriter, BufferedWriter, File} -import java.net.URL +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.junit.{After, Before} -import org.scalatest.junit.JUnitSuite - trait SaslTestHarness extends ZooKeeperTestHarness { - val WorkDir = new File(System.getProperty("test.dir", "target")); - val KdcConf = MiniKdc.createConf(); - val Kdc: MiniKdc = new MiniKdc(KdcConf, WorkDir); + val workDir = new File(System.getProperty("test.dir", "target")) + val kdcConf = MiniKdc.createConf() + val kdc = new MiniKdc(kdcConf, workDir) @Before override def setUp() { - val keytabFile: File = TestUtils.tempFile() - val jaasFile: File = TestUtils.tempFile() - - val writer: BufferedWriter = new BufferedWriter(new FileWriter(jaasFile)) - - val path: String = Thread.currentThread().getContextClassLoader.getResource("kafka_jaas.conf").getPath - for(line <- io.Source.fromFile(path).getLines()) { - val s: String = "\\$keytab-location" - val replaced = line.replaceAll("\\$keytab-location",keytabFile.getAbsolutePath) + 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") + 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") + kdc.start() + kdc.createPrincipal(keytabFile, "client", "kafka/localhost") System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath) super.setUp } @@ -53,7 +50,7 @@ trait SaslTestHarness extends ZooKeeperTestHarness { @After override def tearDown() { super.tearDown - Kdc.stop() + kdc.stop() System.clearProperty("java.security.auth.login.config") Configuration.setConfiguration(null) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 893b46e0bfa20..af9cf109778e4 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} @@ -158,16 +158,22 @@ object TestUtils extends Logging { def createBrokerConfig(nodeId: Int, zkConnect: String, enableControlledShutdown: Boolean = true, enableDeleteTopic: Boolean = false, - port: Int = RandomPort, enableSasl:Boolean = false, saslPort:Int = RandomPort, enableSSL: Boolean = false, + port: Int = RandomPort, enableSasl: Boolean = false, saslPort: Int = RandomPort, enableSSL: Boolean = false, sslPort: Int = RandomPort, trustStoreFile: Option[File] = None): Properties = { + val listeners = { + val protocolAndPorts = ArrayBuffer("PLAINTEXT" -> port) + if (enableSSL) + protocolAndPorts += "SSL" -> sslPort + if (enableSasl) + protocolAndPorts += "PLAINTEXTSASL" -> saslPort + protocolAndPorts.map { case (protocol, port) => + s"$protocol://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 - if (enableSasl) - listeners = listeners + "," + "PLAINTEXTSASL://localhost:" + saslPort.toString props.put("listeners", listeners) props.put("log.dir", TestUtils.tempDir().getAbsolutePath) props.put("zookeeper.connect", zkConnect) @@ -176,9 +182,8 @@ 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(Mode.SERVER, true, trustStoreFile, "server"+nodeId)) - } + if (enableSSL) + props.putAll(addSSLConfigs(Mode.SERVER, true, trustStoreFile, s"server$nodeId")) props.put("port", port.toString) props } From 32ab6f468505edf10be686905019c4d202663f72 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Fri, 9 Oct 2015 15:21:26 -0700 Subject: [PATCH 12/50] KAFKA-1686. Added SaslConsumerTest, fixed a bug in SecurityProtocol. --- .../common/protocol/SecurityProtocol.java | 2 +- .../kafka/api/SaslConsumerTest.scala | 157 ++++++++++++++++++ .../kafka/api/SaslTestHarness.scala | 2 + .../scala/unit/kafka/utils/TestUtils.scala | 7 +- 4 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala 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 cc15472262fb3..5aeed0b6bed9a 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 @@ -29,7 +29,7 @@ public enum SecurityProtocol { /** SASL authenticated, non-encrypted channel */ PLAINTEXTSASL(2, "PLAINTEXTSASL"), /** SASL authenticated, SSL channel */ - SSLSASL(2, "SASL"), + SSLSASL(3, "SSLSASL"), /** 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/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala new file mode 100644 index 0000000000000..3b2a4634b72b9 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala @@ -0,0 +1,157 @@ +/** + * 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 org.apache.kafka.common.protocol.SecurityProtocol +import kafka.integration.KafkaServerTestHarness + +import kafka.utils.{TestUtils, Logging} +import kafka.server.{KafkaConfig, KafkaServer} + +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 + +class SaslConsumerTest extends SaslTestHarness with Logging { + val brokerId1 = 0 + var servers: Buffer[KafkaServer] = null + val numServers = 1 + 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.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]]]() + + 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() + val props = TestUtils.createBrokerConfig(numServers, zkConnect, false, enableSasl=true) + val config = KafkaConfig.fromProps(props, overridingProps) + servers = Buffer(TestUtils.createServer(config)) + + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL)) + 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.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL)) + 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.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), + acks = 1, + enableSasl=true) + for (i <- 0 until consumerCount) + consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), + groupId = "my-test", + partitionAssignmentStrategy= "range", + enableSasl=true) + + + // create the consumer offset topic + TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, + overridingProps.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, + 1, + servers, + servers(0).consumerCoordinator.offsetsTopicConfigs) + + // create the test topic with all the brokers as replicas + TestUtils.createTopic(this.zkClient, 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) + } + + 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/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 27dce9f04eb24..6e0ef4cc2c28a 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -43,7 +43,9 @@ trait SaslTestHarness extends ZooKeeperTestHarness { kdc.start() kdc.createPrincipal(keytabFile, "client", "kafka/localhost") + println("setting java property ", jaasFile.getAbsolutePath); System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath) + println("setting java property ", System.getProperty("java.security.auth.login.config")); super.setUp } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index af9cf109778e4..3e9e71f10134c 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -144,8 +144,10 @@ object TestUtils extends Logging { enableControlledShutdown: Boolean = true, enableDeleteTopic: Boolean = false, enableSSL: Boolean = false, + enableSasl: Boolean = false, trustStoreFile: Option[File] = None): Seq[Properties] = { - (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, enableSSL = enableSSL, trustStoreFile = trustStoreFile)) + (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, + enableSSL = enableSSL, enableSasl = enableSasl, trustStoreFile = trustStoreFile)) } def getBrokerListStrFromServers(servers: Seq[KafkaServer], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): String = { @@ -443,6 +445,7 @@ object TestUtils extends Logging { partitionFetchSize: Long = 4096L, partitionAssignmentStrategy: String = "blah", sessionTimeout: Int = 30000, + enableSasl: Boolean = false, enableSSL: Boolean = false, trustStoreFile: Option[File] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.consumer.ConsumerConfig @@ -461,6 +464,8 @@ object TestUtils extends Logging { if (enableSSL) { consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") consumerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "consumer")) + } else if (enableSasl) { + consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXTSASL") } new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } From 58064b46a7ddbb7d2293e33c7b66c35f76043588 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Fri, 9 Oct 2015 15:33:11 -0700 Subject: [PATCH 13/50] KAFKA-1686. removing unnecessary logs. --- core/src/test/scala/integration/kafka/api/SaslTestHarness.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 6e0ef4cc2c28a..27dce9f04eb24 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -43,9 +43,7 @@ trait SaslTestHarness extends ZooKeeperTestHarness { kdc.start() kdc.createPrincipal(keytabFile, "client", "kafka/localhost") - println("setting java property ", jaasFile.getAbsolutePath); System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath) - println("setting java property ", System.getProperty("java.security.auth.login.config")); super.setUp } From 9e6ba51a54e467177d1c724e1f3ecab264d7a837 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 12 Oct 2015 15:11:37 +0100 Subject: [PATCH 14/50] A number of code clean-ups * Address some of Jun's and Parth's comments * Make fields final * Reduce scope of variables where possible * Remove unused fields and methods * Fix javadoc * Fix formatting and naming issues --- .../kafka/common/network/Authenticator.java | 8 +- .../common/network/DefaultAuthenticator.java | 2 +- .../network/PlaintextTransportLayer.java | 2 +- .../kafka/common/network/TransportLayer.java | 3 +- .../kafka/common/security/JaasUtils.java | 3 - .../security/auth/PrincipalBuilder.java | 19 ++- .../SaslClientAuthenticator.java | 112 +++++++---------- .../SaslServerAuthenticator.java | 52 ++++---- .../SaslServerCallbackHandler.java | 7 +- .../security/kerberos/KerberosName.java | 49 +++----- .../kafka/common/security/kerberos/Login.java | 117 +++++++----------- .../security/kerberos/LoginManager.java | 16 +-- 12 files changed, 154 insertions(+), 236 deletions(-) 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 5a9af8ff52995..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 @@ -31,9 +31,11 @@ public interface Authenticator { /** - * Configures Authenticator using principalBuilder and transportLayer. - * @param transportLayer - * @param 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, Map configs); 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 74958959b3437..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 @@ -55,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/PlaintextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java index 00d8214ba17e7..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 @@ -35,7 +35,7 @@ public class PlaintextTransportLayer implements TransportLayer { private static final Logger log = LoggerFactory.getLogger(PlaintextTransportLayer.class); - public final SelectionKey key; + private final SelectionKey key; private final SocketChannel socketChannel; private final Principal principal = KafkaPrincipal.ANONYMOUS; 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/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java index a95d09bb1e486..8e92e59351a80 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 @@ -26,14 +26,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - public class JaasUtils { private static final Logger LOG = LoggerFactory.getLogger(JaasUtils.class); public static final String LOGIN_CONTEXT_SERVER = "KafkaServer"; public static final String LOGIN_CONTEXT_CLIENT = "KafkaClient"; public static final String SERVICE_NAME = "serviceName"; - // Oid mechanism = use Kerberos V5 as the security mechanism. - //public static final Oid KRB5_MECH_OID = new Oid("1.2.840.113554.1.2.2"); /** * Construct a JAAS configuration object per kafka jaas configuration 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 index fd728a45936a8..9f14bdd5e33fa 100644 --- 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 @@ -43,39 +43,35 @@ 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; import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.security.auth.PrincipalBuilder; import org.apache.kafka.common.security.kerberos.KerberosName; -import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.KafkaException; 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; + + // assigned in `configure` private SaslClient saslClient; - private Subject subject; - private String servicePrincipal; - private PrincipalBuilder principalBuilder; - private String host; - private String node; - private String serverRealm; - private Object[] principals; - private Principal clientPrincipal; - private KerberosName clientKerberosName; - private KerberosName serviceKerberosName; private String clientPrincipalName; private TransportLayer transportLayer; + + // buffers used in `authenticate` private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; - private byte[] saslToken = new byte[0]; - - public enum SaslState { - INITIAL, INTERMEDIATE, COMPLETE, FAILED - } private SaslState saslState = SaslState.INITIAL; @@ -89,20 +85,10 @@ public SaslClientAuthenticator(String node, Subject subject, String servicePrinc public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) throws KafkaException { try { this.transportLayer = transportLayer; - this.principalBuilder = principalBuilder; - this.principals = subject.getPrincipals().toArray(); // determine client principal from subject. - this.clientPrincipal = (Principal) principals[0]; - this.clientKerberosName = new KerberosName(clientPrincipal.getName()); - // assume that server and client are in the same realm (by default; unless the config - // "kafka.server.realm" is set). - if (configs.containsKey(SaslConfigs.SASL_KAFKA_SERVER_REALM)) - this.serverRealm = (String) configs.get(SaslConfigs.SASL_KAFKA_SERVER_REALM); - else - this.serverRealm = clientKerberosName.realm(); - this.serviceKerberosName = new KerberosName(servicePrincipal + "@" + serverRealm); - this.clientPrincipalName = clientKerberosName.toString(); + Principal clientPrincipal = subject.getPrincipals().iterator().next(); + this.clientPrincipalName = new KerberosName(clientPrincipal.getName()).toString(); this.saslClient = createSaslClient(); } catch (Exception e) { throw new KafkaException("Failed to configure SaslClientAuthenticator", e); @@ -111,16 +97,15 @@ public void configure(TransportLayer transportLayer, PrincipalBuilder principalB private SaslClient createSaslClient() { try { - saslClient = Subject.doAs(subject, new PrivilegedExceptionAction() { - public SaslClient run() throws SaslException { - LOG.debug("Client will use GSSAPI as SASL mechanism."); - String[] mechs = {"GSSAPI"}; - LOG.debug("creating sasl client: client=" + clientPrincipalName + ";service=" + servicePrincipal + ";serviceHostname=" + host); - SaslClient saslClient = Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null, new ClientCallbackHandler(null)); - return saslClient; - } - }); - return saslClient; + return Subject.doAs(subject, new PrivilegedExceptionAction() { + public SaslClient run() throws SaslException { + LOG.debug("Client will use GSSAPI as SASL mechanism."); + String[] mechs = {"GSSAPI"}; + LOG.debug("creating sasl client: client=" + clientPrincipalName + ";service=" + servicePrincipal + ";serviceHostname=" + host); + SaslClient saslClient = Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null, new ClientCallbackHandler()); + return saslClient; + } + }); } catch (Exception e) { LOG.error("Exception while trying to create SASL client", e); throw new KafkaException("Failed to create SASL client", e); @@ -133,18 +118,17 @@ public void authenticate() throws IOException { transportLayer.addInterestOps(SelectionKey.OP_WRITE); return; } - byte[] serverToken = new byte[0]; switch (saslState) { case INITIAL: - sendSaslToken(serverToken); + sendSaslToken(new byte[0]); saslState = SaslState.INTERMEDIATE; break; case INTERMEDIATE: if (netInBuffer == null) netInBuffer = new NetworkReceive(node); - long readLen = netInBuffer.readFrom(transportLayer); + netInBuffer.readFrom(transportLayer); if (netInBuffer.complete()) { netInBuffer.payload().rewind(); - serverToken = new byte[netInBuffer.payload().remaining()]; + 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); @@ -162,7 +146,7 @@ public void authenticate() throws IOException { private void sendSaslToken(byte[] serverToken) throws IOException { if (!saslClient.isComplete()) { try { - saslToken = createSaslToken(serverToken); + byte[] saslToken = createSaslToken(serverToken); if (saslToken != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken)); if (!flushNetOutBuffer()) @@ -193,13 +177,11 @@ private byte[] createSaslToken(final byte[] saslToken) throws SaslException { } try { - final byte[] retval = - Subject.doAs(subject, new PrivilegedExceptionAction() { - public byte[] run() throws SaslException { - return saslClient.evaluateChallenge(saslToken); - } - }); - return retval; + 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."; @@ -211,7 +193,8 @@ public byte[] run() throws SaslException { 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."; + " 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."; LOG.error(error); @@ -227,11 +210,6 @@ private boolean flushNetOutBuffer() throws IOException { } public static class ClientCallbackHandler implements CallbackHandler { - private String password = null; - - public ClientCallbackHandler(String password) { - this.password = password; - } public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { @@ -239,17 +217,13 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { NameCallback nc = (NameCallback) callback; nc.setName(nc.getDefaultName()); } else if (callback instanceof PasswordCallback) { - PasswordCallback pc = (PasswordCallback) callback; - if (password != null) { - pc.setPassword(this.password.toCharArray()); - } else { - 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. "); - } + // 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()); 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 index b7c98a18dfa4c..3792cd3eeaa71 100644 --- 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 @@ -49,29 +49,29 @@ 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 SaslServer saslServer; - private Subject subject; + + private final SaslServer saslServer; + private final Subject subject; + private final String node; + + // assigned in `configure` private TransportLayer transportLayer; - private String node = "0"; + + // buffers used in `authenticate` private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; - private SaslServerCallbackHandler saslServerCallbackHandler; - private PrincipalBuilder principalBuilder; public SaslServerAuthenticator(String node, final Subject subject) throws IOException { this.node = node; - this.transportLayer = transportLayer; this.subject = subject; saslServer = createSaslServer(); } public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) { this.transportLayer = transportLayer; - this.principalBuilder = principalBuilder; } private SaslServer createSaslServer() throws IOException { @@ -79,9 +79,8 @@ private SaslServer createSaslServer() throws IOException { // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject. if (subject.getPrincipals().size() > 0) { try { - saslServerCallbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration()); - final Object[] principals = subject.getPrincipals().toArray(); - final Principal servicePrincipal = (Principal) principals[0]; + final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration()); + final Principal servicePrincipal = subject.getPrincipals().iterator().next(); final String servicePrincipalNameAndHostname = servicePrincipal.getName(); int indexOf = servicePrincipalNameAndHostname.indexOf("/"); @@ -93,7 +92,7 @@ private SaslServer createSaslServer() throws IOException { LOG.debug("serviceHostname is '" + serviceHostname + "'"); LOG.debug("servicePrincipalName is '" + servicePrincipalName + "'"); - LOG.debug("SASL mechanism(mech) is '" + mech + "'"); + LOG.debug("SASL mechanism is '" + mech + "'"); boolean usingNativeJgss = Boolean.getBoolean("sun.security.jgss.native"); if (usingNativeJgss) { try { @@ -109,17 +108,15 @@ private SaslServer createSaslServer() throws IOException { try { return Subject.doAs(subject, new PrivilegedExceptionAction() { - public SaslServer run() { - try { - SaslServer saslServer; - saslServer = Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler); - return saslServer; - } catch (SaslException e) { - LOG.error("Kafka Server failed to create a SaslServer to interact with a client during session authentication: " + e); - return null; - } + public SaslServer run() { + try { + return Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler); + } catch (SaslException e) { + LOG.error("Kafka Server failed to create a SaslServer to interact with a client during session authentication: " + e); + return null; } - }); + } + }); } catch (PrivilegedActionException e) { LOG.error("KafkaBroker experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context:" + e); } @@ -142,20 +139,17 @@ public void authenticate() throws IOException { return; } - byte[] clientToken = new byte[0]; - if (netInBuffer == null) netInBuffer = new NetworkReceive(node); - long readLen = netInBuffer.readFrom(transportLayer); + netInBuffer.readFrom(transportLayer); if (netInBuffer.complete()) { netInBuffer.payload().rewind(); - clientToken = new byte[netInBuffer.payload().remaining()]; + 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; - response = saslServer.evaluateResponse(clientToken); + byte[] response = saslServer.evaluateResponse(clientToken); if (response != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); if (!flushNetOutBuffer()) { @@ -169,8 +163,6 @@ public void authenticate() throws IOException { } } - - public Principal principal() { return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, saslServer.getAuthorizationID()); } 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 index e9a6c1f94e374..fa7612dbff2d8 100644 --- 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 @@ -33,7 +33,6 @@ 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); @@ -72,9 +71,9 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) { KerberosName kerberosName = new KerberosName(authenticationID); try { - StringBuilder userNameBuilder = new StringBuilder(kerberosName.shortName()); - LOG.info("Setting authorizedID: " + userNameBuilder); - ac.setAuthorizedID(userNameBuilder.toString()); + 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 index 52445d2f3eee6..ab85f2ceb94ef 100644 --- 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 @@ -26,13 +26,11 @@ import org.apache.kafka.common.security.JaasUtils; - /** * 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 KerberosName { /** The first component of the name */ private final String serviceName; @@ -42,43 +40,46 @@ public class KerberosName { private final String realm; /** - * A pattern that matches a Kerberos name with at most 2 components. + * A pattern that matches a Kerberos name with at most 3 components. */ - private static final Pattern NAMEPARSER = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)"); + private static final Pattern NAME_PARSER = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)"); /** - * A pattern that matches a string with out '$' and then a single + * A pattern that matches a string without '$' and then a single * parameter with $n. */ - private static final Pattern PARAMETERPATTERN = Pattern.compile("([^$]*)(\\$(\\d*))?"); + private static final Pattern PARAMETER_PATTERN = Pattern.compile("([^$]*)(\\$(\\d*))?"); /** * A pattern for parsing a auth_to_local rule. */ - private static final Pattern RULEPARSER = Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?" + + private static final Pattern RULE_PARSER = Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?" + "(s/([^/]*)/([^/]*)/(g)?)?))"); /** * A pattern that recognizes simple/non-simple names. */ - private static final Pattern NONSIMPLEPATTERN = Pattern.compile("[/@]"); + private static final Pattern NON_SIMPLE_PATTERN = Pattern.compile("[/@]"); /** * The list of translation rules. */ - private static List rules; + private static final List RULES; - private static String defaultRealm; + private static final String DEFAULT_REALM; static { + String defaultRealm; try { defaultRealm = JaasUtils.defaultRealm(); } catch (Exception ke) { defaultRealm = ""; } + DEFAULT_REALM = defaultRealm; try { - setConfiguration(); - } catch (IOException e) { + String ruleString = System.getProperty("kafka.security.auth_to_local", "DEFAULT"); + RULES = parseRules(ruleString); + } catch (Exception e) { throw new IllegalArgumentException("Could not configure Kerberos principal name mapping."); } } @@ -88,7 +89,7 @@ public class KerberosName { * @param name */ public KerberosName(String name) { - Matcher match = NAMEPARSER.matcher(name); + Matcher match = NAME_PARSER.matcher(name); if (!match.matches()) { if (name.contains("@")) { throw new IllegalArgumentException("Malformed Kerberos name: " + name); @@ -109,7 +110,7 @@ public KerberosName(String name) { * @return the default realm from the krb5.conf */ public String getDefaultRealm() { - return defaultRealm; + return DEFAULT_REALM; } /** @@ -229,7 +230,7 @@ public String toString() { */ static String replaceParameters(String format, String[] params) throws BadFormatString { - Matcher match = PARAMETERPATTERN.matcher(format); + Matcher match = PARAMETER_PATTERN.matcher(format); int start = 0; StringBuilder result = new StringBuilder(); while (start < format.length() && match.find(start)) { @@ -285,7 +286,7 @@ static String replaceSubstitution(String base, Pattern from, String to, String apply(String[] params) throws IOException { String result = null; if (isDefault) { - if (defaultRealm.equals(params[0])) { + if (DEFAULT_REALM.equals(params[0])) { result = params[1]; } } else if (params.length - 1 == numOfComponents) { @@ -298,7 +299,7 @@ String apply(String[] params) throws IOException { } } } - if (result != null && NONSIMPLEPATTERN.matcher(result).find()) { + if (result != null && NON_SIMPLE_PATTERN.matcher(result).find()) { throw new NoMatchingRule("Non-simple name " + result + " after auth_to_local rule " + this); } @@ -310,7 +311,7 @@ static List parseRules(String rules) { List result = new ArrayList(); String remaining = rules.trim(); while (remaining.length() > 0) { - Matcher matcher = RULEPARSER.matcher(remaining); + Matcher matcher = RULE_PARSER.matcher(remaining); if (!matcher.lookingAt()) { throw new IllegalArgumentException("Invalid rule: " + remaining); } @@ -329,16 +330,6 @@ static List parseRules(String rules) { return result; } - /** - * Set the static configuration to get the rules. - * @param conf the new configuration - * @throws IOException - */ - public static void setConfiguration() throws IOException { - String ruleString = System.getProperty("kafka.security.auth_to_local", "DEFAULT"); - rules = parseRules(ruleString); - } - @SuppressWarnings("serial") public static class BadFormatString extends IOException { BadFormatString(String msg) { @@ -372,7 +363,7 @@ public String shortName() throws IOException { } else { params = new String[]{realm, serviceName, hostName}; } - for (Rule r: rules) { + for (Rule r : RULES) { String result = r.apply(params); if (result != null) 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 index a75ded443842f..c722aace5ff6e 100644 --- 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 @@ -18,11 +18,6 @@ package org.apache.kafka.common.security.kerberos; -/** - * This class is responsible for refreshing Kerberos credentials for - * logins for both Kafka client and server. - */ - import javax.security.auth.kerberos.KerberosPrincipal; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; @@ -38,6 +33,7 @@ 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; @@ -48,40 +44,44 @@ import java.util.Random; import java.util.Set; +/** + * 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); // 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 static final float TICKET_RENEW_WINDOW = 0.80f; + private static final float TICKET_RENEW_WINDOW_FACTOR = 0.80f; /** * Percentage of random jitter added to the renewal time */ private static final float TICKET_RENEW_JITTER = 0.05f; - // Regardless of TICKET_RENEW_WINDOW setting above and the ticket expiry time, + // Regardless of TICKET_RENEW_WINDOW_FACTOR 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 static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L; + private static final Random RNG = new Random(); + private volatile Subject subject = null; - private Thread t = null; - private boolean isKrbTicket = false; - private boolean isUsingTicketCache = false; - /** Random number generator */ - private static Random rng = new Random(); + private final Thread t; + private final boolean isKrbTicket; + private final boolean isUsingTicketCache; - private LoginContext login = null; - private String loginContextName = null; - private String principal = null; - private Configuration jaasConfig = null; - private Time time = new SystemTime(); + private final String loginContextName; + private final String principal; + private final Time time = new SystemTime(); + private final CallbackHandler callbackHandler = new ClientCallbackHandler(); + + private LoginContext login; // Initialize 'lastLogin' to do a login at first time private long lastLogin = time.currentElapsedTime() - MIN_TIME_BEFORE_RELOGIN; - public CallbackHandler callbackHandler = new ClientCallbackHandler(null); /** * Login constructor. The constructor starts the thread used @@ -97,21 +97,28 @@ public Login(final String loginContextName) throws LoginException { login = login(loginContextName); subject = login.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); + AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); - for (AppConfigurationEntry entry: entries) { + if (entries.length == 0) { + isUsingTicketCache = false; + principal = null; + } else { + AppConfigurationEntry entry = entries[0]; // there will only be a single entry, so this for() loop will only be iterated through once. if (entry.getOptions().get("useTicketCache") != null) { String val = (String) entry.getOptions().get("useTicketCache"); - if (val.equals("true")) - isUsingTicketCache = true; - } + isUsingTicketCache = val.equals("true"); + } else + isUsingTicketCache = false; if (entry.getOptions().get("principal") != null) principal = (String) entry.getOptions().get("principal"); - break; + else + principal = null; } log.debug("checking if its isKrbTicket"); if (!isKrbTicket) { + t = null; // if no TGT, do not bother with ticket management. return; } @@ -120,7 +127,7 @@ public Login(final String loginContextName) throws LoginException { // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. 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 = new Thread(new Runnable() { + 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. @@ -136,10 +143,10 @@ public void run() { nextRefresh = getRefreshTime(tgt); long expiry = tgt.getEndTime().getTime(); Date expiryDate = new Date(expiry); - if (isUsingTicketCache && (tgt.getEndTime().equals(tgt.getRenewTill()))) { + if (isUsingTicketCache && tgt.getRenewTill() != null && tgt.getRenewTill().getTime() >= expiry) { log.error("The TGT cannot be renewed beyond the next expiry date: " + expiryDate + "." + "This process will not be able to authenticate new SASL connections after that " + - "time (for example, it will not be authenticate a new connection with a Kafka " + + "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 " + principal + "' within " + "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " + @@ -159,8 +166,8 @@ public void run() { if (nextRefresh < (now + MIN_TIME_BEFORE_RELOGIN)) { // next scheduled refresh is sooner than (now + MIN_TIME_BEFORE_LOGIN). Date until = new Date(nextRefresh); - Date newuntil = new Date(now + MIN_TIME_BEFORE_RELOGIN); - log.warn("TGT refresh thread time adjusted from : " + until + " to : " + newuntil + " since " + Date newUntil = new Date(now + MIN_TIME_BEFORE_RELOGIN); + log.warn("TGT refresh thread time adjusted from : " + until + " to : " + newUntil + " since " + "the former is sooner than the minimum refresh interval (" + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now."); } @@ -181,7 +188,7 @@ public void run() { Thread.sleep(nextRefresh - now); } catch (InterruptedException ie) { log.warn("TGT renewal thread has been interrupted and will exit."); - break; + return; } } else { log.error("nextRefresh:" + nextRefreshDate + " is in the past: exiting refresh thread. Check" @@ -243,12 +250,11 @@ public void run() { } } catch (LoginException le) { log.error("Failed to refresh TGT: refresh thread exiting now.", le); - break; + return; } } } - }); - t.setDaemon(true); + }, false); } public void startThreadIfNeeded() { @@ -291,9 +297,8 @@ private synchronized LoginContext login(final String loginContextName) throws Lo } File configFile = new File(System.getProperty("java.security.auth.login.config")); - Configuration loginConf = null; try { - loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configFile.toURI())); + Configuration loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configFile.toURI())); Configuration.setConfiguration(loginConf); } catch (NoSuchAlgorithmException e) { throw new RuntimeException(e); @@ -311,7 +316,7 @@ private long getRefreshTime(KerberosTicket tgt) { log.info("TGT valid starting at: " + tgt.getStartTime().toString()); log.info("TGT expires: " + tgt.getEndTime().toString()); long proposedRefresh = start + (long) ((expires - start) * - (TICKET_RENEW_WINDOW + (TICKET_RENEW_JITTER * rng.nextDouble()))); + (TICKET_RENEW_WINDOW_FACTOR + (TICKET_RENEW_JITTER * RNG.nextDouble()))); if (proposedRefresh > expires) // proposedRefresh is too far in the future: it's after ticket expires: simply return now. @@ -322,7 +327,7 @@ private long getRefreshTime(KerberosTicket tgt) { private synchronized KerberosTicket getTGT() { Set tickets = subject.getPrivateCredentials(KerberosTicket.class); - for (KerberosTicket ticket: tickets) { + for (KerberosTicket ticket : tickets) { KerberosPrincipal server = ticket.getServer(); if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { log.debug("Found tgt " + ticket + "."); @@ -334,49 +339,17 @@ private synchronized KerberosTicket getTGT() { private boolean hasSufficientTimeElapsed() { long now = time.currentElapsedTime(); - if (now - getLastLogin() < MIN_TIME_BEFORE_RELOGIN) { + if (now - lastLogin < MIN_TIME_BEFORE_RELOGIN) { log.warn("Not attempting to re-login since the last re-login was " + "attempted less than " + (MIN_TIME_BEFORE_RELOGIN / 1000) + " seconds" + " before."); return false; } // register most recent relogin attempt - setLastLogin(now); + lastLogin = now; return true; } - /** - * Returns login object - * @return login - */ - private LoginContext getLogin() { - return login; - } - - /** - * Set the login object - * @param login - */ - private void setLogin(LoginContext login) { - this.login = login; - } - - /** - * Set the last login time. - * @param time the number of milliseconds since the beginning of time - */ - private void setLastLogin(long time) { - lastLogin = time; - } - - /** - * Get the time of the last login. - * @return the number of milliseconds since the beginning of time. - */ - private long getLastLogin() { - return lastLogin; - } - /** * Re-login a principal. This method assumes that {@link #login(String)} has happened already. * @throws javax.security.auth.login.LoginException on a failure @@ -386,8 +359,7 @@ private synchronized void reLogin() if (!isKrbTicket) { return; } - LoginContext login = getLogin(); - if (login == null) { + if (login == null) { throw new LoginException("login must be done first"); } if (!hasSufficientTimeElapsed()) { @@ -404,7 +376,6 @@ private synchronized void reLogin() login = new LoginContext(loginContextName, subject()); log.info("Initiating re-login for " + principal); login.login(); - setLogin(login); } } } 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 index a62b390202dd8..2f44529e858b4 100644 --- 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 @@ -28,20 +28,17 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Configurable; - public class LoginManager implements Configurable { - private Login login; + private final Login login; private final String serviceName; - private final String loginContext; - private final Mode mode; - volatile private static LoginManager instance; + private volatile static LoginManager instance; private LoginManager(Mode mode) throws IOException, LoginException { - this.mode = mode; + String loginContext; if (mode == Mode.SERVER) - this.loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; + loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; else - this.loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; + loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); login = new Login(loginContext); login.startThreadIfNeeded(); @@ -60,8 +57,7 @@ public static final LoginManager getLoginManager(Mode mode) throws IOException, } @Override - public void configure(Map configs) throws KafkaException { - } + public void configure(Map configs) throws KafkaException {} public Subject subject() { return login.subject(); From fc40c986ddd8f9653ca96c8f683074f0114f8590 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 12 Oct 2015 15:12:38 +0100 Subject: [PATCH 15/50] Return non-anonymous `KafkaPrincipal` in `SaslClientAuthenticator.principal` --- .../common/security/authenticator/SaslClientAuthenticator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index 9f14bdd5e33fa..3453765b26188 100644 --- 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 @@ -160,7 +160,7 @@ private void sendSaslToken(byte[] serverToken) throws IOException { } public Principal principal() { - return KafkaPrincipal.ANONYMOUS; + return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, clientPrincipalName); } public boolean complete() { From 1d53bcea1c2e95585ec474cd52f4cf87104ef940 Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Mon, 12 Oct 2015 22:11:08 -0700 Subject: [PATCH 16/50] KAFKA-1686. Added default sasl configs , addressed reviews. --- .../clients/consumer/ConsumerConfig.java | 3 + .../clients/producer/ProducerConfig.java | 3 + .../kafka/common/config/SaslConfigs.java | 15 ++++ .../common/network/SaslChannelBuilder.java | 3 +- .../kafka/common/security/JaasUtils.java | 1 + .../SaslClientAuthenticator.java | 1 + .../SaslServerAuthenticator.java | 2 +- .../kafka/common/security/kerberos/Login.java | 70 +++++++++---------- .../security/kerberos/LoginManager.java | 15 ++-- .../main/scala/kafka/server/KafkaConfig.scala | 35 ++++++++++ .../unit/kafka/server/KafkaConfigTest.scala | 6 ++ 11 files changed, 106 insertions(+), 48 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 8b78f86032e22..1d50e9a339fe7 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 @@ -305,6 +305,9 @@ public class ConsumerConfig extends AbstractConfig { .define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_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(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 37a3667d5b96e..ce19f88df31cf 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 @@ -282,6 +282,9 @@ public class ProducerConfig extends AbstractConfig { .define(SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, Type.STRING, Importance.LOW, SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_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) /* 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 index 1fce0feb47cfc..82fd2a5406189 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -28,4 +28,19 @@ public class SaslConfigs { + "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; + + + } 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 index ad84b8eec0d8b..5453eed7d3511 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -46,8 +46,7 @@ public SaslChannelBuilder(Mode mode, SecurityProtocol securityProtocol) { public void configure(Map configs) throws KafkaException { try { this.configs = configs; - this.loginManager = LoginManager.getLoginManager(mode); - this.loginManager.configure(this.configs); + this.loginManager = LoginManager.getLoginManager(mode, configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); if (this.securityProtocol == SecurityProtocol.SSLSASL) { 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 8e92e59351a80..817049a8fbdf5 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 @@ -31,6 +31,7 @@ public class JaasUtils { 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"; /** * Construct a JAAS configuration object per kafka jaas configuration file 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 index 3453765b26188..ffe3424bf2e5d 100644 --- 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 @@ -137,6 +137,7 @@ public void authenticate() throws IOException { saslState = SaslState.COMPLETE; break; case COMPLETE: + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); break; case FAILED: throw new IOException("SASL handshake failed"); 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 index 3792cd3eeaa71..3fa9bc0b9cc09 100644 --- 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 @@ -135,7 +135,7 @@ public void authenticate() throws IOException { } if (saslServer.isComplete()) { - transportLayer.addInterestOps(SelectionKey.OP_READ); + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); return; } 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 index c722aace5ff6e..0cbef10e1675c 100644 --- 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 @@ -29,6 +29,7 @@ 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; @@ -43,6 +44,7 @@ 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 @@ -54,17 +56,17 @@ public class Login { // 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 static final float TICKET_RENEW_WINDOW_FACTOR = 0.80f; + private Double ticketRenewWindowFactor; /** * Percentage of random jitter added to the renewal time */ - private static final float TICKET_RENEW_JITTER = 0.05f; + private Double ticketRenewJitter; - // Regardless of TICKET_RENEW_WINDOW_FACTOR setting above and the ticket expiry time, + // 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 static final long MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L; + private Long minTimeBeforeRelogin; private static final Random RNG = new Random(); @@ -78,10 +80,12 @@ public class Login { private final String principal; private final Time time = new SystemTime(); private final CallbackHandler callbackHandler = new ClientCallbackHandler(); + private Map configs; private LoginContext login; // Initialize 'lastLogin' to do a login at first time - private long lastLogin = time.currentElapsedTime() - MIN_TIME_BEFORE_RELOGIN; + private long lastLogin; + private String kinitCmd; /** * Login constructor. The constructor starts the thread used @@ -89,11 +93,19 @@ public class Login { * @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) throws LoginException { + public Login(final String loginContextName, Map configs) throws LoginException { + this.configs = configs; 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 = time.currentElapsedTime() - this.minTimeBeforeRelogin; + login = login(loginContextName); subject = login.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); @@ -124,7 +136,7 @@ public Login(final String loginContextName) throws LoginException { } log.debug("its a krb5ticket"); // Refresh the Ticket Granting Ticket (TGT) periodically. How often to refresh is determined by the - // TGT's existing expiry date and the configured MIN_TIME_BEFORE_RELOGIN. For testing and development, + // 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() { @@ -136,7 +148,7 @@ public void run() { long nextRefresh; Date nextRefreshDate; if (tgt == null) { - nextRefresh = now + MIN_TIME_BEFORE_RELOGIN; + nextRefresh = now + minTimeBeforeRelogin; nextRefreshDate = new Date(nextRefresh); log.warn("No TGT found: will try again at " + nextRefreshDate); } else { @@ -155,23 +167,23 @@ public void run() { } // 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 - // MIN_TIME_BEFORE_RELOGIN. Will not sleep less than MIN_TIME_BEFORE_RELOGIN, unless doing so + // minTimeBeforeRelogin. Will not sleep less than minTimeBeforeRelogin, unless doing so // would cause ticket expiration. if ((nextRefresh > expiry) || - ((now + MIN_TIME_BEFORE_RELOGIN) > 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 + MIN_TIME_BEFORE_RELOGIN)) { + 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 + MIN_TIME_BEFORE_RELOGIN); + Date newUntil = new Date(now + minTimeBeforeRelogin); log.warn("TGT refresh thread time adjusted from : " + until + " to : " + newUntil + " since " + "the former is sooner than the minimum refresh interval (" - + MIN_TIME_BEFORE_RELOGIN / 1000 + " seconds) from now."); + + minTimeBeforeRelogin / 1000 + " seconds) from now."); } - nextRefresh = Math.max(nextRefresh, now + MIN_TIME_BEFORE_RELOGIN); + nextRefresh = Math.max(nextRefresh, now + minTimeBeforeRelogin); } nextRefreshDate = new Date(nextRefresh); if (nextRefresh > expiry) { @@ -198,16 +210,12 @@ public void run() { return; } if (isUsingTicketCache) { - String cmd = "/usr/bin/kinit"; - if (System.getProperty("kafka.kinit") != null) { - cmd = System.getProperty("kafka.kinit"); - } String kinitArgs = "-R"; int retry = 1; while (retry >= 0) { try { - log.debug("running ticket cache refresh command: " + cmd + " " + kinitArgs); - Shell.execCommand(cmd, kinitArgs); + log.debug("running ticket cache refresh command: " + kinitCmd + " " + kinitArgs); + Shell.execCommand(kinitCmd, kinitArgs); break; } catch (Exception e) { if (retry > 0) { @@ -220,7 +228,7 @@ public void run() { return; } } else { - log.warn("Could not renew TGT due to problem running shell command: '" + cmd + log.warn("Could not renew TGT due to problem running shell command: '" + kinitCmd + " " + kinitArgs + "'" + "; exception was:" + e + ". Exiting refresh thread.", e); return; } @@ -284,19 +292,11 @@ public String loginContextName() { } private synchronized LoginContext login(final String loginContextName) throws LoginException { - if (loginContextName == null) { - throw new LoginException("loginContext name (JAAS file section header) was null. " + - "Please check your java.security.login.auth.config (=" + - System.getProperty("java.security.login.auth.config") + - ") and your " + JaasUtils.LOGIN_CONTEXT_SERVER + "(=" + - System.getProperty(JaasUtils.LOGIN_CONTEXT_CLIENT, "Client") + ")"); - } - - if (System.getProperty("java.security.auth.login.config") == null) { - throw new IllegalArgumentException("You must pass java.security.auth.login.config in secure mode."); + if (System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) == null) { + throw new IllegalArgumentException("You must pass " + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + " in secure mode."); } - File configFile = new File(System.getProperty("java.security.auth.login.config")); + File configFile = new File(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)); try { Configuration loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configFile.toURI())); Configuration.setConfiguration(loginConf); @@ -316,7 +316,7 @@ private long getRefreshTime(KerberosTicket tgt) { log.info("TGT valid starting at: " + tgt.getStartTime().toString()); log.info("TGT expires: " + tgt.getEndTime().toString()); long proposedRefresh = start + (long) ((expires - start) * - (TICKET_RENEW_WINDOW_FACTOR + (TICKET_RENEW_JITTER * RNG.nextDouble()))); + (ticketRenewWindowFactor + (ticketRenewJitter * RNG.nextDouble()))); if (proposedRefresh > expires) // proposedRefresh is too far in the future: it's after ticket expires: simply return now. @@ -339,9 +339,9 @@ private synchronized KerberosTicket getTGT() { private boolean hasSufficientTimeElapsed() { long now = time.currentElapsedTime(); - if (now - lastLogin < MIN_TIME_BEFORE_RELOGIN) { + if (now - lastLogin < minTimeBeforeRelogin) { log.warn("Not attempting to re-login since the last re-login was " + - "attempted less than " + (MIN_TIME_BEFORE_RELOGIN / 1000) + " seconds" + + "attempted less than " + (minTimeBeforeRelogin / 1000) + " seconds" + " before."); return false; } 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 index 2f44529e858b4..7d1f33e49401c 100644 --- 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 @@ -25,40 +25,35 @@ import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.security.JaasUtils; -import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.Configurable; -public class LoginManager implements Configurable { +public class LoginManager { private final Login login; private final String serviceName; private volatile static LoginManager instance; - private LoginManager(Mode mode) throws IOException, LoginException { + private LoginManager(Mode mode, Map configs) throws IOException, LoginException { String loginContext; if (mode == Mode.SERVER) loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; else loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); - login = new Login(loginContext); + login = new Login(loginContext, configs); login.startThreadIfNeeded(); } - public static final LoginManager getLoginManager(Mode mode) throws IOException, LoginException { + public static final LoginManager getLoginManager(Mode mode, Map configs) throws IOException, LoginException { if (instance != null) { return instance; } else { synchronized (LoginManager.class) { if (instance == null) - instance = new LoginManager(mode); + instance = new LoginManager(mode, configs); } } return instance; } - @Override - public void configure(Map configs) throws KafkaException {} - public Subject subject() { return login.subject(); } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 46f4a25073855..700b5b815dbf8 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -27,6 +27,7 @@ import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, Message import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.config.SSLConfigs +import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.config.ConfigDef.Importance._ import org.apache.kafka.common.config.ConfigDef.Range._ import org.apache.kafka.common.config.ConfigDef.Type._ @@ -177,6 +178,12 @@ object Defaults { val SSLClientAuthNone = "none" val SSLClientAuth = SSLClientAuthNone + /** ********* 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 + } object KafkaConfig { @@ -325,6 +332,11 @@ object KafkaConfig { val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_CONFIG + /** ********* SSL Configuration ****************/ + 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 /* Documentation */ /** ********* Zookeeper Configuration ***********/ @@ -496,6 +508,14 @@ object KafkaConfig { val SSLEndpointIdentificationAlgorithmDoc = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_DOC val SSLClientAuthDoc = SSLConfigs.SSL_CLIENT_AUTH_DOC + /** ********* Sasl Configuration ****************/ + 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 + + + private val configDef = { import ConfigDef.Range._ import ConfigDef.ValidString._ @@ -651,6 +671,11 @@ object KafkaConfig { .define(SSLTrustManagerAlgorithmProp, STRING, Defaults.SSLTrustManagerAlgorithm, MEDIUM, SSLTrustManagerAlgorithmDoc) .define(SSLClientAuthProp, STRING, Defaults.SSLClientAuth, in(Defaults.SSLClientAuthRequired, Defaults.SSLClientAuthRequested, Defaults.SSLClientAuthNone), MEDIUM, SSLClientAuthDoc) + /** ********* Sasl Configuration ****************/ + .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) } def configNames() = { @@ -810,6 +835,12 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka val sslTrustManagerAlgorithm = getString(KafkaConfig.SSLTrustManagerAlgorithmProp) val sslClientAuth = getString(KafkaConfig.SSLClientAuthProp) + /** ********* Sasl Configuration **************/ + val saslKerberosKinitCmd = getString(KafkaConfig.SaslKerberosKinitCmdProp) + val saslKerberosTicketRenewWindowFactor = getDouble(KafkaConfig.SaslKerberosTicketRenewWindowFactorProp) + val saslKerberosTicketRenewJitter = getDouble(KafkaConfig.SaslKerberosTicketRenewJitterProp) + val saslKerberosMinTimeBeforeRelogin = getLong(KafkaConfig.SaslKerberosMinTimeBeforeReloginProp) + /** ********* Quota Configuration **************/ val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) val consumerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ConsumerQuotaBytesPerSecondDefaultProp) @@ -948,6 +979,10 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka channelConfigs.put(SSLKeyManagerAlgorithmProp, sslKeyManagerAlgorithm) channelConfigs.put(SSLTrustManagerAlgorithmProp, sslTrustManagerAlgorithm) channelConfigs.put(SSLClientAuthProp, sslClientAuth) + channelConfigs.put(SaslKerberosKinitCmdProp, saslKerberosKinitCmd) + channelConfigs.put(SaslKerberosTicketRenewWindowFactorProp, saslKerberosTicketRenewWindowFactor) + channelConfigs.put(SaslKerberosTicketRenewJitterProp, saslKerberosTicketRenewJitter) + channelConfigs.put(SaslKerberosMinTimeBeforeReloginProp, saslKerberosMinTimeBeforeRelogin) channelConfigs } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index bfec4266c0cec..1b90ff83d7585 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -506,6 +506,12 @@ class KafkaConfigTest { case KafkaConfig.SSLTrustManagerAlgorithmProp => case KafkaConfig.SSLClientAuthProp => // ignore string + //Sasl Configs + case KafkaConfig.SaslKerberosKinitCmdProp => + case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp => + case KafkaConfig.SaslKerberosTicketRenewJitterProp => + case KafkaConfig.SaslKerberosMinTimeBeforeReloginProp => + case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") } }) From e637120467c530ecf559eb2e6278e55fab655551 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 16:13:35 +0100 Subject: [PATCH 17/50] Simplify `Shell` by removing unused functionality and other clean-ups. --- .../org/apache/kafka/common/utils/Shell.java | 113 +++++++----------- 1 file changed, 41 insertions(+), 72 deletions(-) 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 index aaa64e1493573..f5db5c333be29 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/Shell.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Shell.java @@ -38,38 +38,25 @@ abstract public class Shell { private static final Logger LOG = LoggerFactory.getLogger(Shell.class); - private int exitCode; - - /** return an array containing the command name & its parameters */ + /** 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; + protected abstract void parseExecResult(BufferedReader lines) throws IOException; - /**Time after which the executing script would be timedout*/ - protected long timeOutInterval = 0L; + private final long timeout; - private long interval; // refresh interval in msec - private long lastTime; // last time the command was performed + private int exitCode; private Process process; // sub process used to execute the command - /**If or not script finished executing*/ + /* If or not script finished executing */ private volatile AtomicBoolean completed; - public static final Time TIME = new SystemTime(); - - public Shell() { - this(0L); - } - /** - * @param interval the minimum duration to wait before re-executing the - * command. + * @param timeout Specifies the time in milliseconds, after which the command will be killed. -1 means no timeout. */ - public Shell(long interval) { - this.interval = interval; - this.lastTime = (interval < 0) ? 0 : -interval; + public Shell(long timeout) { + this.timeout = timeout; } /** get the exit code @@ -86,10 +73,7 @@ public Process process() { return process; } - /** check to see if a command needs to be executed and execute if needed */ protected void run() throws IOException { - if (lastTime + interval > TIME.currentElapsedTime()) - return; exitCode = 0; // reset for next run runCommand(); } @@ -97,55 +81,47 @@ protected void run() throws IOException { /** Run a command */ private void runCommand() throws IOException { ProcessBuilder builder = new ProcessBuilder(execString()); - Timer timeOutTimer = null; - ShellTimeoutTimerTask timeoutTimerTask = null; + Timer timeoutTimer = null; completed = new AtomicBoolean(false); process = builder.start(); - if (timeOutInterval > 0) { - timeOutTimer = new Timer(); - timeoutTimerTask = new ShellTimeoutTimerTask( - this); + if (timeout > -1) { + timeoutTimer = new Timer(); //One time scheduling. - timeOutTimer.schedule(timeoutTimerTask, timeOutInterval); + timeoutTimer.schedule(new ShellTimeoutTimerTask(this), timeout); } - final BufferedReader errReader = - new BufferedReader(new InputStreamReader(process - .getErrorStream())); - BufferedReader inReader = - new BufferedReader(new InputStreamReader(process - .getInputStream())); + 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 = new Thread() { - @Override - public void run() { - try { - String line = errReader.readLine(); - while ((line != null) && !isInterrupted()) { - errMsg.append(line); - errMsg.append(System.getProperty("line.separator")); - line = errReader.readLine(); - } - } catch (IOException ioe) { - LOG.warn("Error reading the error stream", ioe); + 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); } - }; - try { - errThread.start(); - } catch (IllegalStateException ise) { } + } + }, false); + errThread.start(); + try { parseExecResult(inReader); // parse the output // clear the input stream buffer - String line = inReader.readLine(); + String line = null; while (line != null) { line = inReader.readLine(); } // wait for the process to finish and check the exit code - exitCode = process.waitFor(); + exitCode = process.waitFor(); try { // make sure that the error thread exits errThread.join(); @@ -161,8 +137,8 @@ public void run() { } catch (InterruptedException ie) { throw new IOException(ie.toString()); } finally { - if (timeOutTimer != null) - timeOutTimer.cancel(); + if (timeoutTimer != null) + timeoutTimer.cancel(); // close the input stream try { @@ -180,7 +156,6 @@ public void run() { } process.destroy(); - lastTime = TIME.currentElapsedTime(); } } @@ -212,26 +187,20 @@ public int getExitCode() { */ public static class ShellCommandExecutor extends Shell { - private String[] command; + private final String[] command; private StringBuffer output; - public ShellCommandExecutor(String[] execString) { - this(execString, 0L); - } - - /** * 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 and the status marked as timedout. - * If 0, the command will not be timed out. + * command will be killed. -1 means no timeout. */ public ShellCommandExecutor(String[] execString, long timeout) { + super(timeout); command = execString.clone(); - timeOutInterval = timeout; } @@ -244,11 +213,11 @@ protected String[] execString() { return command; } - protected void parseExecResult(BufferedReader lines) throws IOException { + protected void parseExecResult(BufferedReader reader) throws IOException { output = new StringBuffer(); char[] buf = new char[512]; int nRead; - while ((nRead = lines.read(buf, 0, buf.length)) > 0) { + while ((nRead = reader.read(buf, 0, buf.length)) > 0) { output.append(buf, 0, nRead); } } @@ -288,7 +257,7 @@ public String toString() { * @return the output of the executed command. */ public static String execCommand(String ... cmd) throws IOException { - return execCommand(cmd, 0L); + return execCommand(cmd, -1); } /** @@ -296,7 +265,7 @@ public static String execCommand(String ... cmd) throws IOException { * 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 marked timeout + * @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 { @@ -310,7 +279,7 @@ public static String execCommand(String[] cmd, long timeout) throws IOException */ private static class ShellTimeoutTimerTask extends TimerTask { - private Shell shell; + private final Shell shell; public ShellTimeoutTimerTask(Shell shell) { this.shell = shell; From a3bd8d25a575b0fabf0811fd448d96a677f2ed4c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 16:13:50 +0100 Subject: [PATCH 18/50] Config clean-ups --- .../java/org/apache/kafka/common/config/SaslConfigs.java | 8 +++----- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 -- 2 files changed, 3 insertions(+), 7 deletions(-) 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 index 82fd2a5406189..c8cfadf491eda 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -31,16 +31,14 @@ public class SaslConfigs { 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 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 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 long DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN = 1 * 60 * 1000L; } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 700b5b815dbf8..0a508903b5f62 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -514,8 +514,6 @@ object KafkaConfig { val SaslKerberosTicketRenewJitterDoc = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC val SaslKerberosMinTimeBeforeReloginDoc = SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC - - private val configDef = { import ConfigDef.Range._ import ConfigDef.ValidString._ From 6dea484dd3a20468fde9aef318446eb761446d4a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 16:14:39 +0100 Subject: [PATCH 19/50] Tweak logging and make fields final in `Login` --- .../kafka/common/security/kerberos/Login.java | 64 +++++++++---------- 1 file changed, 32 insertions(+), 32 deletions(-) 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 index 0cbef10e1675c..92b729dc27cee 100644 --- 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 @@ -53,39 +53,39 @@ 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 Double ticketRenewWindowFactor; + private final double ticketRenewWindowFactor; /** * Percentage of random jitter added to the renewal time */ - private Double ticketRenewJitter; + 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 Long minTimeBeforeRelogin; + private final long minTimeBeforeRelogin; - private static final Random RNG = new Random(); - - private volatile Subject subject = null; + private final String kinitCmd; - 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(); - private Map configs; + private volatile Subject subject; private LoginContext login; // Initialize 'lastLogin' to do a login at first time private long lastLogin; - private String kinitCmd; /** * Login constructor. The constructor starts the thread used @@ -98,7 +98,6 @@ public class Login { * Thrown if authentication fails. */ public Login(final String loginContextName, Map configs) throws LoginException { - this.configs = configs; 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); @@ -128,13 +127,14 @@ public Login(final String loginContextName, Map configs) throws Login principal = null; } - log.debug("checking if its isKrbTicket"); if (!isKrbTicket) { + log.debug("It is not a Kerberos ticket"); t = null; // if no TGT, do not bother with ticket management. return; } - log.debug("its a krb5ticket"); + 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 : @@ -172,7 +172,7 @@ public void run() { if ((nextRefresh > expiry) || ((now + minTimeBeforeRelogin) > expiry)) { // expiry is before next scheduled refresh). - log.info("refreshing now because expiry is before next scheduled refresh time."); + log.info("Refreshing now because expiry is before next scheduled refresh time."); nextRefresh = now; } else { if (nextRefresh < (now + minTimeBeforeRelogin)) { @@ -187,8 +187,8 @@ public void run() { } nextRefreshDate = new Date(nextRefresh); if (nextRefresh > expiry) { - log.error("next refresh: " + nextRefreshDate + " is later than expiry " + expiryDate - + ". This may indicate a clock skew problem. Check that this host and the KDC's " + log.error("Next refresh: " + nextRefreshDate + " is later than expiry " + expiryDate + + ". This may indicate a clock skew problem. Check that this host and the KDC " + "hosts' clocks are in sync. Exiting refresh thread."); return; } @@ -203,7 +203,7 @@ public void run() { return; } } else { - log.error("nextRefresh:" + nextRefreshDate + " is in the past: exiting refresh thread. Check" + log.error("NextRefresh:" + nextRefreshDate + " 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."); @@ -214,7 +214,7 @@ public void run() { int retry = 1; while (retry >= 0) { try { - log.debug("running ticket cache refresh command: " + kinitCmd + " " + kinitArgs); + log.debug("Running ticket cache refresh command: " + kinitCmd + " " + kinitArgs); Shell.execCommand(kinitCmd, kinitArgs); break; } catch (Exception e) { @@ -229,7 +229,7 @@ public void run() { } } else { log.warn("Could not renew TGT due to problem running shell command: '" + kinitCmd - + " " + kinitArgs + "'" + "; exception was:" + e + ". Exiting refresh thread.", e); + + " " + kinitArgs + "'" + "; exception was: " + e + ". Exiting refresh thread.", e); return; } } @@ -278,7 +278,7 @@ public void shutdown() { try { t.join(); } catch (InterruptedException e) { - log.warn("error while waiting for Login thread to shutdown: " + e); + log.warn("Error while waiting for Login thread to shutdown: " + e); } } } @@ -306,15 +306,15 @@ private synchronized LoginContext login(final String loginContextName) throws Lo LoginContext loginContext = new LoginContext(loginContextName, callbackHandler); loginContext.login(); - log.info("successfully logged in."); + 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().toString()); - log.info("TGT expires: " + tgt.getEndTime().toString()); + log.info("TGT valid starting at: " + tgt.getStartTime().toString()); + log.info("TGT expires: " + tgt.getEndTime().toString()); long proposedRefresh = start + (long) ((expires - start) * (ticketRenewWindowFactor + (ticketRenewJitter * RNG.nextDouble()))); @@ -330,7 +330,7 @@ private synchronized KerberosTicket getTGT() { for (KerberosTicket ticket : tickets) { KerberosPrincipal server = ticket.getServer(); if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { - log.debug("Found tgt " + ticket + "."); + log.debug("Found TGT " + ticket + "."); return ticket; } } @@ -360,7 +360,7 @@ private synchronized void reLogin() return; } if (login == null) { - throw new LoginException("login must be done first"); + throw new LoginException("Login must be done first"); } if (!hasSufficientTimeElapsed()) { return; @@ -373,7 +373,7 @@ private synchronized void reLogin() 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()); + login = new LoginContext(loginContextName, subject); log.info("Initiating re-login for " + principal); login.login(); } From d5768c8fb55f0f71fd7f067fc532aae708209416 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 17:03:47 +0100 Subject: [PATCH 20/50] Minor simplification of `SaslClientAuthenticator.complete` as per Jun's comment --- .../common/security/authenticator/SaslClientAuthenticator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 index ffe3424bf2e5d..f5fb56a8b8bd6 100644 --- 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 @@ -165,7 +165,7 @@ public Principal principal() { } public boolean complete() { - return saslClient.isComplete() && saslState == SaslState.COMPLETE; + return saslState == SaslState.COMPLETE; } public void close() throws IOException { From 37980d736cdaa60e938ada28b00653c4688aaec6 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 17:04:40 +0100 Subject: [PATCH 21/50] Tweak assignment of `Login.lastLogin` --- .../apache/kafka/common/security/kerberos/Login.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) 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 index 92b729dc27cee..ec423251cfbd3 100644 --- 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 @@ -84,7 +84,6 @@ public class Login { private volatile Subject subject; private LoginContext login; - // Initialize 'lastLogin' to do a login at first time private long lastLogin; /** @@ -103,8 +102,8 @@ public Login(final String loginContextName, Map configs) throws Login 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 = time.currentElapsedTime() - this.minTimeBeforeRelogin; + this.lastLogin = time.currentElapsedTime(); login = login(loginContextName); subject = login.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); @@ -169,8 +168,7 @@ public void run() { // 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)) { + 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; @@ -345,8 +343,6 @@ private boolean hasSufficientTimeElapsed() { " before."); return false; } - // register most recent relogin attempt - lastLogin = now; return true; } @@ -367,6 +363,8 @@ private synchronized void reLogin() } log.info("Initiating logout for " + principal); synchronized (Login.class) { + // register most recent relogin attempt + lastLogin = time.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 From ae430bebc6ddc04afac50b1be0c82b0c2faebb88 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 17:11:49 +0100 Subject: [PATCH 22/50] Remove `currentWallTime()` and `currentElapsedTime()` from `Time` for now Add private methods to `Login` instead as it's the only class where they are used. --- .../kafka/common/security/kerberos/Login.java | 19 ++++++++++++----- .../apache/kafka/common/utils/SystemTime.java | 10 --------- .../org/apache/kafka/common/utils/Time.java | 21 +++---------------- .../apache/kafka/common/utils/MockTime.java | 10 --------- 4 files changed, 17 insertions(+), 43 deletions(-) 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 index ec423251cfbd3..1d83c50e4e5fd 100644 --- 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 @@ -103,7 +103,7 @@ public Login(final String loginContextName, Map configs) throws Login this.minTimeBeforeRelogin = (Long) configs.get(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN); this.kinitCmd = (String) configs.get(SaslConfigs.SASL_KERBEROS_KINIT_CMD); - this.lastLogin = time.currentElapsedTime(); + this.lastLogin = currentElapsedTime(); login = login(loginContextName); subject = login.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); @@ -143,7 +143,7 @@ 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 = time.currentWallTime(); + long now = currentWallTime(); long nextRefresh; Date nextRefreshDate; if (tgt == null) { @@ -318,7 +318,7 @@ private long getRefreshTime(KerberosTicket tgt) { if (proposedRefresh > expires) // proposedRefresh is too far in the future: it's after ticket expires: simply return now. - return time.currentWallTime(); + return currentWallTime(); else return proposedRefresh; } @@ -336,7 +336,7 @@ private synchronized KerberosTicket getTGT() { } private boolean hasSufficientTimeElapsed() { - long now = time.currentElapsedTime(); + long now = currentElapsedTime(); if (now - lastLogin < minTimeBeforeRelogin) { log.warn("Not attempting to re-login since the last re-login was " + "attempted less than " + (minTimeBeforeRelogin / 1000) + " seconds" + @@ -364,7 +364,7 @@ private synchronized void reLogin() log.info("Initiating logout for " + principal); synchronized (Login.class) { // register most recent relogin attempt - lastLogin = time.currentElapsedTime(); + 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 @@ -376,4 +376,13 @@ private synchronized void reLogin() 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/utils/SystemTime.java b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java index cd75d209091a2..18725de836838 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java @@ -40,14 +40,4 @@ public void sleep(long ms) { } } - @Override - public long currentElapsedTime() { - return System.nanoTime() / 1000000; - } - - @Override - public long currentWallTime() { - return System.currentTimeMillis(); - } - } 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 93434c2b2ad79..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,31 +24,16 @@ public interface Time { /** * The current time in milliseconds */ - public long milliseconds(); + long milliseconds(); /** * The current time in nanoseconds */ - public long nanoseconds(); - - /** - * Returns time in milliseconds as does System.currentTimeMillis(), - * but uses elapsed time from an arbitrary epoch more like System.nanoTime(). - * The difference is that if somebody changes the system clock, - * Time.currentElapsedTime will change but nanoTime won't. - * @return The time in milliseconds from some arbitrary point in time. - */ - public long currentElapsedTime(); - - /** - * Explicitly returns system dependent current wall time. - * @return Current time in msec. - */ - public long currentWallTime(); + 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/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java index e92116d0f0536..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 @@ -40,14 +40,4 @@ public void sleep(long ms) { this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); } - @Override - public long currentElapsedTime() { - return 0; - } - - @Override - public long currentWallTime() { - return 0; - } - } From 190fe8613558258d4f478fa380cbfc2d868a876c Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 17:38:59 +0100 Subject: [PATCH 23/50] Rename SSLSASL to SASL_SSL and PLAINTEXTSASL to SASL_PLAIN --- .../main/java/org/apache/kafka/clients/ClientUtils.java | 2 +- .../org/apache/kafka/common/network/ChannelBuilders.java | 4 ++-- .../apache/kafka/common/network/SaslChannelBuilder.java | 4 ++-- .../apache/kafka/common/protocol/SecurityProtocol.java | 4 ++-- .../scala/integration/kafka/api/SaslConsumerTest.scala | 8 ++++---- .../scala/integration/kafka/api/SaslProducerTest.scala | 2 +- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 6 +++--- 7 files changed, 15 insertions(+), 15 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 2d6b3e397e9b2..14f23a7f87c82 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -74,7 +74,7 @@ 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 && securityProtocol != SecurityProtocol.PLAINTEXTSASL) + if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.SASL_PLAIN) throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); return ChannelBuilders.create(securityProtocol, Mode.CLIENT, configs); } 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 667c6f2e57930..f9e7bf08e74cf 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 @@ -36,8 +36,8 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode requireNonNullMode(mode, securityProtocol); channelBuilder = new SSLChannelBuilder(mode); break; - case SSLSASL: - case PLAINTEXTSASL: + case SASL_SSL: + case SASL_PLAIN: requireNonNullMode(mode, securityProtocol); channelBuilder = new SaslChannelBuilder(mode, securityProtocol); break; 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 index 5453eed7d3511..ef8a4db3a3b52 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -49,7 +49,7 @@ public void configure(Map configs) throws KafkaException { this.loginManager = LoginManager.getLoginManager(mode, configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); - if (this.securityProtocol == SecurityProtocol.SSLSASL) { + if (this.securityProtocol == SecurityProtocol.SASL_SSL) { this.sslFactory = new SSLFactory(mode); this.sslFactory.configure(this.configs); } @@ -63,7 +63,7 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize try { SocketChannel socketChannel = (SocketChannel) key.channel(); TransportLayer transportLayer; - if (this.securityProtocol == SecurityProtocol.SSLSASL) { + if (this.securityProtocol == SecurityProtocol.SASL_SSL) { transportLayer = new SSLTransportLayer(id, key, sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), socketChannel.socket().getPort())); 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 5aeed0b6bed9a..66d735b72e968 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 @@ -27,9 +27,9 @@ public enum SecurityProtocol { /** SSL channel */ SSL(1, "SSL"), /** SASL authenticated, non-encrypted channel */ - PLAINTEXTSASL(2, "PLAINTEXTSASL"), + SASL_PLAIN(2, "SASL_PLAIN"), /** SASL authenticated, SSL channel */ - SSLSASL(3, "SSLSASL"), + 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/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala index 3b2a4634b72b9..277b948b6dadc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala @@ -77,20 +77,20 @@ class SaslConsumerTest extends SaslTestHarness with Logging { val config = KafkaConfig.fromProps(props, overridingProps) servers = Buffer(TestUtils.createServer(config)) - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL)) + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN)) 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.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL)) + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN)) 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.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), + producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), acks = 1, enableSasl=true) for (i <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), + consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), groupId = "my-test", partitionAssignmentStrategy= "range", enableSasl=true) diff --git a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala index 5ff5a3ad3f51b..5d9be733b4d4d 100755 --- a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala @@ -67,7 +67,7 @@ class SaslProducerTest extends SaslTestHarness { @Test def testProduceAndConsume() { val topic = "new-topic" - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.PLAINTEXTSASL), enableSasl=true) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), enableSasl=true) val partition = new Integer(0) val numRecords = 100 diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 3e9e71f10134c..5572343a49b72 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -168,7 +168,7 @@ object TestUtils extends Logging { if (enableSSL) protocolAndPorts += "SSL" -> sslPort if (enableSasl) - protocolAndPorts += "PLAINTEXTSASL" -> saslPort + protocolAndPorts += "SASL_PLAIN" -> saslPort protocolAndPorts.map { case (protocol, port) => s"$protocol://localhost:$port" }.mkString(",") @@ -431,7 +431,7 @@ object TestUtils extends Logging { producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") producerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "producer")) } else if (enableSasl) { - producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXTSASL") + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAIN") } new KafkaProducer[Array[Byte],Array[Byte]](producerProps) } @@ -465,7 +465,7 @@ object TestUtils extends Logging { consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") consumerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "consumer")) } else if (enableSasl) { - consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXTSASL") + consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAIN") } new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } From ba29a432585c3dbe19c53e16b9ea12f6d4917bc0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 18:33:11 +0100 Subject: [PATCH 24/50] Call `removeInterestOps` when we complete. --- .../security/authenticator/SaslClientAuthenticator.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 index f5fb56a8b8bd6..2929a4965aec5 100644 --- 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 @@ -133,11 +133,12 @@ public void authenticate() throws IOException { netInBuffer = null; // reset the networkReceive as we read all the data. sendSaslToken(serverToken); } - if (saslClient.isComplete()) + if (saslClient.isComplete()) { saslState = SaslState.COMPLETE; + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + } break; case COMPLETE: - transportLayer.removeInterestOps(SelectionKey.OP_WRITE); break; case FAILED: throw new IOException("SASL handshake failed"); From 592c52c816385e43552f9da89ddbb8d71e5c6690 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 18:35:55 +0100 Subject: [PATCH 25/50] Remove methods from copycat.util.MockTime --- .../java/org/apache/kafka/copycat/util/MockTime.java | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java index 4f6ef4d6d582b..53149db016f84 100644 --- a/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java +++ b/copycat/runtime/src/test/java/org/apache/kafka/copycat/util/MockTime.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.utils.Time; import java.util.concurrent.TimeUnit; -import java.util.Date; /** * A clock that you can manually advance by calling sleep @@ -47,14 +46,4 @@ public void sleep(long ms) { this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS); } - @Override - public long currentElapsedTime() { - return 0; - } - - @Override - public long currentWallTime() { - return 0; - } - } From 2fc9708a94805c96ae9867d1f4865fd473ce009e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 13 Oct 2015 23:01:15 +0100 Subject: [PATCH 26/50] Rename SASL_PLAIN to SASL_PLAINTEXT In order to avoid confusion with the SASL/PLAIN mechanism. --- .../main/java/org/apache/kafka/clients/ClientUtils.java | 2 +- .../org/apache/kafka/common/network/ChannelBuilders.java | 2 +- .../apache/kafka/common/protocol/SecurityProtocol.java | 2 +- .../scala/integration/kafka/api/SaslConsumerTest.scala | 8 ++++---- .../scala/integration/kafka/api/SaslProducerTest.scala | 2 +- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 6 +++--- 6 files changed, 11 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java index 14f23a7f87c82..79ad10c2f74bb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -74,7 +74,7 @@ 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 && securityProtocol != SecurityProtocol.SASL_PLAIN) + if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.SASL_PLAINTEXT) throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); return ChannelBuilders.create(securityProtocol, Mode.CLIENT, configs); } 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 f9e7bf08e74cf..74dfb1a58a522 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 @@ -37,7 +37,7 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode channelBuilder = new SSLChannelBuilder(mode); break; case SASL_SSL: - case SASL_PLAIN: + case SASL_PLAINTEXT: requireNonNullMode(mode, securityProtocol); channelBuilder = new SaslChannelBuilder(mode, securityProtocol); break; 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 66d735b72e968..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 @@ -27,7 +27,7 @@ public enum SecurityProtocol { /** SSL channel */ SSL(1, "SSL"), /** SASL authenticated, non-encrypted channel */ - SASL_PLAIN(2, "SASL_PLAIN"), + 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. */ diff --git a/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala index 277b948b6dadc..64056bec9156d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala @@ -77,20 +77,20 @@ class SaslConsumerTest extends SaslTestHarness with Logging { val config = KafkaConfig.fromProps(props, overridingProps) servers = Buffer(TestUtils.createServer(config)) - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN)) + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) 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.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN)) + consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) 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.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), + producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), acks = 1, enableSasl=true) for (i <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), + consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), groupId = "my-test", partitionAssignmentStrategy= "range", enableSasl=true) diff --git a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala index 5d9be733b4d4d..c277aaf16d73e 100755 --- a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala @@ -67,7 +67,7 @@ class SaslProducerTest extends SaslTestHarness { @Test def testProduceAndConsume() { val topic = "new-topic" - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAIN), enableSasl=true) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), enableSasl=true) val partition = new Integer(0) val numRecords = 100 diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 5572343a49b72..78c12eb1aa34d 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -168,7 +168,7 @@ object TestUtils extends Logging { if (enableSSL) protocolAndPorts += "SSL" -> sslPort if (enableSasl) - protocolAndPorts += "SASL_PLAIN" -> saslPort + protocolAndPorts += "SASL_PLAINTEXT" -> saslPort protocolAndPorts.map { case (protocol, port) => s"$protocol://localhost:$port" }.mkString(",") @@ -431,7 +431,7 @@ object TestUtils extends Logging { producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") producerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "producer")) } else if (enableSasl) { - producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAIN") + producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT") } new KafkaProducer[Array[Byte],Array[Byte]](producerProps) } @@ -465,7 +465,7 @@ object TestUtils extends Logging { consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL") consumerProps.putAll(addSSLConfigs(Mode.CLIENT, false, trustStoreFile, "consumer")) } else if (enableSasl) { - consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAIN") + consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT") } new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } From da60654c190fbc9d4720bd84ff0a0ceaa37a57b2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 14 Oct 2015 17:57:38 +0100 Subject: [PATCH 27/50] Various improvements and fixes in SaslServerAuthenticator * Improve error handling * Use `KerberosName` to simplify code * Improve logging * Add comments explaining `usingNativeJgss` --- .../SaslServerAuthenticator.java | 97 ++++++++++--------- 1 file changed, 51 insertions(+), 46 deletions(-) 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 index 3fa9bc0b9cc09..26b1d2e91b961 100644 --- 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 @@ -33,6 +33,8 @@ 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.ietf.jgss.GSSContext; import org.ietf.jgss.GSSCredential; import org.ietf.jgss.GSSException; @@ -65,6 +67,10 @@ public class SaslServerAuthenticator implements Authenticator { private NetworkSend netOutBuffer; public SaslServerAuthenticator(String node, final Subject subject) 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; saslServer = createSaslServer(); @@ -75,57 +81,56 @@ public void configure(TransportLayer transportLayer, PrincipalBuilder principalB } private SaslServer createSaslServer() throws IOException { - if (subject != null) { - // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject. - if (subject.getPrincipals().size() > 0) { - try { - final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration()); - final Principal servicePrincipal = subject.getPrincipals().iterator().next(); - - final String servicePrincipalNameAndHostname = servicePrincipal.getName(); - int indexOf = servicePrincipalNameAndHostname.indexOf("/"); - final String servicePrincipalName = servicePrincipalNameAndHostname.substring(0, indexOf); - final String serviceHostnameAndKerbDomain = servicePrincipalNameAndHostname.substring(indexOf + 1, servicePrincipalNameAndHostname.length()); - indexOf = serviceHostnameAndKerbDomain.indexOf("@"); - final String serviceHostname = serviceHostnameAndKerbDomain.substring(0, indexOf); - final String mech = "GSSAPI"; - - LOG.debug("serviceHostname is '" + serviceHostname + "'"); - LOG.debug("servicePrincipalName is '" + servicePrincipalName + "'"); - LOG.debug("SASL mechanism is '" + mech + "'"); - boolean usingNativeJgss = Boolean.getBoolean("sun.security.jgss.native"); - if (usingNativeJgss) { - try { - GSSManager manager = GSSManager.getInstance(); - 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); - } - } + // 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()); + final Principal servicePrincipal = subject.getPrincipals().iterator().next(); + KerberosName kerberosName; + try { + kerberosName = new KerberosName(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 Subject.doAs(subject, new PrivilegedExceptionAction() { - public SaslServer run() { - try { - return Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler); - } catch (SaslException e) { - LOG.error("Kafka Server failed to create a SaslServer to interact with a client during session authentication: " + e); - return null; - } - } - }); - } catch (PrivilegedActionException e) { - LOG.error("KafkaBroker experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context:" + e); + 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 (IndexOutOfBoundsException e) { - LOG.error("Kafka Server principal name/hostname determination error: ", e); } - } + }); + } catch (PrivilegedActionException e) { + throw new KafkaException("Kafka Broker experienced a PrivilegedActionException exception while creating a SaslServer using a JAAS principal context", e); } - return null; } public void authenticate() throws IOException { From 03f6c0862a11a6c5c18923dab3307862b441f987 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 11:04:45 +0100 Subject: [PATCH 28/50] Avoid string concatenation in `Login` logging statements This way we don't pay for the cost unless the log is enabled. --- .../kafka/common/security/kerberos/Login.java | 54 +++++++++---------- 1 file changed, 25 insertions(+), 29 deletions(-) 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 index 1d83c50e4e5fd..9ce19cc3b40db 100644 --- 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 @@ -136,7 +136,7 @@ public Login(final String loginContextName, Map configs) throws Login // 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 : + // 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() { @@ -149,19 +149,20 @@ public void run() { if (tgt == null) { nextRefresh = now + minTimeBeforeRelogin; nextRefreshDate = new Date(nextRefresh); - log.warn("No TGT found: will try again at " + nextRefreshDate); + 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: " + expiryDate + "." + + 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 " + principal + "' within " + - "kadmin, or instead, to generate a keytab for " + principal + ". Because the TGT's " + - "expiry cannot be further extended by refreshing, exiting refresh thread now."); + "'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. @@ -177,23 +178,23 @@ public void run() { // 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 : " + until + " to : " + newUntil + " since " - + "the former is sooner than the minimum refresh interval (" - + minTimeBeforeRelogin / 1000 + " seconds) from now."); + 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: " + nextRefreshDate + " is later than expiry " + expiryDate - + ". This may indicate a clock skew problem. Check that this host and the KDC " - + "hosts' clocks are in sync. Exiting refresh thread."); + 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.toString()); + log.info("TGT refresh sleeping until: {}", until); try { Thread.sleep(nextRefresh - now); } catch (InterruptedException ie) { @@ -201,10 +202,10 @@ public void run() { return; } } else { - log.error("NextRefresh:" + nextRefreshDate + " is in the past: exiting refresh thread. Check" + 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."); + + " Exiting refresh thread.", nextRefreshDate); return; } if (isUsingTicketCache) { @@ -212,7 +213,7 @@ public void run() { int retry = 1; while (retry >= 0) { try { - log.debug("Running ticket cache refresh command: " + kinitCmd + " " + kinitArgs); + log.debug("Running ticket cache refresh command: {} {}", kinitCmd, kinitArgs); Shell.execCommand(kinitCmd, kinitArgs); break; } catch (Exception e) { @@ -276,7 +277,7 @@ public void shutdown() { try { t.join(); } catch (InterruptedException e) { - log.warn("Error while waiting for Login thread to shutdown: " + e); + log.warn("Error while waiting for Login thread to shutdown: " + e, e); } } } @@ -285,10 +286,6 @@ public Subject subject() { return subject; } - public String loginContextName() { - return loginContextName; - } - private synchronized LoginContext login(final String loginContextName) throws LoginException { if (System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) == null) { throw new IllegalArgumentException("You must pass " + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + " in secure mode."); @@ -311,8 +308,8 @@ private synchronized LoginContext login(final String loginContextName) throws Lo private long getRefreshTime(KerberosTicket tgt) { long start = tgt.getStartTime().getTime(); long expires = tgt.getEndTime().getTime(); - log.info("TGT valid starting at: " + tgt.getStartTime().toString()); - log.info("TGT expires: " + tgt.getEndTime().toString()); + log.info("TGT valid starting at: {}", tgt.getStartTime()); + log.info("TGT expires: {}", tgt.getEndTime()); long proposedRefresh = start + (long) ((expires - start) * (ticketRenewWindowFactor + (ticketRenewJitter * RNG.nextDouble()))); @@ -328,7 +325,7 @@ private synchronized KerberosTicket getTGT() { for (KerberosTicket ticket : tickets) { KerberosPrincipal server = ticket.getServer(); if (server.getName().equals("krbtgt/" + server.getRealm() + "@" + server.getRealm())) { - log.debug("Found TGT " + ticket + "."); + log.debug("Found TGT {}.", ticket); return ticket; } } @@ -338,9 +335,8 @@ private synchronized KerberosTicket getTGT() { 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 " + (minTimeBeforeRelogin / 1000) + " seconds" + - " before."); + log.warn("Not attempting to re-login since the last re-login was attempted less than {} seconds before.", + minTimeBeforeRelogin / 1000); return false; } return true; @@ -361,7 +357,7 @@ private synchronized void reLogin() if (!hasSufficientTimeElapsed()) { return; } - log.info("Initiating logout for " + principal); + log.info("Initiating logout for {}", principal); synchronized (Login.class) { // register most recent relogin attempt lastLogin = currentElapsedTime(); @@ -372,7 +368,7 @@ private synchronized void reLogin() //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); + log.info("Initiating re-login for {}", principal); login.login(); } } From cd26542b03317857f53a565cef48b026280fd7c5 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 13:13:56 +0100 Subject: [PATCH 29/50] Fix `LoginManager` to support multiple `Mode`s simultaneously and fix leak We now close `ChannelBuilder` instances and we have a simple reference counting system to ensure that we close `LoginManager` when it's no longer being used. --- .../common/network/SaslChannelBuilder.java | 14 ++-- .../apache/kafka/common/network/Selector.java | 1 + .../security/kerberos/LoginManager.java | 66 +++++++++++++++---- 3 files changed, 63 insertions(+), 18 deletions(-) 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 index ef8a4db3a3b52..81998585ebcb4 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -31,11 +31,13 @@ public class SaslChannelBuilder implements ChannelBuilder { private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class); + + private final SecurityProtocol securityProtocol; + private final Mode mode; + private LoginManager loginManager; private PrincipalBuilder principalBuilder; - private SecurityProtocol securityProtocol; private SSLFactory sslFactory; - private Mode mode; private Map configs; public SaslChannelBuilder(Mode mode, SecurityProtocol securityProtocol) { @@ -46,7 +48,7 @@ public SaslChannelBuilder(Mode mode, SecurityProtocol securityProtocol) { public void configure(Map configs) throws KafkaException { try { this.configs = configs; - this.loginManager = LoginManager.getLoginManager(mode, configs); + this.loginManager = LoginManager.acquireLoginManager(mode, configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); if (this.securityProtocol == SecurityProtocol.SASL_SSL) { @@ -59,7 +61,6 @@ public void configure(Map configs) throws KafkaException { } public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException { - KafkaChannel channel = null; try { SocketChannel socketChannel = (SocketChannel) key.channel(); TransportLayer transportLayer; @@ -76,16 +77,15 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize else authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(), socketChannel.socket().getInetAddress().getHostName()); authenticator.configure(transportLayer, this.principalBuilder, this.configs); - channel = new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); + return new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize); } catch (Exception e) { log.info("Failed to create channel due to ", e); throw new KafkaException(e); } - return channel; } public void close() { this.principalBuilder.close(); - this.loginManager.close(); + this.loginManager.release(); } } 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/security/kerberos/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java index 7d1f33e49401c..950413177e540 100644 --- 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 @@ -27,9 +27,14 @@ import org.apache.kafka.common.security.JaasUtils; public class LoginManager { + + private static LoginManager serverInstance; + private static LoginManager clientInstance; + private final Login login; private final String serviceName; - private volatile static LoginManager instance; + private final Mode mode; + private int refCount; private LoginManager(Mode mode, Map configs) throws IOException, LoginException { String loginContext; @@ -37,21 +42,38 @@ private LoginManager(Mode mode, Map configs) throws IOException, Logi loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; else loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; + this.mode = mode; this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); login = new Login(loginContext, configs); login.startThreadIfNeeded(); } - public static final LoginManager getLoginManager(Mode mode, Map configs) throws IOException, LoginException { - if (instance != null) { - return instance; - } else { - synchronized (LoginManager.class) { - if (instance == null) - instance = new LoginManager(mode, configs); + /** + * 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 a bit more complex. + */ + public static final LoginManager acquireLoginManager(Mode mode, Map configs) throws IOException, LoginException { + synchronized (LoginManager.class) { + switch (mode) { + case SERVER: + if (serverInstance == null) + serverInstance = new LoginManager(mode, configs); + return serverInstance.acquire(); + case CLIENT: + if (clientInstance == null) + clientInstance = new LoginManager(mode, configs); + return clientInstance.acquire(); + default: + throw new IllegalArgumentException("Unsupported `mode` received " + mode); } } - return instance; } public Subject subject() { @@ -62,7 +84,29 @@ public String serviceName() { return serviceName; } - public void close() { - login.shutdown(); + 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) { + if (mode == Mode.SERVER) + serverInstance = null; + else if (mode == Mode.CLIENT) + clientInstance = null; + else + throw new IllegalStateException("Unsupported `mode` " + mode); + + login.shutdown(); + } + --refCount; + } } } From 1885fdbc399a7a8765b009f9bcd90115ce128006 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 13:31:08 +0100 Subject: [PATCH 30/50] Clean-up logging in `SaslServerCallbackHandler` --- .../authenticator/SaslServerCallbackHandler.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) 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 index fa7612dbff2d8..6a06cd910bf36 100644 --- 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 @@ -39,11 +39,8 @@ public class SaslServerCallbackHandler implements CallbackHandler { public SaslServerCallbackHandler(Configuration configuration) throws IOException { AppConfigurationEntry[] configurationEntries = configuration.getAppConfigurationEntry(JaasUtils.LOGIN_CONTEXT_SERVER); - if (configurationEntries == null) { - String errorMessage = "Could not find a 'KafkaServer' entry in this configuration: Kafka Server cannot start."; - LOG.error(errorMessage); - throw new IOException(errorMessage); - } + if (configurationEntries == null) + throw new IOException("Could not find a 'KafkaServer' entry in this configuration: Kafka Server cannot start."); } public void handle(Callback[] callbacks) throws UnsupportedCallbackException { @@ -57,7 +54,7 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { } private void handleRealmCallback(RealmCallback rc) { - LOG.trace("client supplied realm: {} ", rc.getDefaultText()); + LOG.trace("Client supplied realm: {} ", rc.getDefaultText()); rc.setText(rc.getDefaultText()); } @@ -65,14 +62,14 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) { String authenticationID = ac.getAuthenticationID(); String authorizationID = ac.getAuthorizationID(); - LOG.info("Successfully authenticated client: authenticationID=" + authenticationID - + "; authorizationID=" + authorizationID + "."); + LOG.info("Successfully authenticated client: authenticationID={}; authorizationID={}.", authenticationID, + authorizationID); ac.setAuthorized(true); KerberosName kerberosName = new KerberosName(authenticationID); try { String userName = kerberosName.shortName(); - LOG.info("Setting authorizedID: " + userName); + LOG.info("Setting authorizedID: {}", userName); ac.setAuthorizedID(userName); } catch (IOException e) { LOG.error("Failed to set name based on Kerberos authentication rules."); From 3f26fd3b44ecf656e9553753f2e0b7f8b78c6e11 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 13:44:33 +0100 Subject: [PATCH 31/50] Remove `Configuration.setConfiguration` call As pointed out by Rajini, `JavaLoginConfig` is only guaranteed to be available in the SUN provider and the default behaviour of `getConfiguration` should do the job. --- .../apache/kafka/common/security/kerberos/Login.java | 11 ----------- 1 file changed, 11 deletions(-) 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 index 9ce19cc3b40db..8f9c9ca87dabc 100644 --- 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 @@ -38,9 +38,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.security.NoSuchAlgorithmException; -import java.security.URIParameter; import java.util.Date; import java.util.Random; import java.util.Set; @@ -291,14 +288,6 @@ private synchronized LoginContext login(final String loginContextName) throws Lo throw new IllegalArgumentException("You must pass " + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + " in secure mode."); } - File configFile = new File(System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM)); - try { - Configuration loginConf = Configuration.getInstance("JavaLoginConfig", new URIParameter(configFile.toURI())); - Configuration.setConfiguration(loginConf); - } catch (NoSuchAlgorithmException e) { - throw new RuntimeException(e); - } - LoginContext loginContext = new LoginContext(loginContextName, callbackHandler); loginContext.login(); log.info("Successfully logged in."); From f12eefa16ff466ab4b8baa3f5f4b286af801d485 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 15:13:17 +0100 Subject: [PATCH 32/50] Remove interestOps after data is sent in `Sasl*Authenticator` --- .../security/authenticator/SaslClientAuthenticator.java | 4 +++- .../security/authenticator/SaslServerAuthenticator.java | 6 +++--- 2 files changed, 6 insertions(+), 4 deletions(-) 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 index 2929a4965aec5..d104d0705c1a5 100644 --- 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 @@ -151,7 +151,9 @@ private void sendSaslToken(byte[] serverToken) throws IOException { byte[] saslToken = createSaslToken(serverToken); if (saslToken != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken)); - if (!flushNetOutBuffer()) + if (flushNetOutBuffer()) + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + else transportLayer.addInterestOps(SelectionKey.OP_WRITE); } } catch (SaslException se) { 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 index 26b1d2e91b961..4d160bfd5ffde 100644 --- 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 @@ -157,10 +157,10 @@ public void authenticate() throws IOException { byte[] response = saslServer.evaluateResponse(clientToken); if (response != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); - if (!flushNetOutBuffer()) { + if (flushNetOutBuffer()) + transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + else transportLayer.addInterestOps(SelectionKey.OP_WRITE); - return; - } } } catch (Exception e) { throw new IOException(e); From b889f5d15c286897856c0c9b673161ec7bcdb01b Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 15 Oct 2015 15:14:50 +0100 Subject: [PATCH 33/50] Clean-up logging in `SaslClientAuthenticator` and minor code clean-up --- .../SaslClientAuthenticator.java | 27 +++++++------------ 1 file changed, 10 insertions(+), 17 deletions(-) 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 index d104d0705c1a5..29fbf8f2262e2 100644 --- 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 @@ -21,6 +21,7 @@ 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; @@ -99,15 +100,14 @@ private SaslClient createSaslClient() { try { return Subject.doAs(subject, new PrivilegedExceptionAction() { public SaslClient run() throws SaslException { - LOG.debug("Client will use GSSAPI as SASL mechanism."); String[] mechs = {"GSSAPI"}; - LOG.debug("creating sasl client: client=" + clientPrincipalName + ";service=" + servicePrincipal + ";serviceHostname=" + host); - SaslClient saslClient = Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null, new ClientCallbackHandler()); - return saslClient; + 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) { - LOG.error("Exception while trying to create SASL client", e); throw new KafkaException("Failed to create SASL client", e); } } @@ -201,7 +201,6 @@ public byte[] run() throws SaslException { " `socketChannel.socket().getInetAddress().getHostName()` must match the hostname in `principal/hostname@realm`"; } error += " Kafka Client will go to AUTH_FAILED state."; - LOG.error(error); throw new SaslException(error); } } @@ -224,7 +223,7 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { // 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 " + + " 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."); @@ -233,17 +232,11 @@ public void handle(Callback[] callbacks) throws UnsupportedCallbackException { rc.setText(rc.getDefaultText()); } else if (callback instanceof AuthorizeCallback) { AuthorizeCallback ac = (AuthorizeCallback) callback; - String authid = ac.getAuthenticationID(); - String authzid = ac.getAuthorizationID(); - - if (authid.equals(authzid)) - ac.setAuthorized(true); - else - ac.setAuthorized(false); - - + String authId = ac.getAuthenticationID(); + String authzId = ac.getAuthorizationID(); + ac.setAuthorized(authId.equals(authzId)); if (ac.isAuthorized()) - ac.setAuthorizedID(authzid); + ac.setAuthorizedID(authzId); } else { throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback"); } From b121710aa24e76e4dde230227ca6bd5bcc933e0f Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 15 Oct 2015 19:58:03 -0700 Subject: [PATCH 34/50] KAFKA-1686. Renamed SaslConsumerTest to SaslProducerTest. --- ...erTest.scala => SaslIntegrationTest.scala} | 2 +- .../kafka/api/SaslProducerTest.scala | 130 ------------------ 2 files changed, 1 insertion(+), 131 deletions(-) rename core/src/test/scala/integration/kafka/api/{SaslConsumerTest.scala => SaslIntegrationTest.scala} (99%) delete mode 100755 core/src/test/scala/integration/kafka/api/SaslProducerTest.scala diff --git a/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala similarity index 99% rename from core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala rename to core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala index 64056bec9156d..e5b65d57954bb 100644 --- a/core/src/test/scala/integration/kafka/api/SaslConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala @@ -42,7 +42,7 @@ import scala.collection.mutable.Buffer import scala.collection.JavaConversions._ import kafka.coordinator.ConsumerCoordinator -class SaslConsumerTest extends SaslTestHarness with Logging { +class SaslIntegrationTest extends SaslTestHarness with Logging { val brokerId1 = 0 var servers: Buffer[KafkaServer] = null val numServers = 1 diff --git a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala b/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala deleted file mode 100755 index c277aaf16d73e..0000000000000 --- a/core/src/test/scala/integration/kafka/api/SaslProducerTest.scala +++ /dev/null @@ -1,130 +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 -import java.util.Properties - -import kafka.admin.AdminUtils -import kafka.common.{ErrorMapping, FailedToSendMessageException} -import kafka.consumer.SimpleConsumer -import kafka.message.Message -import kafka.producer.KeyedMessage -import kafka.serializer.StringEncoder -import kafka.server.{KafkaConfig, KafkaRequestHandler, KafkaServer} -import kafka.utils._ -import kafka.zk.ZooKeeperTestHarness -import org.apache.kafka.common.protocol.SecurityProtocol -import org.apache.kafka.clients.producer._ - -import org.junit.Assert._ -import org.junit.{After, Before, Test} - - -class SaslProducerTest extends SaslTestHarness { - private val brokerId1 = 0 - private var server1: KafkaServer = null - private var consumer1: SimpleConsumer = null - private var servers = List.empty[KafkaServer] - - @Before - override def setUp() { - super.setUp() - // set up 2 brokers with 4 partitions each - val props1 = TestUtils.createBrokerConfig(brokerId1, zkConnect, false, enableSasl = true) - props1.put("num.partitions", "1") - props1.put(KafkaConfig.DefaultReplicationFactorProp, "1") - val config1 = KafkaConfig.fromProps(props1) - - server1 = TestUtils.createServer(config1) - servers = List(server1) - consumer1 = new SimpleConsumer("localhost", server1.boundPort(SecurityProtocol.PLAINTEXT), 1000000, 64*1024, "") - } - - @After - override def tearDown() { - consumer1.close() - server1.shutdown - CoreUtils.rm(server1.config.logDirs) - super.tearDown() - } - - @Test - def testProduceAndConsume() { - val topic = "new-topic" - val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), enableSasl=true) - val partition = new Integer(0) - val numRecords = 100 - - 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 with 1 partition and await leadership - TestUtils.createTopic(zkClient, topic, 1, 1, servers) - - val record0 = new ProducerRecord[Array[Byte],Array[Byte]](topic, partition, "key".getBytes, "value".getBytes) - assertEquals("Should have offset 0", 0L, producer.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, producer.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, producer.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, producer.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) - producer.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 producer - for (i <- 1 to numRecords) - producer.send(record0, callback) - // check that all messages have been acked via offset - assertEquals("Should have offset " + numRecords + 4L, numRecords + 4L, producer.send(record0, callback).get.offset) - - - } finally { - producer.close() - } - } -} From 538b6eb974a2bc5f43ff649213eab1f64cc925cc Mon Sep 17 00:00:00 2001 From: Sriharsha Chintalapani Date: Thu, 15 Oct 2015 20:47:36 -0700 Subject: [PATCH 35/50] KAFKA-1686. added verify to SaslIntegrationTest. --- .../kafka/api/SaslIntegrationTest.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala index e5b65d57954bb..afa6db50f0b3b 100644 --- a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala @@ -47,7 +47,7 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { var servers: Buffer[KafkaServer] = null val numServers = 1 val producerCount = 1 - val consumerCount = 2 + val consumerCount = 1 val producerConfig = new Properties val consumerConfig = new Properties @@ -57,8 +57,8 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { 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]]]() + var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() + var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() val topic = "topic" val part = 0 @@ -78,11 +78,7 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { servers = Buffer(TestUtils.createServer(config)) producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) - 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.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) - 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) @@ -123,17 +119,17 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { 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) + consumeAndVerifyRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) } 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)) + this.producers(0).send(new ProducerRecord(topic, part, ("key" + i).getBytes, ("value" + i).getBytes)) } futures.map(_.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) { val records = new ArrayList[ConsumerRecord[Array[Byte], Array[Byte]]]() val maxIters = numRecords * 300 var iters = 0 @@ -151,6 +147,8 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { assertEquals(topic, record.topic()) assertEquals(part, record.partition()) assertEquals(offset.toLong, record.offset()) + assertEquals("key" + i, new String(record.key())) + assertEquals("value" + i, new String(record.value())) } } From 80949b4836d53ddcc393ff2e66a012d130cbc686 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 03:04:28 +0100 Subject: [PATCH 36/50] Introduce `LoginType` to use the right JAAS section in broker Also simplify `LoginManager` caching by using `EnumMap`. --- .../org/apache/kafka/clients/ClientUtils.java | 3 +- .../kafka/common/network/ChannelBuilders.java | 12 +++-- .../kafka/common/network/LoginType.java | 39 +++++++++++++++ .../common/network/SaslChannelBuilder.java | 6 ++- .../kafka/common/security/JaasUtils.java | 4 -- .../security/kerberos/LoginManager.java | 50 +++++++------------ .../controller/ControllerChannelManager.scala | 4 +- .../scala/kafka/network/SocketServer.scala | 4 +- .../main/scala/kafka/server/KafkaServer.scala | 7 ++- .../kafka/server/ReplicaFetcherThread.scala | 4 +- 10 files changed, 81 insertions(+), 52 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/network/LoginType.java 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 79ad10c2f74bb..6ed6a08a91137 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -20,6 +20,7 @@ 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; @@ -76,7 +77,7 @@ public static ChannelBuilder createChannelBuilder(Map configs) { SecurityProtocol securityProtocol = SecurityProtocol.valueOf((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); if (securityProtocol != SecurityProtocol.SSL && securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.SASL_PLAINTEXT) throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); - return ChannelBuilders.create(securityProtocol, Mode.CLIENT, configs); + return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs); } } 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 74dfb1a58a522..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 @@ -23,13 +23,15 @@ 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, 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: @@ -39,7 +41,9 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode case SASL_SSL: case SASL_PLAINTEXT: requireNonNullMode(mode, securityProtocol); - channelBuilder = new SaslChannelBuilder(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: 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/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index 81998585ebcb4..f0f0758812d5e 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -34,21 +34,23 @@ public class SaslChannelBuilder implements ChannelBuilder { private final SecurityProtocol securityProtocol; private final Mode mode; + private final LoginType loginType; private LoginManager loginManager; private PrincipalBuilder principalBuilder; private SSLFactory sslFactory; private Map configs; - public SaslChannelBuilder(Mode mode, SecurityProtocol securityProtocol) { + 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(mode, configs); + this.loginManager = LoginManager.acquireLoginManager(loginType, configs); this.principalBuilder = (PrincipalBuilder) Utils.newInstance((Class) configs.get(SSLConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)); this.principalBuilder.configure(configs); if (this.securityProtocol == SecurityProtocol.SASL_SSL) { 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 817049a8fbdf5..96b33f4c75e44 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 @@ -23,11 +23,7 @@ import java.lang.reflect.Method; import java.io.IOException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - public class JaasUtils { - private static final Logger LOG = LoggerFactory.getLogger(JaasUtils.class); public static final String LOGIN_CONTEXT_SERVER = "KafkaServer"; public static final String LOGIN_CONTEXT_CLIENT = "KafkaClient"; public static final String SERVICE_NAME = "serviceName"; 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 index 950413177e540..66d3169fec9e4 100644 --- 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 @@ -21,28 +21,24 @@ 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.network.Mode; +import org.apache.kafka.common.network.LoginType; import org.apache.kafka.common.security.JaasUtils; public class LoginManager { - private static LoginManager serverInstance; - private static LoginManager clientInstance; + private static final EnumMap CACHED_INSTANCES = new EnumMap(LoginType.class); private final Login login; private final String serviceName; - private final Mode mode; + private final LoginType loginType; private int refCount; - private LoginManager(Mode mode, Map configs) throws IOException, LoginException { - String loginContext; - if (mode == Mode.SERVER) - loginContext = JaasUtils.LOGIN_CONTEXT_SERVER; - else - loginContext = JaasUtils.LOGIN_CONTEXT_CLIENT; - this.mode = mode; + private LoginManager(LoginType loginType, Map configs) throws IOException, LoginException { + this.loginType = loginType; + String loginContext = loginType.contextName(); this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); login = new Login(loginContext, configs); login.startThreadIfNeeded(); @@ -57,22 +53,20 @@ private LoginManager(Mode mode, Map configs) throws IOException, Logi * * 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 a bit more complex. + * 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(Mode mode, Map configs) throws IOException, LoginException { + public static final LoginManager acquireLoginManager(LoginType loginType, Map configs) throws IOException, LoginException { synchronized (LoginManager.class) { - switch (mode) { - case SERVER: - if (serverInstance == null) - serverInstance = new LoginManager(mode, configs); - return serverInstance.acquire(); - case CLIENT: - if (clientInstance == null) - clientInstance = new LoginManager(mode, configs); - return clientInstance.acquire(); - default: - throw new IllegalArgumentException("Unsupported `mode` received " + mode); + LoginManager loginManager = CACHED_INSTANCES.get(loginType); + if (loginManager == null) { + loginManager = new LoginManager(loginType, configs); + CACHED_INSTANCES.put(loginType, loginManager); } + return loginManager.acquire(); } } @@ -97,13 +91,7 @@ public void release() { if (refCount == 0) throw new IllegalStateException("release called on LoginManager with refCount == 0"); else if (refCount == 1) { - if (mode == Mode.SERVER) - serverInstance = null; - else if (mode == Mode.CLIENT) - clientInstance = null; - else - throw new IllegalStateException("Unsupported `mode` " + mode); - + CACHED_INSTANCES.remove(loginType); login.shutdown(); } --refCount; diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 5a26698e5e4ba..dae20efa1be48 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -21,7 +21,7 @@ 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, Mode} +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.utils.Time @@ -96,7 +96,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf "controller-channel", Map("broker-id" -> broker.id.toString).asJava, false, - ChannelBuilders.create(config.interBrokerSecurityProtocol, 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 f09272fe45621..1066fbee1a5c0 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -33,7 +33,7 @@ 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.{Mode, 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.protocol.SecurityProtocol import org.apache.kafka.common.protocol.types.SchemaException @@ -377,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, 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/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 9ad8e6b469f96..1e1b1a18cb06c 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,11 +34,10 @@ 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, Mode} +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.ssl.SSLFactory import org.apache.kafka.common.utils.AppInfoParser import scala.collection.mutable @@ -304,7 +303,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr "kafka-server-controlled-shutdown", Map.empty.asJava, false, - ChannelBuilders.create(config.interBrokerSecurityProtocol, 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/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 7123ded20a788..a80366097a05f 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, Mode} +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, Mode.CLIENT, brokerConfig.channelConfigs) + ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, brokerConfig.channelConfigs) ) new NetworkClient( selector, From 7ce068c0db5113702d82f82af82175bf547da862 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 08:59:47 +0100 Subject: [PATCH 37/50] Remove unused `KerberosName.getDefaultRealm()` and add TODO to `JaasUtils.defaultRealm()` --- .../java/org/apache/kafka/common/security/JaasUtils.java | 4 ++++ .../kafka/common/security/kerberos/KerberosName.java | 8 -------- 2 files changed, 4 insertions(+), 8 deletions(-) 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 96b33f4c75e44..56f092fb6b769 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 @@ -54,6 +54,10 @@ 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; 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 index ab85f2ceb94ef..501a5567894c1 100644 --- 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 @@ -105,14 +105,6 @@ public KerberosName(String name) { } } - /** - * Get the configured default realm. - * @return the default realm from the krb5.conf - */ - public String getDefaultRealm() { - return DEFAULT_REALM; - } - /** * Put the name back together from the parts. */ From 0a1678e8a5df1f9e28396f4dc98c084566e69d16 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 11:10:24 +0100 Subject: [PATCH 38/50] Refactor `KerberosName` in order to make authToLocal rules configurable The original class has been split into multiple ones and a simple test was introduced. --- .../clients/consumer/ConsumerConfig.java | 1 + .../clients/producer/ProducerConfig.java | 1 + .../kafka/common/config/SaslConfigs.java | 6 + .../common/network/SaslChannelBuilder.java | 19 +- .../SaslClientAuthenticator.java | 8 +- .../SaslServerAuthenticator.java | 10 +- .../SaslServerCallbackHandler.java | 8 +- .../security/kerberos/KerberosName.java | 276 +----------------- .../security/kerberos/KerberosNameParser.java | 103 +++++++ .../security/kerberos/KerberosRule.java | 189 ++++++++++++ .../security/kerberos/NoMatchingRule.java | 27 ++ .../security/kerberos/KerberosNameTest.java | 59 ++++ .../main/scala/kafka/server/KafkaConfig.scala | 6 + .../unit/kafka/server/KafkaConfigTest.scala | 1 + 14 files changed, 439 insertions(+), 275 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosNameParser.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosRule.java create mode 100644 clients/src/main/java/org/apache/kafka/common/security/kerberos/NoMatchingRule.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/kerberos/KerberosNameTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 1d50e9a339fe7..bfbb47a2998e8 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 @@ -308,6 +308,7 @@ public class ConsumerConfig extends AbstractConfig { .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 ce19f88df31cf..bad830beca803 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 @@ -285,6 +285,7 @@ public class ProducerConfig extends AbstractConfig { .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 index c8cfadf491eda..c4d9e4866dd57 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -13,6 +13,8 @@ package org.apache.kafka.common.config; +import java.util.Collections; +import java.util.List; public class SaslConfigs { /* @@ -41,4 +43,8 @@ public class SaslConfigs { 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/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index f0f0758812d5e..7db8e3a9c85a0 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -14,9 +14,13 @@ 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; @@ -40,6 +44,7 @@ public class SaslChannelBuilder implements ChannelBuilder { private PrincipalBuilder principalBuilder; private SSLFactory sslFactory; private Map configs; + private KerberosNameParser kerberosNameParser; public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol) { this.mode = mode; @@ -53,6 +58,15 @@ public void configure(Map configs) throws KafkaException { 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); @@ -75,9 +89,10 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize } Authenticator authenticator; if (mode == Mode.SERVER) - authenticator = new SaslServerAuthenticator(id, loginManager.subject()); + authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosNameParser); else - authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(), socketChannel.socket().getInetAddress().getHostName()); + 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) { 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 index 29fbf8f2262e2..646af199950c0 100644 --- 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 @@ -46,9 +46,9 @@ 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.security.kerberos.KerberosName; import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.security.kerberos.KerberosNameParser; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +64,7 @@ public enum SaslState { private final String servicePrincipal; private final String host; private final String node; + private final KerberosNameParser kerberosNameParser; // assigned in `configure` private SaslClient saslClient; @@ -76,11 +77,12 @@ public enum SaslState { private SaslState saslState = SaslState.INITIAL; - public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host) throws IOException { + 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 { @@ -89,7 +91,7 @@ public void configure(TransportLayer transportLayer, PrincipalBuilder principalB // determine client principal from subject. Principal clientPrincipal = subject.getPrincipals().iterator().next(); - this.clientPrincipalName = new KerberosName(clientPrincipal.getName()).toString(); + this.clientPrincipalName = kerberosNameParser.parse(clientPrincipal.getName()).toString(); this.saslClient = createSaslClient(); } catch (Exception e) { throw new KafkaException("Failed to configure SaslClientAuthenticator", e); 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 index 4d160bfd5ffde..fe07f08dccf0a 100644 --- 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 @@ -35,6 +35,7 @@ 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; @@ -58,6 +59,7 @@ public class SaslServerAuthenticator implements Authenticator { private final SaslServer saslServer; private final Subject subject; private final String node; + private final KerberosNameParser kerberosNameParser; // assigned in `configure` private TransportLayer transportLayer; @@ -66,13 +68,14 @@ public class SaslServerAuthenticator implements Authenticator { private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; - public SaslServerAuthenticator(String node, final Subject subject) throws IOException { + 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(); } @@ -82,11 +85,12 @@ public void configure(TransportLayer transportLayer, PrincipalBuilder principalB 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()); + final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler( + Configuration.getConfiguration(), kerberosNameParser); final Principal servicePrincipal = subject.getPrincipals().iterator().next(); KerberosName kerberosName; try { - kerberosName = new KerberosName(servicePrincipal.getName()); + kerberosName = kerberosNameParser.parse(servicePrincipal.getName()); } catch (IllegalArgumentException e) { throw new KafkaException("Principal has name with unexpected format " + servicePrincipal); } 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 index 6a06cd910bf36..8474faf491edd 100644 --- 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 @@ -20,6 +20,7 @@ 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; @@ -35,12 +36,13 @@ public class SaslServerCallbackHandler implements CallbackHandler { private static final Logger LOG = LoggerFactory.getLogger(SaslServerCallbackHandler.class); + private final KerberosNameParser kerberosNameParser; - public SaslServerCallbackHandler(Configuration configuration) throws IOException { + 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 { @@ -66,7 +68,7 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) { authorizationID); ac.setAuthorized(true); - KerberosName kerberosName = new KerberosName(authenticationID); + KerberosName kerberosName = kerberosNameParser.parse(authenticationID); try { String userName = kerberosName.shortName(); LOG.info("Setting authorizedID: {}", userName); 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 index 501a5567894c1..aef10db83ced1 100644 --- 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 @@ -19,19 +19,10 @@ 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; -import org.apache.kafka.common.security.JaasUtils; - -/** - * 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 KerberosName { + /** The first component of the name */ private final String serviceName; /** The second component of the name. It may be null. */ @@ -39,70 +30,19 @@ public class KerberosName { /** The realm of the name. */ private final String realm; - /** - * A pattern that matches a Kerberos name with at most 3 components. - */ - private static final Pattern NAME_PARSER = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)"); - - /** - * 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 for parsing a auth_to_local rule. - */ - private static final Pattern RULE_PARSER = Pattern.compile("\\s*((DEFAULT)|(RULE:\\[(\\d*):([^\\]]*)](\\(([^)]*)\\))?" + - "(s/([^/]*)/([^/]*)/(g)?)?))"); - - /** - * A pattern that recognizes simple/non-simple names. - */ - private static final Pattern NON_SIMPLE_PATTERN = Pattern.compile("[/@]"); + /* Rules for the translation of the principal name into an operating system name */ + private final List authToLocalRules; /** - * The list of translation rules. + * Creates an instance of `KerberosName` with the provided parameters. */ - private static final List RULES; - - private static final String DEFAULT_REALM; - - static { - String defaultRealm; - try { - defaultRealm = JaasUtils.defaultRealm(); - } catch (Exception ke) { - defaultRealm = ""; - } - DEFAULT_REALM = defaultRealm; - try { - String ruleString = System.getProperty("kafka.security.auth_to_local", "DEFAULT"); - RULES = parseRules(ruleString); - } catch (Exception e) { - throw new IllegalArgumentException("Could not configure Kerberos principal name mapping."); - } - } - - /** - * Create a name from the full Kerberos principal name. - * @param name - */ - public KerberosName(String name) { - Matcher match = NAME_PARSER.matcher(name); - if (!match.matches()) { - if (name.contains("@")) { - throw new IllegalArgumentException("Malformed Kerberos name: " + name); - } else { - serviceName = name; - hostName = null; - realm = null; - } - } else { - serviceName = match.group(1); - hostName = match.group(3); - realm = match.group(4); - } + 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; } /** @@ -147,198 +87,6 @@ public String realm() { return realm; } - /** - * An encoding of a rule for translating kerberos names. - */ - private static class Rule { - 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; - - Rule() { - isDefault = true; - numOfComponents = 0; - format = null; - match = null; - fromPattern = null; - toPattern = null; - repeat = false; - } - - Rule(int numOfComponents, String format, String match, String fromPattern, - String toPattern, boolean repeat) { - 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 BadFormatString - */ - static String replaceParameters(String format, - String[] params) throws 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 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 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 (DEFAULT_REALM.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; - } - } - - static List parseRules(String rules) { - List result = new ArrayList(); - String remaining = rules.trim(); - while (remaining.length() > 0) { - Matcher matcher = RULE_PARSER.matcher(remaining); - if (!matcher.lookingAt()) { - throw new IllegalArgumentException("Invalid rule: " + remaining); - } - if (matcher.group(2) != null) { - result.add(new Rule()); - } else { - result.add(new Rule(Integer.parseInt(matcher.group(4)), - matcher.group(5), - matcher.group(7), - matcher.group(9), - matcher.group(10), - "g".equals(matcher.group(11)))); - } - remaining = remaining.substring(matcher.end()); - } - return result; - } - - @SuppressWarnings("serial") - public static class BadFormatString extends IOException { - BadFormatString(String msg) { - super(msg); - } - BadFormatString(String msg, Throwable err) { - super(msg, err); - } - } - - @SuppressWarnings("serial") - public static class NoMatchingRule extends IOException { - NoMatchingRule(String msg) { - super(msg); - } - } - /** * Get the translation of the principal name into an operating system * user name. @@ -355,7 +103,7 @@ public String shortName() throws IOException { } else { params = new String[]{realm, serviceName, hostName}; } - for (Rule r : RULES) { + for (KerberosRule r : authToLocalRules) { String result = r.apply(params); if (result != null) return result; 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/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/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/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 729fc3841b9d8..95949d4371484 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -180,6 +180,7 @@ object Defaults { 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 } @@ -334,6 +335,7 @@ object KafkaConfig { 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 ***********/ @@ -510,6 +512,7 @@ object KafkaConfig { 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._ @@ -672,6 +675,7 @@ object KafkaConfig { .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) } @@ -838,6 +842,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka 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) @@ -982,6 +987,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka channelConfigs.put(SaslKerberosTicketRenewWindowFactorProp, saslKerberosTicketRenewWindowFactor) channelConfigs.put(SaslKerberosTicketRenewJitterProp, saslKerberosTicketRenewJitter) channelConfigs.put(SaslKerberosMinTimeBeforeReloginProp, saslKerberosMinTimeBeforeRelogin) + channelConfigs.put(AuthToLocalProp, authToLocal) channelConfigs } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index e664a3226cabe..17d66a47529e0 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -512,6 +512,7 @@ class KafkaConfigTest { case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp => case KafkaConfig.SaslKerberosTicketRenewJitterProp => case KafkaConfig.SaslKerberosMinTimeBeforeReloginProp => + case KafkaConfig.AuthToLocalProp => // ignore string case nonNegativeIntProperty => assertPropertyInvalid(getBaseProperties(), name, "not_a_number", "-1") } From a7cb82ce0a162e3acb81c80f9d6fafce5415c0c1 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 11:30:36 +0100 Subject: [PATCH 39/50] Tweak `KafkaChannel.prepare` to attempt authentication after handshake if transportLayer is ready --- .../java/org/apache/kafka/common/network/KafkaChannel.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 1612fe80a7d5a..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,12 +61,12 @@ 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()) transportLayer.handshake(); - else if (transportLayer.ready() && !authenticator.complete()) + if (transportLayer.ready() && !authenticator.complete()) authenticator.authenticate(); } From e94fe3e31b584fed3fb5a1862e6efed4d7456f88 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 15:22:25 +0100 Subject: [PATCH 40/50] Fix issue where `SaslChannelBuilder` was not calling `SSLTransportLayer.startHandshake` There is now a `SSLTransportLayer.create` method that initialises the instance properly. --- .../common/network/SSLChannelBuilder.java | 8 +++---- .../common/network/SSLTransportLayer.java | 9 +++++++- .../common/network/SaslChannelBuilder.java | 21 ++++++++++++------- 3 files changed, 24 insertions(+), 14 deletions(-) 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 b9ee9c2bd8d19..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 @@ -68,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 index 7db8e3a9c85a0..53953c5504821 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.network; +import java.io.IOException; import java.nio.channels.SelectionKey; import java.nio.channels.SocketChannel; import java.util.List; @@ -79,14 +80,7 @@ public void configure(Map configs) throws KafkaException { public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize) throws KafkaException { try { SocketChannel socketChannel = (SocketChannel) key.channel(); - TransportLayer transportLayer; - if (this.securityProtocol == SecurityProtocol.SASL_SSL) { - transportLayer = new SSLTransportLayer(id, key, - sslFactory.createSSLEngine(socketChannel.socket().getInetAddress().getHostName(), - socketChannel.socket().getPort())); - } else { - transportLayer = new PlaintextTransportLayer(key); - } + TransportLayer transportLayer = buildTransportLayer(id, key, socketChannel); Authenticator authenticator; if (mode == Mode.SERVER) authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosNameParser); @@ -105,4 +99,15 @@ 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); + } + } + } From 11521255c72767a2973a1aaec6cbf0298872ade0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 15:17:43 +0100 Subject: [PATCH 41/50] Introduce inter-broker SASL tests Refactor createBrokerConfig methods in `TestUtils` to make this possible. --- .../kafka/api/SSLConsumerTest.scala | 2 +- .../kafka/api/SSLProducerSendTest.scala | 2 +- .../kafka/api/SaslIntegrationTest.scala | 3 +- .../kafka/api/SaslTestHarness.scala | 7 +- .../integration/BaseTopicMetadataTest.scala | 12 ++- .../PlaintextTopicMetadataTest.scala | 3 + .../SaslPlaintextTopicMetadataTest.scala | 26 +++++++ .../SaslSslTopicMetadataTest.scala | 28 +++++++ .../integration/SslTopicMetadataTest.scala | 3 + .../unit/kafka/network/SocketServerTest.scala | 3 +- .../kafka/server/BaseReplicaFetchTest.scala | 10 ++- .../server/PlaintextReplicaFetchTest.scala | 3 + .../SaslPlaintextReplicaFetchTest.scala | 26 +++++++ .../server/SaslSslReplicaFetchTest.scala | 28 +++++++ .../kafka/server/SslReplicaFetchTest.scala | 3 + .../scala/unit/kafka/utils/TestUtils.scala | 77 +++++++++++++------ 16 files changed, 199 insertions(+), 37 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/integration/SaslPlaintextTopicMetadataTest.scala create mode 100644 core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/SaslPlaintextReplicaFetchTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala index 69b2958b5cde1..15b91dd9df385 100644 --- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala @@ -61,7 +61,7 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging { val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() def generateConfigs() = - TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) + TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSsl=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) val topic = "topic" val part = 0 diff --git a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala index 967c9f443a612..aef1842c39485 100644 --- a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala @@ -43,7 +43,7 @@ class SSLProducerSendTest extends KafkaServerTestHarness { overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) def generateConfigs() = - TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSSL=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) + TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSsl=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) private var consumer1: SimpleConsumer = null private var consumer2: SimpleConsumer = null diff --git a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala index afa6db50f0b3b..c908b890a2718 100644 --- a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala @@ -29,7 +29,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.SecurityProtocol -import kafka.integration.KafkaServerTestHarness import kafka.utils.{TestUtils, Logging} import kafka.server.{KafkaConfig, KafkaServer} @@ -73,7 +72,7 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { @Before override def setUp() { super.setUp() - val props = TestUtils.createBrokerConfig(numServers, zkConnect, false, enableSasl=true) + val props = TestUtils.createBrokerConfig(numServers, zkConnect, false, enableSaslPlaintext=true) val config = KafkaConfig.fromProps(props, overridingProps) servers = Buffer(TestUtils.createServer(config)) diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 27dce9f04eb24..6dce776a3efa8 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -18,6 +18,7 @@ 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 { @@ -27,6 +28,8 @@ trait SaslTestHarness extends ZooKeeperTestHarness { @Before override def setUp() { + // Clean-up global configuration set by other tests + Configuration.setConfiguration(null) val keytabFile = TestUtils.tempFile() val jaasFile = TestUtils.tempFile() @@ -43,7 +46,7 @@ trait SaslTestHarness extends ZooKeeperTestHarness { kdc.start() kdc.createPrincipal(keytabFile, "client", "kafka/localhost") - System.setProperty("java.security.auth.login.config", jaasFile.getAbsolutePath) + System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile.getAbsolutePath) super.setUp } @@ -51,7 +54,7 @@ trait SaslTestHarness extends ZooKeeperTestHarness { override def tearDown() { super.tearDown kdc.stop() - System.clearProperty("java.security.auth.login.config") + System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) Configuration.setConfiguration(null) } } diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala index 20a4068228395..4430da7a08172 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/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..9174637d26e4c --- /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 def 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..f010d45bdb31e 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 securityProtocol = SecurityProtocol.SSL protected def 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 b744b94cf4e5a..41698d8fcf7da 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/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..7291967821215 --- /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 def 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..62ca702878939 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 securityProtocol = SecurityProtocol.SSL protected def 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 78c12eb1aa34d..dba1ba79e8278 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -137,17 +137,25 @@ 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, - enableSasl: Boolean = false, - trustStoreFile: Option[File] = None): Seq[Properties] = { - (0 until numConfigs).map(node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, - enableSSL = enableSSL, enableSasl = enableSasl, trustStoreFile = trustStoreFile)) + 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 getBrokerListStrFromServers(servers: Seq[KafkaServer], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): String = { @@ -155,20 +163,38 @@ 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 createBrokerConfig(nodeId: Int, zkConnect: String, enableControlledShutdown: Boolean = true, enableDeleteTopic: Boolean = false, - port: Int = RandomPort, enableSasl: Boolean = false, saslPort: 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 sslEnabled = enableSsl || shouldEnable(SecurityProtocol.SSL) + val saslSslEnabled = enableSaslSsl || shouldEnable(SecurityProtocol.SASL_SSL) val listeners = { - val protocolAndPorts = ArrayBuffer("PLAINTEXT" -> port) - if (enableSSL) + + val protocolAndPorts = ArrayBuffer[(String, Int)]() + if (enablePlaintext || shouldEnable(SecurityProtocol.PLAINTEXT)) + protocolAndPorts += ("PLAINTEXT" -> port) + if (sslEnabled) protocolAndPorts += "SSL" -> sslPort - if (enableSasl) - protocolAndPorts += "SASL_PLAINTEXT" -> saslPort + if (enableSaslPlaintext || shouldEnable(SecurityProtocol.SASL_PLAINTEXT)) + protocolAndPorts += "SASL_PLAINTEXT" -> saslPlaintextPort + if (saslSslEnabled) + protocolAndPorts += "SASL_SSL" -> saslSslPort protocolAndPorts.map { case (protocol, port) => s"$protocol://localhost:$port" }.mkString(",") @@ -184,8 +210,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) + + if (sslEnabled || saslSslEnabled) props.putAll(addSSLConfigs(Mode.SERVER, true, trustStoreFile, s"server$nodeId")) + + interBrokerSecurityProtocol.foreach { protocol => + props.put(KafkaConfig.InterBrokerSecurityProtocolProp, protocol.name) + } + props.put("port", port.toString) props } @@ -923,19 +955,18 @@ object TestUtils extends Logging { new String(bytes, encoding) } - def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { - if (!trustStoreFile.isDefined) { + def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { + + val trustStore = trustStoreFile.getOrElse { throw new Exception("enableSSL set to true but no trustStoreFile provided") } + val sslConfigs = { - if (mode == 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() From cffc5e8c6634e2ef8c3d8238a5e8bee508b4aa18 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 16 Oct 2015 16:34:40 +0100 Subject: [PATCH 42/50] Refactor `ProducerSendTest` to support multiple security protocols and fix bug in `ClientUtils.createChannelBuilder` --- .../org/apache/kafka/clients/ClientUtils.java | 4 +- ...dTest.scala => BaseProducerSendTest.scala} | 91 +++---- .../kafka/api/PlaintextProducerSendTest.scala | 57 +++++ .../kafka/api/SSLConsumerTest.scala | 12 +- .../kafka/api/SSLProducerSendTest.scala | 241 ------------------ .../kafka/api/SaslIntegrationTest.scala | 4 +- .../api/SaslPlaintextProducerSendTest.scala | 25 ++ .../kafka/api/SaslSslProducerSendTest.scala | 27 ++ .../kafka/api/SslProducerSendTest.scala | 27 ++ .../integration/KafkaServerTestHarness.scala | 5 +- .../SaslSslTopicMetadataTest.scala | 2 +- .../integration/SslTopicMetadataTest.scala | 2 +- .../server/SaslSslReplicaFetchTest.scala | 2 +- .../kafka/server/SslReplicaFetchTest.scala | 2 +- .../scala/unit/kafka/utils/TestUtils.scala | 89 +++---- 15 files changed, 230 insertions(+), 360 deletions(-) rename core/src/test/scala/integration/kafka/api/{ProducerSendTest.scala => BaseProducerSendTest.scala} (82%) create mode 100644 core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala delete mode 100644 core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala 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 6ed6a08a91137..41e5d74fc673c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java @@ -75,8 +75,8 @@ 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 && securityProtocol != SecurityProtocol.SASL_PLAINTEXT) - throw new ConfigException("Invalid SecurityProtocol " + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG); + if (securityProtocol == SecurityProtocol.TRACE) + throw new ConfigException("Invalid SecurityProtocol " + securityProtocol); return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs); } 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 637d6f3157a2c..11a880032fc33 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,21 @@ 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) + protected def trustStoreFile: Option[File] - 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 +66,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 +78,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 +133,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 +165,7 @@ class ProducerSendTest extends KafkaServerTestHarness { */ @Test def testClose() { - var producer = TestUtils.createNewProducer(brokerList) + val producer = createProducer(brokerList) try { // create topic @@ -204,7 +179,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 +186,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 +198,7 @@ class ProducerSendTest extends KafkaServerTestHarness { */ @Test def testSendToPartition() { - var producer = TestUtils.createNewProducer(brokerList) + val producer = createProducer(brokerList) try { // create topic @@ -241,7 +213,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 +239,8 @@ class ProducerSendTest extends KafkaServerTestHarness { assertEquals(i.toLong, messageSet1(i).offset) } } finally { - if (producer != null) { + if (producer != null) producer.close() - producer = null - } } } @@ -281,7 +251,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 +264,6 @@ class ProducerSendTest extends KafkaServerTestHarness { } finally { if (producer != null) { producer.close() - producer = null } } } @@ -304,7 +273,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(zkClient, topic, 2, 2, servers) val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, "value".getBytes) @@ -338,7 +307,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 +360,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/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala new file mode 100644 index 0000000000000..e3341cd00a621 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -0,0 +1,57 @@ +/** + * 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.protocol.SecurityProtocol +import org.apache.kafka.common.serialization.ByteArraySerializer +import org.junit.Test + +class PlaintextProducerSendTest extends BaseProducerSendTest { + override protected def securityProtocol = SecurityProtocol.PLAINTEXT + protected def trustStoreFile = None + + @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 index 15b91dd9df385..ed413d7d7f513 100644 --- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala @@ -61,7 +61,7 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging { val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() def generateConfigs() = - TestUtils.createBrokerConfigs(numServers, zkConnect, false, enableSsl=true, trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) + TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), trustStoreFile=Some(trustStoreFile)).map(KafkaConfig.fromProps(_, overridingProps)) val topic = "topic" val part = 0 @@ -73,6 +73,8 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") + override protected def securityProtocol: SecurityProtocol = SecurityProtocol.SSL + @Before override def setUp() { super.setUp() @@ -87,14 +89,14 @@ class SSLConsumerTest extends KafkaServerTestHarness with Logging { for (i <- 0 until producerCount) producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), acks = 1, - enableSSL=true, - trustStoreFile=Some(trustStoreFile)) + securityProtocol = SecurityProtocol.SSL, + trustStoreFile = Some(trustStoreFile)) for (i <- 0 until consumerCount) consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), groupId = "my-test", partitionAssignmentStrategy= "range", - enableSSL=true, - trustStoreFile=Some(trustStoreFile)) + securityProtocol = SecurityProtocol.SSL, + trustStoreFile = Some(trustStoreFile)) // create the consumer offset topic 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 aef1842c39485..0000000000000 --- a/core/src/test/scala/integration/kafka/api/SSLProducerSendTest.scala +++ /dev/null @@ -1,241 +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.apache.kafka.common.protocol.SecurityProtocol -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.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), 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(zkClient, 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.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), enableSSL=true, trustStoreFile=Some(trustStoreFile)) - try { - // create topic - TestUtils.createTopic(zkClient, 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.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), enableSSL=true, trustStoreFile=Some(trustStoreFile)) - try { - // create topic - val leaders = TestUtils.createTopic(zkClient, 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/SaslIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala index c908b890a2718..3e79ec4b3c951 100644 --- a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala @@ -83,12 +83,12 @@ class SaslIntegrationTest extends SaslTestHarness with Logging { for (i <- 0 until producerCount) producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), acks = 1, - enableSasl=true) + securityProtocol = SecurityProtocol.SASL_PLAINTEXT) for (i <- 0 until consumerCount) consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), groupId = "my-test", partitionAssignmentStrategy= "range", - enableSasl=true) + securityProtocol = SecurityProtocol.SASL_PLAINTEXT) // create the consumer offset topic diff --git a/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala new file mode 100644 index 0000000000000..2ff2b02fcbc9f --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala @@ -0,0 +1,25 @@ +/** + * 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 SaslPlaintextProducerSendTest extends BaseProducerSendTest with SaslTestHarness { + override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT + protected def trustStoreFile = None +} diff --git a/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala new file mode 100644 index 0000000000000..9fed745553f4a --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.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 SaslSslProducerSendTest extends BaseProducerSendTest with SaslTestHarness { + override protected def securityProtocol = SecurityProtocol.SASL_SSL + 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..b4b4ff0b481af --- /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 + protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) +} diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index bca0dccf049f8..369627c9a103b 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -23,6 +23,7 @@ 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 +53,15 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { def bootstrapUrl = servers.map(s => s.config.hostName + ":" + s.boundPort()).mkString(",") + protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT + @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/SaslSslTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala index 9174637d26e4c..ea1541911fd83 100644 --- a/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/SaslSslTopicMetadataTest.scala @@ -24,5 +24,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SaslSslTopicMetadataTest extends BaseTopicMetadataTest with SaslTestHarness { protected def securityProtocol = SecurityProtocol.SASL_SSL - protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + 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 f010d45bdb31e..ee734573a8cff 100644 --- a/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/SslTopicMetadataTest.scala @@ -23,5 +23,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SslTopicMetadataTest extends BaseTopicMetadataTest { protected def securityProtocol = SecurityProtocol.SSL - protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) } diff --git a/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala index 7291967821215..1bcf8ac7d64e0 100644 --- a/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslSslReplicaFetchTest.scala @@ -24,5 +24,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SaslSslReplicaFetchTest extends BaseReplicaFetchTest with SaslTestHarness { protected def securityProtocol = SecurityProtocol.SASL_SSL - protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + 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 62ca702878939..dad22851186ef 100644 --- a/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/SslReplicaFetchTest.scala @@ -23,5 +23,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SslReplicaFetchTest extends BaseReplicaFetchTest { protected def securityProtocol = SecurityProtocol.SSL - protected def trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + 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 dba1ba79e8278..cc575247a2b83 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -181,24 +181,19 @@ object TestUtils extends Logging { def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol) - val sslEnabled = enableSsl || shouldEnable(SecurityProtocol.SSL) - val saslSslEnabled = enableSaslSsl || shouldEnable(SecurityProtocol.SASL_SSL) - - val listeners = { - - val protocolAndPorts = ArrayBuffer[(String, Int)]() - if (enablePlaintext || shouldEnable(SecurityProtocol.PLAINTEXT)) - protocolAndPorts += ("PLAINTEXT" -> port) - if (sslEnabled) - protocolAndPorts += "SSL" -> sslPort - if (enableSaslPlaintext || shouldEnable(SecurityProtocol.SASL_PLAINTEXT)) - protocolAndPorts += "SASL_PLAINTEXT" -> saslPlaintextPort - if (saslSslEnabled) - protocolAndPorts += "SASL_SSL" -> saslSslPort - protocolAndPorts.map { case (protocol, port) => - s"$protocol://localhost:$port" - }.mkString(",") - } + 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 if (nodeId >= 0) props.put("broker.id", nodeId.toString) @@ -211,8 +206,8 @@ object TestUtils extends Logging { props.put("delete.topic.enable", enableDeleteTopic.toString) props.put("controlled.shutdown.retry.backoff.ms", "100") - if (sslEnabled || saslSslEnabled) - props.putAll(addSSLConfigs(Mode.SERVER, true, trustStoreFile, s"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) @@ -443,31 +438,41 @@ object TestUtils extends Logging { bufferSize: Long = 1024L * 1024L, retries: Int = 0, lingerMs: Long = 0, - enableSasl: Boolean = false, - 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(Mode.CLIENT, false, trustStoreFile, "producer")) - } else if (enableSasl) { - producerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT") + + /* 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. */ @@ -477,8 +482,7 @@ object TestUtils extends Logging { partitionFetchSize: Long = 4096L, partitionAssignmentStrategy: String = "blah", sessionTimeout: Int = 30000, - enableSasl: Boolean = false, - enableSSL: Boolean = false, + securityProtocol: SecurityProtocol, trustStoreFile: Option[File] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.consumer.ConsumerConfig @@ -493,12 +497,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(Mode.CLIENT, false, trustStoreFile, "consumer")) - } else if (enableSasl) { - consumerProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SASL_PLAINTEXT") - } + 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) } @@ -955,10 +956,10 @@ object TestUtils extends Logging { new String(bytes, encoding) } - def addSSLConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { + def sslConfigs(mode: Mode, clientCert: Boolean, trustStoreFile: Option[File], certAlias: String): Properties = { val trustStore = trustStoreFile.getOrElse { - throw new Exception("enableSSL set to true but no trustStoreFile provided") + throw new Exception("SSL enabled but no trustStoreFile provided") } From 699da1a9de60b9d647c94280ee6d4975fe1b6475 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 08:32:50 +0100 Subject: [PATCH 43/50] Refactor `ConsumerTest` to make it easy to test various security protocols Include improvements from `SaslIntegrationTest` and remove duplicated code. --- ...sumerTest.scala => BaseConsumerTest.scala} | 44 ++-- .../kafka/api/BaseProducerSendTest.scala | 2 - .../kafka/api/IntegrationTestHarness.scala | 9 +- .../kafka/api/PlaintextConsumerTest.scala | 15 ++ .../kafka/api/PlaintextProducerSendTest.scala | 3 - .../kafka/api/SSLConsumerTest.scala | 218 +----------------- .../kafka/api/SaslIntegrationTest.scala | 154 ------------- .../kafka/api/SaslPlaintextConsumerTest.scala | 19 ++ .../api/SaslPlaintextProducerSendTest.scala | 1 - .../kafka/api/SaslSslConsumerTest.scala | 22 ++ .../kafka/api/SaslSslProducerSendTest.scala | 2 +- .../kafka/api/SslProducerSendTest.scala | 2 +- .../integration/KafkaServerTestHarness.scala | 2 + 13 files changed, 91 insertions(+), 402 deletions(-) rename core/src/test/scala/integration/kafka/api/{ConsumerTest.scala => BaseConsumerTest.scala} (94%) create mode 100644 core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala delete mode 100644 core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala create mode 100644 core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala 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 a64c2f3e53492..393292d874c2a 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/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 11a880032fc33..9ca8a6545f263 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -33,8 +33,6 @@ import org.junit.{After, Before, Test} abstract class BaseProducerSendTest extends KafkaServerTestHarness { - protected def trustStoreFile: Option[File] - def generateConfigs = { val overridingProps = new Properties() val numServers = 2 diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index 8080b085a38db..a55a1bbda96c7 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 index e3341cd00a621..d017d13d2cc43 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -21,13 +21,10 @@ import java.util.Properties import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer} import org.apache.kafka.common.config.ConfigException -import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.Test class PlaintextProducerSendTest extends BaseProducerSendTest { - override protected def securityProtocol = SecurityProtocol.PLAINTEXT - protected def trustStoreFile = None @Test def testSerializerConstructors() { diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala index ed413d7d7f513..1d13d88c311ff 100644 --- a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala @@ -12,223 +12,11 @@ */ 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 org.apache.kafka.common.protocol.SecurityProtocol -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, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), 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") - - override protected def securityProtocol: SecurityProtocol = SecurityProtocol.SSL - - @Before - override def setUp() { - super.setUp() - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL)) - 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.getBrokerListStrFromServers(servers, SecurityProtocol.SSL)) - 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.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), - acks = 1, - securityProtocol = SecurityProtocol.SSL, - trustStoreFile = Some(trustStoreFile)) - for (i <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SSL), - groupId = "my-test", - partitionAssignmentStrategy= "range", - securityProtocol = SecurityProtocol.SSL, - trustStoreFile = Some(trustStoreFile)) - - - // create the consumer offset topic - TestUtils.createTopic(zkClient, 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(this.zkClient, 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(this.zkClient, "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()) - } - } +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/SaslIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala deleted file mode 100644 index 3e79ec4b3c951..0000000000000 --- a/core/src/test/scala/integration/kafka/api/SaslIntegrationTest.scala +++ /dev/null @@ -1,154 +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 org.apache.kafka.common.protocol.SecurityProtocol - -import kafka.utils.{TestUtils, Logging} -import kafka.server.{KafkaConfig, KafkaServer} - -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 - -class SaslIntegrationTest extends SaslTestHarness with Logging { - val brokerId1 = 0 - var servers: Buffer[KafkaServer] = null - val numServers = 1 - val producerCount = 1 - val consumerCount = 1 - 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.OffsetsTopicPartitionsProp, "1") - overridingProps.put(KafkaConfig.ConsumerMinSessionTimeoutMsProp, "100") // set small enough session timeout - - var consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() - var producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() - - 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() - val props = TestUtils.createBrokerConfig(numServers, zkConnect, false, enableSaslPlaintext=true) - val config = KafkaConfig.fromProps(props, overridingProps) - servers = Buffer(TestUtils.createServer(config)) - - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) - consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT)) - consumerConfig.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range") - - for (i <- 0 until producerCount) - producers += TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), - acks = 1, - securityProtocol = SecurityProtocol.SASL_PLAINTEXT) - for (i <- 0 until consumerCount) - consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers, SecurityProtocol.SASL_PLAINTEXT), - groupId = "my-test", - partitionAssignmentStrategy= "range", - securityProtocol = SecurityProtocol.SASL_PLAINTEXT) - - - // create the consumer offset topic - TestUtils.createTopic(zkClient, ConsumerCoordinator.OffsetsTopicName, - overridingProps.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt, - 1, - servers, - servers(0).consumerCoordinator.offsetsTopicConfigs) - - // create the test topic with all the brokers as replicas - TestUtils.createTopic(this.zkClient, 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) - consumeAndVerifyRecords(this.consumers(0), numRecords = numRecords, startingOffset = 0) - } - - private def sendRecords(numRecords: Int) { - val futures = (0 until numRecords).map { i => - this.producers(0).send(new ProducerRecord(topic, part, ("key" + i).getBytes, ("value" + i).getBytes)) - } - futures.map(_.get) - } - - private def consumeAndVerifyRecords(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()) - assertEquals("key" + i, new String(record.key())) - assertEquals("value" + i, new String(record.value())) - } - } - -} 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/SaslPlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala index 2ff2b02fcbc9f..dfd06dd06ddfc 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala @@ -21,5 +21,4 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SaslPlaintextProducerSendTest extends BaseProducerSendTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT - protected def trustStoreFile = None } 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/SaslSslProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala index 9fed745553f4a..bca557c678bb8 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala @@ -23,5 +23,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SaslSslProducerSendTest extends BaseProducerSendTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_SSL - protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + 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 index b4b4ff0b481af..4d9189c065c61 100644 --- a/core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslProducerSendTest.scala @@ -23,5 +23,5 @@ import org.apache.kafka.common.protocol.SecurityProtocol class SslProducerSendTest extends BaseProducerSendTest { override protected def securityProtocol = SecurityProtocol.SSL - protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) } diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 369627c9a103b..26b86f751a1bd 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -17,6 +17,7 @@ package kafka.integration +import java.io.File import java.util.Arrays import kafka.common.KafkaException @@ -54,6 +55,7 @@ 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() { From 47fee8548759729f7d90ec57e953af73d216cb5f Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 08:49:16 +0100 Subject: [PATCH 44/50] Fix issue where interestOps was not being turned off when it should As pointed out by Jun --- .../SaslClientAuthenticator.java | 19 ++++++++++++------- .../SaslServerAuthenticator.java | 18 +++++++++++------- 2 files changed, 23 insertions(+), 14 deletions(-) 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 index 646af199950c0..099a1164f39a8 100644 --- 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 @@ -116,10 +116,9 @@ public SaslClient run() throws SaslException { public void authenticate() throws IOException { - if (netOutBuffer != null && !flushNetOutBuffer()) { - transportLayer.addInterestOps(SelectionKey.OP_WRITE); + if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps()) return; - } + switch (saslState) { case INITIAL: sendSaslToken(new byte[0]); @@ -153,10 +152,7 @@ private void sendSaslToken(byte[] serverToken) throws IOException { byte[] saslToken = createSaslToken(serverToken); if (saslToken != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken)); - if (flushNetOutBuffer()) - transportLayer.removeInterestOps(SelectionKey.OP_WRITE); - else - transportLayer.addInterestOps(SelectionKey.OP_WRITE); + flushNetOutBufferAndUpdateInterestOps(); } } catch (SaslException se) { saslState = SaslState.FAILED; @@ -165,6 +161,15 @@ private void sendSaslToken(byte[] serverToken) throws IOException { } } + 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); } 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 index fe07f08dccf0a..657df0aca24b1 100644 --- 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 @@ -138,10 +138,8 @@ public SaslServer run() { } public void authenticate() throws IOException { - if (netOutBuffer != null && !flushNetOutBuffer()) { - transportLayer.addInterestOps(SelectionKey.OP_WRITE); + if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps()) return; - } if (saslServer.isComplete()) { transportLayer.removeInterestOps(SelectionKey.OP_WRITE); @@ -161,10 +159,7 @@ public void authenticate() throws IOException { byte[] response = saslServer.evaluateResponse(clientToken); if (response != null) { netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); - if (flushNetOutBuffer()) - transportLayer.removeInterestOps(SelectionKey.OP_WRITE); - else - transportLayer.addInterestOps(SelectionKey.OP_WRITE); + flushNetOutBufferAndUpdateInterestOps(); } } catch (Exception e) { throw new IOException(e); @@ -184,6 +179,15 @@ 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); From a2a7b88937885d00dd5662a8089dd280f47e6edf Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 09:21:22 +0100 Subject: [PATCH 45/50] Document `authenticate` in `SaslClientAuthenticator` and `SaslServerAuthenticator` --- .../security/authenticator/SaslClientAuthenticator.java | 8 +++++++- .../security/authenticator/SaslServerAuthenticator.java | 7 +++++++ 2 files changed, 14 insertions(+), 1 deletion(-) 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 index 099a1164f39a8..39291607886f7 100644 --- 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 @@ -114,7 +114,13 @@ public SaslClient run() throws SaslException { } } - + /** + * 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; 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 index 657df0aca24b1..d06a22a192139 100644 --- 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 @@ -137,6 +137,13 @@ public SaslServer run() { } } + /** + * 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; From cdb8b86424dee739f4f066ec5ba8a46b2f7e7912 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 12:30:52 +0100 Subject: [PATCH 46/50] Make it possible to configure serviceName via KafkaConfig --- .../clients/consumer/ConsumerConfig.java | 1 + .../clients/producer/ProducerConfig.java | 1 + .../kafka/common/config/SaslConfigs.java | 4 ++ .../security/kerberos/LoginManager.java | 20 +++++++- .../main/scala/kafka/server/KafkaConfig.scala | 51 +++++++++++-------- .../unit/kafka/server/KafkaConfigTest.scala | 1 + 6 files changed, 56 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index bfbb47a2998e8..1dc23a2890cbc 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 @@ -303,6 +303,7 @@ 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) 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 bad830beca803..f9f2d7f98febd 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 @@ -280,6 +280,7 @@ 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) 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 index c4d9e4866dd57..0abefe715b3ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -25,6 +25,10 @@ public class SaslConfigs { 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"; 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 index 66d3169fec9e4..9382aa035e2ff 100644 --- 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 @@ -24,6 +24,7 @@ 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; @@ -39,11 +40,28 @@ public class LoginManager { private LoginManager(LoginType loginType, Map configs) throws IOException, LoginException { this.loginType = loginType; String loginContext = loginType.contextName(); - this.serviceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); + this.serviceName = getServiceName(loginContext, configs); login = new Login(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. * diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 95949d4371484..6b90f95afffd3 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -330,7 +330,8 @@ object KafkaConfig { val SSLEndpointIdentificationAlgorithmProp = SSLConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG val SSLClientAuthProp = SSLConfigs.SSL_CLIENT_AUTH_CONFIG - /** ********* SSL Configuration ****************/ + /** ********* 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 @@ -508,6 +509,7 @@ object KafkaConfig { 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 @@ -671,6 +673,7 @@ object KafkaConfig { .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) @@ -838,6 +841,7 @@ case class KafkaConfig (props: java.util.Map[_, _]) extends AbstractConfig(Kafka 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) @@ -865,7 +869,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 }) @@ -969,25 +973,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) - channelConfigs.put(SaslKerberosKinitCmdProp, saslKerberosKinitCmd) - channelConfigs.put(SaslKerberosTicketRenewWindowFactorProp, saslKerberosTicketRenewWindowFactor) - channelConfigs.put(SaslKerberosTicketRenewJitterProp, saslKerberosTicketRenewJitter) - channelConfigs.put(SaslKerberosMinTimeBeforeReloginProp, saslKerberosMinTimeBeforeRelogin) - channelConfigs.put(AuthToLocalProp, authToLocal) + 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/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 17d66a47529e0..d8f8f8bb19cef 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -508,6 +508,7 @@ class KafkaConfigTest { case KafkaConfig.SSLCipherSuitesProp => // ignore string //Sasl Configs + case KafkaConfig.SaslKerberosServiceNameProp => // ignore string case KafkaConfig.SaslKerberosKinitCmdProp => case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp => case KafkaConfig.SaslKerberosTicketRenewJitterProp => From 15cf778518575918ebfbd64e6f83803a2c6d3618 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 13:38:54 +0100 Subject: [PATCH 47/50] Rename `SSLConsumerTest` to `SslConsumerTest` --- .../kafka/api/{SSLConsumerTest.scala => SslConsumerTest.scala} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename core/src/test/scala/integration/kafka/api/{SSLConsumerTest.scala => SslConsumerTest.scala} (100%) diff --git a/core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SslConsumerTest.scala similarity index 100% rename from core/src/test/scala/integration/kafka/api/SSLConsumerTest.scala rename to core/src/test/scala/integration/kafka/api/SslConsumerTest.scala From b6f408a0916771c92b5c1e22e88af3ff1ef9d5c9 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 19 Oct 2015 13:53:16 +0100 Subject: [PATCH 48/50] Document why we exclude `api-ldap-schema-data` --- build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index 8a235b1f428ed..cea945e87a332 100644 --- a/build.gradle +++ b/build.gradle @@ -282,6 +282,8 @@ 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' } From b74a5d29cfc91c8af9304e54b5a34f210fc691cb Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 20 Oct 2015 16:42:44 +0100 Subject: [PATCH 49/50] Force reload of JAAS configuration file if we can't find our entry This seems to be necessary for our tests to pass consistently. --- .../kafka/common/security/kerberos/Login.java | 20 +++++++++++++++++-- .../security/kerberos/LoginManager.java | 2 +- .../kafka/api/SaslTestHarness.scala | 3 +++ 3 files changed, 22 insertions(+), 3 deletions(-) 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 index 8f9c9ca87dabc..dd885e542f555 100644 --- 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 @@ -110,8 +110,8 @@ public Login(final String loginContextName, Map configs) throws Login isUsingTicketCache = false; principal = null; } else { + // there will only be a single entry AppConfigurationEntry entry = entries[0]; - // there will only be a single entry, so this for() loop will only be iterated through once. if (entry.getOptions().get("useTicketCache") != null) { String val = (String) entry.getOptions().get("useTicketCache"); isUsingTicketCache = val.equals("true"); @@ -284,10 +284,26 @@ public Subject subject() { } private synchronized LoginContext login(final String loginContextName) throws LoginException { - if (System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) == null) { + 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."); 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 index 9382aa035e2ff..18651c8611e0e 100644 --- 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 @@ -40,8 +40,8 @@ public class LoginManager { private LoginManager(LoginType loginType, Map configs) throws IOException, LoginException { this.loginType = loginType; String loginContext = loginType.contextName(); - this.serviceName = getServiceName(loginContext, configs); login = new Login(loginContext, configs); + this.serviceName = getServiceName(loginContext, configs); login.startThreadIfNeeded(); } diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 6dce776a3efa8..9575fdae6afc1 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -36,6 +36,9 @@ trait SaslTestHarness extends ZooKeeperTestHarness { 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) From 9464c7a7b447ecce330b21dd683d59ae9c885150 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 20 Oct 2015 17:55:57 +0100 Subject: [PATCH 50/50] Remove `Sasl*ProducerTest` to keep build times down `Sasl*ConsumerTest` already tests similar scenarios. --- .../api/SaslPlaintextProducerSendTest.scala | 24 ----------------- .../kafka/api/SaslSslProducerSendTest.scala | 27 ------------------- 2 files changed, 51 deletions(-) delete mode 100644 core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala delete mode 100644 core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala diff --git a/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala deleted file mode 100644 index dfd06dd06ddfc..0000000000000 --- a/core/src/test/scala/integration/kafka/api/SaslPlaintextProducerSendTest.scala +++ /dev/null @@ -1,24 +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 org.apache.kafka.common.protocol.SecurityProtocol - -class SaslPlaintextProducerSendTest extends BaseProducerSendTest with SaslTestHarness { - override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT -} diff --git a/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala deleted file mode 100644 index bca557c678bb8..0000000000000 --- a/core/src/test/scala/integration/kafka/api/SaslSslProducerSendTest.scala +++ /dev/null @@ -1,27 +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.io.File - -import org.apache.kafka.common.protocol.SecurityProtocol - -class SaslSslProducerSendTest extends BaseProducerSendTest with SaslTestHarness { - override protected def securityProtocol = SecurityProtocol.SASL_SSL - override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) -}