Repository: kafka
Updated Branches:
  refs/heads/trunk 2eb32d6a7 -> 8d9d84790


KAFKA-4581; Fail early if multiple client login modules in sasl.jaas.config

Validate and fail client connection if multiple login modules are specified in 
sasl.jaas.config to avoid harder-to-debug authentication failures later on.

Author: Rajini Sivaram <rajinisiva...@googlemail.com>

Reviewers: Ismael Juma <ism...@juma.me.uk>

Closes #2356 from rajinisivaram/KAFKA-4581


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/8d9d8479
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/8d9d8479
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/8d9d8479

Branch: refs/heads/trunk
Commit: 8d9d847904e28759b44ffc87752802e8e6f1f659
Parents: 2eb32d6
Author: Rajini Sivaram <rajinisiva...@googlemail.com>
Authored: Fri Jan 13 10:34:15 2017 +0000
Committer: Ismael Juma <ism...@juma.me.uk>
Committed: Fri Jan 13 10:34:15 2017 +0000

----------------------------------------------------------------------
 .../org/apache/kafka/common/security/JaasUtils.java    | 10 ++++++++--
 .../apache/kafka/common/security/JaasUtilsTest.java    |  4 +---
 .../security/authenticator/SaslAuthenticatorTest.java  | 13 +++++++++++++
 3 files changed, 22 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/8d9d8479/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java
----------------------------------------------------------------------
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 10e591d..e326156 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
@@ -55,8 +55,14 @@ public class JaasUtils {
         if (jaasConfigArgs != null) {
             if (loginType == LoginType.SERVER)
                 throw new IllegalArgumentException("JAAS config property not 
supported for server");
-            else
-                return new JaasConfig(loginType, jaasConfigArgs.value());
+            else {
+                JaasConfig jaasConfig = new JaasConfig(loginType, 
jaasConfigArgs.value());
+                AppConfigurationEntry[] clientModules = 
jaasConfig.getAppConfigurationEntry(LoginType.CLIENT.contextName());
+                int numModules = clientModules == null ? 0 : 
clientModules.length;
+                if (numModules != 1)
+                    throw new IllegalArgumentException("JAAS config property 
contains " + numModules + " login modules, should be one module");
+                return jaasConfig;
+            }
         } else
             return defaultJaasConfig(loginType);
     }

http://git-wip-us.apache.org/repos/asf/kafka/blob/8d9d8479/clients/src/test/java/org/apache/kafka/common/security/JaasUtilsTest.java
----------------------------------------------------------------------
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/JaasUtilsTest.java 
b/clients/src/test/java/org/apache/kafka/common/security/JaasUtilsTest.java
index 443393f..10ec390 100644
--- a/clients/src/test/java/org/apache/kafka/common/security/JaasUtilsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/security/JaasUtilsTest.java
@@ -129,9 +129,7 @@ public class JaasUtilsTest {
         }
         String jaasConfigProp = builder.toString();
 
-        Map<String, Object> configs = new HashMap<>();
-        configs.put(SaslConfigs.SASL_JAAS_CONFIG, new 
Password(jaasConfigProp));
-        Configuration configuration = JaasUtils.jaasConfig(LoginType.CLIENT, 
configs);
+        Configuration configuration = new JaasConfig(LoginType.CLIENT, 
jaasConfigProp);
         AppConfigurationEntry[] dynamicEntries = 
configuration.getAppConfigurationEntry(LoginType.CLIENT.contextName());
         assertEquals(moduleCount, dynamicEntries.length);
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/8d9d8479/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
----------------------------------------------------------------------
diff --git 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
index b515ef4..ac9beb4 100644
--- 
a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
+++ 
b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java
@@ -15,6 +15,7 @@ package org.apache.kafka.common.security.authenticator;
 import org.apache.kafka.clients.NetworkClient;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.config.types.Password;
 import org.apache.kafka.common.network.CertStores;
 import org.apache.kafka.common.network.ChannelBuilder;
 import org.apache.kafka.common.network.ChannelBuilders;
@@ -62,6 +63,7 @@ import java.util.Random;
 import javax.security.auth.login.Configuration;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 /**
@@ -659,6 +661,17 @@ public class SaslAuthenticatorTest {
         // Check that another user 'user2' can also connect with a Jaas config 
override without any changes to static configuration
         saslClientConfigs.put(SaslConfigs.SASL_JAAS_CONFIG, 
TestJaasConfig.jaasConfigProperty("PLAIN", "user2", "user2-secret"));
         createAndCheckClientConnection(securityProtocol, "4");
+
+        // Check that clients specifying multiple login modules fail even if 
the credentials are valid
+        String module1 = TestJaasConfig.jaasConfigProperty("PLAIN", "user1", 
"user1-secret").value();
+        String module2 = TestJaasConfig.jaasConfigProperty("PLAIN", "user2", 
"user2-secret").value();
+        saslClientConfigs.put(SaslConfigs.SASL_JAAS_CONFIG, new 
Password(module1 + " " + module2));
+        try {
+            createClientConnection(securityProtocol, "1");
+            fail("Connection created with multiple login modules in 
sasl.jaas.config");
+        } catch (KafkaException e) {
+            assertTrue("Unexpected exception " + e, e.getCause() instanceof 
IllegalArgumentException);
+        }
     }
 
     /**

Reply via email to