Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-17921: Support SASL_PLAINTEXT protocol with java.security.auth.login.config #17671

Open
wants to merge 1 commit into
base: trunk
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions checkstyle/import-control-test-common.xml
Original file line number Diff line number Diff line change
Expand Up @@ -24,4 +24,5 @@
<allow pkg="org" />
<allow pkg="kafka" />
<allow pkg="scala.jdk.javaapi" />
<allow pkg="javax.security" />
</import-control>
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.test;

import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;

public record JaasModule(String name, boolean debug, Map<String, String> entries) {

public static JaasModule plainLoginModule(String username, String password, boolean debug, Map<String, String> validUsers) {
String name = "org.apache.kafka.common.security.plain.PlainLoginModule";

Map<String, String> entries = new HashMap<>();
entries.put("username", username);
entries.put("password", password);
validUsers.forEach((user, pass) -> entries.put("user_" + user, pass));

return new JaasModule(
name,
debug,
entries
);
}

@Override
public String toString() {
return String.format("%s required%n debug=%b%n %s;%n", name, debug, entries.entrySet().stream()
.map(e -> e.getKey() + "=\"" + e.getValue() + "\"")
.collect(Collectors.joining("\n ")));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.common.test;

import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.io.OutputStreamWriter;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;

import javax.security.auth.login.Configuration;

public class JaasUtils {
public record JaasSection(String contextName, List<JaasModule> modules) {
@Override
public String toString() {
return String.format(
"%s {%n %s%n};%n",
contextName,
modules.stream().map(Object::toString).collect(Collectors.joining("\n "))
);
}
}

public static final String KAFKA_SERVER_CONTEXT_NAME = "KafkaServer";

public static final String KAFKA_PLAIN_USER1 = "plain-user1";
public static final String KAFKA_PLAIN_USER1_PASSWORD = "plain-user1-secret";
public static final String KAFKA_PLAIN_ADMIN = "plain-admin";
public static final String KAFKA_PLAIN_ADMIN_PASSWORD = "plain-admin-secret";

public static File writeJaasContextsToFile(Map<String, JaasSection> jaasSections) throws IOException {
File jaasFile = TestUtils.tempFile();
try (FileOutputStream fileStream = new FileOutputStream(jaasFile);
OutputStreamWriter writer = new OutputStreamWriter(fileStream, StandardCharsets.UTF_8);) {
writer.write(String.join("", jaasSections.values().stream().map(Object::toString).toArray(String[]::new)));
}
return jaasFile;
}

public static void refreshJavaLoginConfigParam(File file) {
System.setProperty(org.apache.kafka.common.security.JaasUtils.JAVA_LOGIN_CONFIG_PARAM, file.getAbsolutePath());
// This will cause a reload of the Configuration singleton when `getConfiguration` is called
Configuration.setConfiguration(null);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,15 @@

import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.AdminClientConfig;
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.ThreadUtils;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.controller.Controller;
import org.apache.kafka.metadata.authorizer.StandardAuthorizer;
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
import org.apache.kafka.metadata.storage.Formatter;
import org.apache.kafka.network.SocketServerConfigs;
Expand Down Expand Up @@ -176,6 +179,15 @@ private KafkaConfig createNodeConfig(TestKitNode node) throws IOException {
// reduce log cleaner offset map memory usage
props.putIfAbsent(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152");

if (brokerSecurityProtocol.equals(SecurityProtocol.SASL_PLAINTEXT.name)) {
props.putIfAbsent(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, "PLAIN");
props.putIfAbsent(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, "PLAIN");
props.putIfAbsent(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG, "PLAIN");
props.putIfAbsent(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, StandardAuthorizer.class.getName());
props.putIfAbsent(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "false");
props.putIfAbsent(StandardAuthorizer.SUPER_USERS_CONFIG, "User:" + JaasUtils.KAFKA_PLAIN_ADMIN);
}

// Add associated broker node property overrides
if (brokerNode != null) {
props.putAll(brokerNode.propertyOverrides());
Expand All @@ -194,6 +206,25 @@ public KafkaClusterTestKit build() throws Exception {
Map<Integer, BrokerServer> brokers = new HashMap<>();
Map<Integer, SharedServer> jointServers = new HashMap<>();
File baseDirectory = null;
File jaasFile = null;

if (brokerSecurityProtocol.equals(SecurityProtocol.SASL_PLAINTEXT.name)) {
jaasFile = JaasUtils.writeJaasContextsToFile(Map.of(
JaasUtils.KAFKA_SERVER_CONTEXT_NAME,
new JaasUtils.JaasSection(JaasUtils.KAFKA_SERVER_CONTEXT_NAME,
List.of(
JaasModule.plainLoginModule(
JaasUtils.KAFKA_PLAIN_ADMIN, JaasUtils.KAFKA_PLAIN_ADMIN_PASSWORD,
true,
Map.of(
JaasUtils.KAFKA_PLAIN_USER1, JaasUtils.KAFKA_PLAIN_USER1_PASSWORD,
JaasUtils.KAFKA_PLAIN_ADMIN, JaasUtils.KAFKA_PLAIN_ADMIN_PASSWORD)
)
)
)
));
JaasUtils.refreshJavaLoginConfigParam(jaasFile);
}

try {
baseDirectory = new File(nodes.baseDirectory());
Expand Down Expand Up @@ -272,7 +303,8 @@ public KafkaClusterTestKit build() throws Exception {
brokers,
baseDirectory,
faultHandlerFactory,
socketFactoryManager);
socketFactoryManager,
jaasFile == null ? Optional.empty() : Optional.of(jaasFile));
}

private String listeners(int node) {
Expand Down Expand Up @@ -316,14 +348,16 @@ private static void setupNodeDirectories(File baseDirectory,
private final SimpleFaultHandlerFactory faultHandlerFactory;
private final PreboundSocketFactoryManager socketFactoryManager;
private final String controllerListenerName;
private final Optional<File> jaasFile;

private KafkaClusterTestKit(
TestKitNodes nodes,
Map<Integer, ControllerServer> controllers,
Map<Integer, BrokerServer> brokers,
File baseDirectory,
SimpleFaultHandlerFactory faultHandlerFactory,
PreboundSocketFactoryManager socketFactoryManager
PreboundSocketFactoryManager socketFactoryManager,
Optional<File> jaasFile
) {
/*
Number of threads = Total number of brokers + Total number of controllers + Total number of Raft Managers
Expand All @@ -339,6 +373,7 @@ private KafkaClusterTestKit(
this.faultHandlerFactory = faultHandlerFactory;
this.socketFactoryManager = socketFactoryManager;
this.controllerListenerName = nodes.controllerListenerName().value();
this.jaasFile = jaasFile;
}

public void format() throws Exception {
Expand Down Expand Up @@ -602,6 +637,13 @@ public void close() throws Exception {
waitForAllFutures(futureEntries);
futureEntries.clear();
Utils.delete(baseDirectory);
jaasFile.ifPresent(f -> {
try {
Utils.delete(f);
} catch (IOException e) {
throw new RuntimeException(e);
}
});
} catch (Exception e) {
for (Entry<String, Future<?>> entry : futureEntries) {
entry.getValue().cancel(true);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -159,8 +159,9 @@ public TestKitNodes build() {
throw new IllegalArgumentException("Invalid value for numDisksPerBroker");
}
// TODO: remove this assertion after https://issues.apache.org/jira/browse/KAFKA-16680 is finished
if (brokerSecurityProtocol != SecurityProtocol.PLAINTEXT || controllerSecurityProtocol != SecurityProtocol.PLAINTEXT) {
throw new IllegalArgumentException("Currently only support PLAINTEXT security protocol");
if ((brokerSecurityProtocol != SecurityProtocol.PLAINTEXT && brokerSecurityProtocol != SecurityProtocol.SASL_PLAINTEXT) ||
(controllerSecurityProtocol != SecurityProtocol.PLAINTEXT && controllerSecurityProtocol != SecurityProtocol.SASL_PLAINTEXT)) {
throw new IllegalArgumentException("Currently only support PLAINTEXT / SASL_PLAINTEXT security protocol");
}
if (baseDirectory == null) {
this.baseDirectory = TestUtils.tempDirectory().toPath();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,11 @@ public class TestKitNodeTest {
@ParameterizedTest
@EnumSource(SecurityProtocol.class)
public void testSecurityProtocol(SecurityProtocol securityProtocol) {
if (securityProtocol != SecurityProtocol.PLAINTEXT) {
assertEquals("Currently only support PLAINTEXT security protocol",
if (securityProtocol != SecurityProtocol.PLAINTEXT && securityProtocol != SecurityProtocol.SASL_PLAINTEXT) {
assertEquals("Currently only support PLAINTEXT / SASL_PLAINTEXT security protocol",
assertThrows(IllegalArgumentException.class,
() -> new TestKitNodes.Builder().setBrokerSecurityProtocol(securityProtocol).build()).getMessage());
assertEquals("Currently only support PLAINTEXT security protocol",
assertEquals("Currently only support PLAINTEXT / SASL_PLAINTEXT security protocol",
assertThrows(IllegalArgumentException.class,
() -> new TestKitNodes.Builder().setControllerSecurityProtocol(securityProtocol).build()).getMessage());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,9 +37,12 @@
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.acl.AccessControlEntry;
import org.apache.kafka.common.acl.AclBindingFilter;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.test.JaasUtils;
import org.apache.kafka.common.test.TestUtils;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.fault.FaultHandlerException;
Expand Down Expand Up @@ -162,7 +165,7 @@ default <K, V> Producer<K, V> producer(Map<String, Object> configs) {
props.putIfAbsent(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
props.putIfAbsent(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
props.putIfAbsent(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers());
return new KafkaProducer<>(props);
return new KafkaProducer<>(setClientSaslConfig(props));
}

default <K, V> Producer<K, V> producer() {
Expand All @@ -176,7 +179,7 @@ default <K, V> Consumer<K, V> consumer(Map<String, Object> configs) {
props.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
props.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group_" + TestUtils.randomString(5));
props.putIfAbsent(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers());
return new KafkaConsumer<>(props);
return new KafkaConsumer<>(setClientSaslConfig(props));
}

default <K, V> Consumer<K, V> consumer() {
Expand All @@ -192,7 +195,23 @@ default Admin admin(Map<String, Object> configs, boolean usingBootstrapControlle
props.putIfAbsent(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers());
props.remove(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG);
}
return Admin.create(props);
return Admin.create(setClientSaslConfig(props));
}

default Map<String, Object> setClientSaslConfig(Map<String, Object> configs) {
Map<String, Object> props = new HashMap<>(configs);
if (config().brokerSecurityProtocol() == SecurityProtocol.SASL_PLAINTEXT) {
props.putIfAbsent(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_PLAINTEXT.name);
props.putIfAbsent(SaslConfigs.SASL_MECHANISM, "PLAIN");
props.putIfAbsent(
SaslConfigs.SASL_JAAS_CONFIG,
String.format(
"org.apache.kafka.common.security.plain.PlainLoginModule required username=\"%s\" password=\"%s\";",
JaasUtils.KAFKA_PLAIN_ADMIN, JaasUtils.KAFKA_PLAIN_ADMIN_PASSWORD
)
);
}
return props;
}

default Admin admin(Map<String, Object> configs) {
Expand Down
Loading
Loading