Skip to content

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

Merged
chia7712 merged 3 commits intoapache:trunkfrom
FrankYang0529:KAFKA-17921
Jan 7, 2025
Merged

KAFKA-17921: Support SASL_PLAINTEXT protocol with java.security.auth.login.config#17671
chia7712 merged 3 commits intoapache:trunkfrom
FrankYang0529:KAFKA-17921

Conversation

@FrankYang0529
Copy link
Copy Markdown
Member

Support SASL_PLAINTEXT protocol in ClusterTest framework.

Committer Checklist (excluded from commit message)

  • Verify design and implementation
  • Verify test coverage and CI build status
  • Verify documentation (including upgrade notes)

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 thanks for this patch

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

public class JaasModule {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is this duplicate to core JaasModule?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, this is duplicated. Original JaasModule is in core module. I think we can remove original one after we migrate all sasl test case to new framework. WDYT?

waitForAllFutures(futureEntries);
futureEntries.clear();
Utils.delete(baseDirectory);
if (jaasFile.isPresent()) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please use ifPresent instead?


import javax.security.auth.login.Configuration;

public class JaasTestUtils {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is already in xxx.test package, maybe we can call it JaasUtils?

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

public static File writeJaasContextsToFile(List<JaasSection> jaasSections) throws IOException {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we really need this object? How about using map? It is more simple


private final Map<String, String> entries;

private JaasModule(String name, boolean debug, Map<String, String> entries) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you please try to use record class?

configs.put(SaslConfigs.SASL_JAAS_CONFIG,
String.format("org.apache.kafka.common.security.plain.PlainLoginModule required username=\"%s\" password=\"%s\";",
JaasTestUtils.KAFKA_PLAIN_USER1, JaasTestUtils.KAFKA_PLAIN_USER1_PASSWORD));
try (Admin admin = clusterInstance.admin(configs)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Those helper should return authorized client object, right? Otherwise, developers have to configure them manually

@FrankYang0529 FrankYang0529 force-pushed the KAFKA-17921 branch 2 times, most recently from 7e4b4a6 to c9ea0ad Compare December 29, 2024 14:08
Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 thanks for this patch. two minor comments are left PTAL

assertDoesNotThrow(() -> admin.describeAcls(AclBindingFilter.ANY).values().get());
}
String topic = "sasl-plaintext-topic";
Assertions.assertDoesNotThrow(() -> clusterInstance.createTopic(topic, 1, (short) 1));
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the APIs in question only throw runtime exceptions, there's no need to use assertDoesNotThrow, which is typically employed to convert checked exceptions into runtime exceptions.

}

// client with non-admin credentials
Map<String, Object> nonAdminConfig = Map.of(
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the helper methods of clusterInstance should be able configure the sasl.jaas.config automatically - that is why we add those helper to clusterInstance

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @chia7712, thanks for the review. Yes, we already set default user to admin, consumer, and producer helper function automatically. Here, we would like to test developer overrides sasl.jaas.config with non-admin credentials. We also test overriding with unknown credential, so we can make sure the cluster instance with sasl plaintext works.

@FrankYang0529
Copy link
Copy Markdown
Member Author

Hi @chia7712, thanks for the review. I addressed all comments. Could you review it when you have time? Thank you.

Copy link
Copy Markdown
Member

@chia7712 chia7712 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@FrankYang0529 thanks for this patch! it is great to see our new test infra has such support

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 {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems we don't use the key, so maybe we can change the collection from Map to Set?

futureEntries.clear();
Utils.delete(baseDirectory);
jaasFile.ifPresent(f -> {
try {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To enhance readability, we should explore alternatives to nested try-catch blocks, even if it involves using a more traditional coding style. for example: if (jaasFile.isPresent()) Utils.delete(jaasFile.get());

@ClusterTest(
types = {Type.KRAFT, Type.CO_KRAFT},
brokerSecurityProtocol = SecurityProtocol.SASL_PLAINTEXT,
controllerSecurityProtocol = SecurityProtocol.SASL_PLAINTEXT,
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we support the SASL_PLAINTEXT on controller? If not, could you please remove this config and open a jira for it?

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, we already support it in this PR. I update KafkaClusterTestKit.Builder#createNodeConfig to respect controllerSecurityProtocol setting.

…login.config

Signed-off-by: PoAn Yang <payang@apache.org>
// reduce log cleaner offset map memory usage
props.putIfAbsent(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152");

if (brokerSecurityProtocol.equals(SecurityProtocol.SASL_PLAINTEXT.name)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it seems we set security for both controller and broker based on the brokerSecurityProtocol, right? if so, that is weird to me as we totally ignore controllerSecurityProtocol ...

Signed-off-by: PoAn Yang <payang@apache.org>
Signed-off-by: PoAn Yang <payang@apache.org>
ExecutionException.class,
() -> admin.describeAcls(AclBindingFilter.ANY, new DescribeAclsOptions().timeoutMs(5000)).values().get()
);
assertInstanceOf(TimeoutException.class, exception.getCause());
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I open https://issues.apache.org/jira/browse/KAFKA-18440 to fix it. please feel free to take over the jira if you have free cycle

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for review. I will handle it.

@chia7712
Copy link
Copy Markdown
Member

chia7712 commented Jan 7, 2025

The failed tests are unrelated and they pass on my local

@chia7712
Copy link
Copy Markdown
Member

chia7712 commented Jan 7, 2025

@chia7712 chia7712 merged commit 228b325 into apache:trunk Jan 7, 2025
chia7712 pushed a commit that referenced this pull request Jan 7, 2025
…ogin.config (#17671)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
@FrankYang0529 FrankYang0529 deleted the KAFKA-17921 branch January 7, 2025 23:58
ijuma added a commit to ijuma/kafka that referenced this pull request Jan 8, 2025
…og-compaction-write-record-v2

* apache-github/trunk: (34 commits)
  MINOR: Bump year to 2025 in NOTICE file (apache#18427)
  KAFKA-18411 Remove ZkProducerIdManager (apache#18413)
  KAFKA-18408 tweak the 'tag' field for BrokerHeartbeatRequest.json, BrokerRegistrationChangeRecord.json and RegisterBrokerRecord.json (apache#18421)
  KAFKA-18414 Remove KRaftRegistrationResult (apache#18401)
  KAFKA-17921 Support SASL_PLAINTEXT protocol with java.security.auth.login.config (apache#17671)
  KAFKA-18384 Remove ZkAlterPartitionManager (apache#18364)
  KAFKA-10790: Add deadlock detection to producer#flush (apache#17946)
  KAFKA-18412: Remove EmbeddedZookeeper (apache#18399)
  MINOR : Improve Exception log in NotEnoughReplicasException(apache#12394)
  MINOR: Improve PlaintextAdminIntegrationTest#testConsumerGroups (apache#18409)
  MINOR: Remove unused local variable (apache#18410)
  MINOR: Remove RaftManager.maybeDeleteMetadataLogDir and AutoTopicCreationManagerTest.scala (apache#17365)
  KAFKA-18368 Remove TestUtils#MockZkConnect and remove zkConnect from TestUtils#createBrokerConfig (apache#18352)
  MINOR: Update Consumer group timeout default to 30 sec (apache#16406)
  MINOR: Fix typo in CommitRequestManager (apache#18407)
  MINOR: cleanup JavaDocs for deprecation warnings (apache#18402)
  KAFKA-18303; Update ShareCoordinator to use new record format (apache#18396)
  MINOR: Update Consumer and Producer JavaDocs for committing offsets (apache#18336)
  KAFKA-16446: Improve controller event duration logging (apache#15622)
  KAFKA-18388 test-kraft-server-start.sh should use log4j2.yaml (apache#18370)
  ...
pranavt84 pushed a commit to pranavt84/kafka that referenced this pull request Jan 27, 2025
…ogin.config (apache#17671)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
…ogin.config (apache#17671)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants