Skip to content

Commit

Permalink
KAFKA-17922 add helper to ClusterInstance to create client component (a…
Browse files Browse the repository at this point in the history
…pache#17666)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
  • Loading branch information
m1a2st authored Nov 13, 2024
1 parent edab667 commit 6bc7be7
Show file tree
Hide file tree
Showing 26 changed files with 1,830 additions and 1,615 deletions.
10 changes: 6 additions & 4 deletions core/src/test/java/kafka/admin/AdminFenceProducersTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@
import org.junit.jupiter.api.extension.ExtendWith;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Properties;
import java.util.concurrent.ExecutionException;

Expand Down Expand Up @@ -82,7 +84,7 @@ void testFenceAfterProducerCommit() throws Exception {
clusterInstance.createTopic(TOPIC_NAME, 1, (short) 1);

try (KafkaProducer<byte[], byte[]> producer = createProducer();
Admin adminClient = clusterInstance.createAdminClient()) {
Admin adminClient = clusterInstance.admin()) {
producer.initTransactions();
producer.beginTransaction();
producer.send(RECORD).get();
Expand All @@ -103,10 +105,10 @@ void testFenceAfterProducerCommit() throws Exception {

@ClusterTest
void testFenceProducerTimeoutMs() {
Properties config = new Properties();
Map<String, Object> config = new HashMap<>();
config.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:" + INCORRECT_BROKER_PORT);

try (Admin adminClient = clusterInstance.createAdminClient(config)) {
try (Admin adminClient = clusterInstance.admin(config)) {
ExecutionException exception = assertThrows(
ExecutionException.class, () ->
adminClient.fenceProducers(Collections.singletonList(TXN_ID), new FenceProducersOptions().timeoutMs(0)).all().get());
Expand All @@ -119,7 +121,7 @@ void testFenceBeforeProducerCommit() throws Exception {
clusterInstance.createTopic(TOPIC_NAME, 1, (short) 1);

try (KafkaProducer<byte[], byte[]> producer = createProducer();
Admin adminClient = clusterInstance.createAdminClient()) {
Admin adminClient = clusterInstance.admin()) {

producer.initTransactions();
producer.beginTransaction();
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/java/kafka/admin/ClientTelemetryTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ public void testClientInstanceId(ClusterInstance clusterInstance) throws Interru
public void testIntervalMsParser(ClusterInstance clusterInstance) {
List<String> alterOpts = asList("--bootstrap-server", clusterInstance.bootstrapServers(),
"--alter", "--entity-type", "client-metrics", "--entity-name", "test", "--add-config", "interval.ms=bbb");
try (Admin client = clusterInstance.createAdminClient()) {
try (Admin client = clusterInstance.admin()) {
ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(alterOpts));

Throwable e = assertThrows(ExecutionException.class, () -> ConfigCommand.alterConfig(client, addOpts));
Expand Down
14 changes: 7 additions & 7 deletions core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -142,7 +142,7 @@ public void testNullStatusOnKraftCommandAlterClientMetrics() {
public void testDynamicBrokerConfigUpdateUsingKraft() throws Exception {
List<String> alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers());

try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
// Add config
alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "110000"), alterOpts);
alterAndVerifyConfig(client, Optional.empty(), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "120000"), alterOpts);
Expand Down Expand Up @@ -191,7 +191,7 @@ public void testGroupConfigUpdateUsingKraft() throws Exception {
}

private void verifyGroupConfigUpdate(List<String> alterOpts) throws Exception {
try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
// Add config
Map<String, String> configs = new HashMap<>();
configs.put(CONSUMER_SESSION_TIMEOUT_MS_CONFIG, "50000");
Expand Down Expand Up @@ -221,7 +221,7 @@ public void testClientMetricsConfigUpdate() throws Exception {
}

private void verifyClientMetricsConfigUpdate(List<String> alterOpts) throws Exception {
try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
// Add config
Map<String, String> configs = new HashMap<>();
configs.put("metrics", "");
Expand All @@ -240,7 +240,7 @@ private void verifyClientMetricsConfigUpdate(List<String> alterOpts) throws Exce
public void testAlterReadOnlyConfigInKRaftThenShouldFail() {
List<String> alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers());

try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
assertThrows(ExecutionException.class,
() -> alterConfigWithKraft(client, Optional.of(defaultBrokerId),
singletonMap(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"), alterOpts));
Expand All @@ -257,7 +257,7 @@ public void testAlterReadOnlyConfigInKRaftThenShouldFail() {
public void testUpdateClusterWideConfigInKRaftThenShouldSuccessful() throws Exception {
List<String> alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers());

try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
singletonMap("log.flush.interval.messages", "100"), alterOpts);
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
Expand All @@ -272,7 +272,7 @@ public void testUpdatePerBrokerConfigWithListenerNameInKRaftThenShouldSuccessful
List<String> alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers());
String listenerName = "listener.name.internal.";

try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
singletonMap(listenerName + "ssl.truststore.type", "PKCS12"), alterOpts);
alterAndVerifyConfig(client, Optional.of(defaultBrokerId),
Expand All @@ -288,7 +288,7 @@ public void testUpdatePerBrokerConfigWithListenerNameInKRaftThenShouldSuccessful
public void testUpdatePerBrokerConfigInKRaftThenShouldFail() {
List<String> alterOpts = generateDefaultAlterOpts(cluster.bootstrapServers());

try (Admin client = cluster.createAdminClient()) {
try (Admin client = cluster.admin()) {
assertThrows(ExecutionException.class,
() -> alterConfigWithKraft(client, Optional.of(defaultBrokerId),
singletonMap(SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"), alterOpts));
Expand Down
22 changes: 11 additions & 11 deletions core/src/test/java/kafka/admin/DeleteTopicTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ public class DeleteTopicTest {

@ClusterTest
public void testDeleteTopicWithAllAliveReplicas(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
admin.deleteTopics(List.of(DEFAULT_TOPIC)).all().get();
cluster.waitForTopic(DEFAULT_TOPIC, 0);
Expand All @@ -88,7 +88,7 @@ public void testDeleteTopicWithAllAliveReplicas(ClusterInstance cluster) throws

@ClusterTest
public void testResumeDeleteTopicWithRecoveredFollower(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
int leaderId = waitUtilLeaderIsKnown(cluster.brokers(), topicPartition);
Expand All @@ -111,7 +111,7 @@ public void testResumeDeleteTopicWithRecoveredFollower(ClusterInstance cluster)

@ClusterTest(brokers = 4)
public void testPartitionReassignmentDuringDeleteTopic(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
Map<Integer, KafkaBroker> servers = findPartitionHostingBrokers(cluster.brokers());
Expand All @@ -137,7 +137,7 @@ public void testPartitionReassignmentDuringDeleteTopic(ClusterInstance cluster)

@ClusterTest(brokers = 4)
public void testIncreasePartitionCountDuringDeleteTopic(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
Map<Integer, KafkaBroker> partitionHostingBrokers = findPartitionHostingBrokers(cluster.brokers());
Expand Down Expand Up @@ -165,7 +165,7 @@ public void testIncreasePartitionCountDuringDeleteTopic(ClusterInstance cluster)

@ClusterTest
public void testDeleteTopicDuringAddPartition(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
int leaderId = waitUtilLeaderIsKnown(cluster.brokers(), new TopicPartition(DEFAULT_TOPIC, 0));
TopicPartition newTopicPartition = new TopicPartition(DEFAULT_TOPIC, 1);
Expand All @@ -190,7 +190,7 @@ public void testDeleteTopicDuringAddPartition(ClusterInstance cluster) throws Ex

@ClusterTest
public void testAddPartitionDuringDeleteTopic(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
// partitions to be added to the topic later
TopicPartition newTopicPartition = new TopicPartition(DEFAULT_TOPIC, 1);
Expand All @@ -204,7 +204,7 @@ public void testAddPartitionDuringDeleteTopic(ClusterInstance cluster) throws Ex

@ClusterTest
public void testRecreateTopicAfterDeletion(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
admin.deleteTopics(List.of(DEFAULT_TOPIC)).all().get();
Expand All @@ -216,7 +216,7 @@ public void testRecreateTopicAfterDeletion(ClusterInstance cluster) throws Excep
}
@ClusterTest
public void testDeleteNonExistingTopic(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
String topic = "test2";
Expand All @@ -243,7 +243,7 @@ public void testDeleteNonExistingTopic(ClusterInstance cluster) throws Exception
@ClusterConfigProperty(key = "log.cleaner.dedupe.buffer.size", value = "1048577")
})
public void testDeleteTopicWithCleaner(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
// for simplicity, we are validating cleaner offsets on a single broker
Expand All @@ -262,7 +262,7 @@ public void testDeleteTopicWithCleaner(ClusterInstance cluster) throws Exception

@ClusterTest
public void testDeleteTopicAlreadyMarkedAsDeleted(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
admin.deleteTopics(List.of(DEFAULT_TOPIC)).all().get();

Expand All @@ -282,7 +282,7 @@ public void testDeleteTopicAlreadyMarkedAsDeleted(ClusterInstance cluster) throw
@ClusterTest(controllers = 1,
serverProperties = {@ClusterConfigProperty(key = ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, value = "false")})
public void testDisableDeleteTopic(ClusterInstance cluster) throws Exception {
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(List.of(new NewTopic(DEFAULT_TOPIC, expectedReplicaAssignment))).all().get();
TopicPartition topicPartition = new TopicPartition(DEFAULT_TOPIC, 0);
TestUtils.waitForCondition(() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,7 @@ public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOExcept
RaftClusterInvocationContext.RaftClusterInstance raftInstance =
(RaftClusterInvocationContext.RaftClusterInstance) cluster;

try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
admin.createTopics(Collections.singletonList(new NewTopic("foo", 1, (short) 3))).all().get();
}
cluster.waitForTopic("foo", 1);
Expand All @@ -82,7 +82,7 @@ public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOExcept
assertTrue(partitionMetadataFile.isPresent());

raftInstance.getUnderlying().brokers().get(0).shutdown();
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
TestUtils.waitForCondition(() -> {
List<TopicPartitionInfo> partitionInfos = admin.describeTopics(Collections.singletonList("foo"))
.topicNameValues().get("foo").get().partitions();
Expand All @@ -96,7 +96,7 @@ public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOExcept
raftInstance.getUnderlying().brokers().get(0).startup();
// make sure there is no error during load logs
assertDoesNotThrow(() -> raftInstance.getUnderlying().fatalFaultHandler().maybeRethrowFirstException());
try (Admin admin = cluster.createAdminClient()) {
try (Admin admin = cluster.admin()) {
TestUtils.waitForCondition(() -> {
List<TopicPartitionInfo> partitionInfos = admin.describeTopics(Collections.singletonList("foo"))
.topicNameValues().get("foo").get().partitions();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -289,7 +289,7 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
}

private def withAdmin(f: Admin => Unit): Unit = {
val admin: Admin = cluster.createAdminClient()
val admin: Admin = cluster.admin()
try {
f(admin)
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,42 +40,54 @@ class MetadataVersionIntegrationTest {
new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_4_IV0)
))
def testBasicMetadataVersionUpgrade(clusterInstance: ClusterInstance): Unit = {
val admin = clusterInstance.createAdminClient()
val describeResult = admin.describeFeatures()
val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
assertEquals(ff.minVersionLevel(), clusterInstance.config().metadataVersion().featureLevel())
assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel())
val admin = clusterInstance.admin()
try {
val describeResult = admin.describeFeatures()
val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
assertEquals(ff.minVersionLevel(), clusterInstance.config().metadataVersion().featureLevel())
assertEquals(ff.maxVersionLevel(), clusterInstance.config().metadataVersion().featureLevel())

// Update to new version
val updateVersion = MetadataVersion.IBP_3_5_IV1.featureLevel.shortValue
val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get()
// Update to new version
val updateVersion = MetadataVersion.IBP_3_5_IV1.featureLevel.shortValue
val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get()

// Verify that new version is visible on broker
TestUtils.waitUntilTrue(() => {
val describeResult2 = admin.describeFeatures()
val ff2 = describeResult2.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
ff2.minVersionLevel() == updateVersion && ff2.maxVersionLevel() == updateVersion
}, "Never saw metadata.version increase on broker")
// Verify that new version is visible on broker
TestUtils.waitUntilTrue(() => {
val describeResult2 = admin.describeFeatures()
val ff2 = describeResult2.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
ff2.minVersionLevel() == updateVersion && ff2.maxVersionLevel() == updateVersion
}, "Never saw metadata.version increase on broker")
} finally {
admin.close()
}
}

@ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV0)
def testUpgradeSameVersion(clusterInstance: ClusterInstance): Unit = {
val admin = clusterInstance.createAdminClient()
val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue
val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get()
val admin = clusterInstance.admin()
try {
val updateVersion = MetadataVersion.IBP_3_3_IV0.featureLevel.shortValue
val updateResult = admin.updateFeatures(
Map("metadata.version" -> new FeatureUpdate(updateVersion, UpgradeType.UPGRADE)).asJava, new UpdateFeaturesOptions())
updateResult.all().get()
} finally {
admin.close()
}
}

@ClusterTest(types = Array(Type.KRAFT))
def testDefaultIsLatestVersion(clusterInstance: ClusterInstance): Unit = {
val admin = clusterInstance.createAdminClient()
val describeResult = admin.describeFeatures()
val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
assertEquals(ff.minVersionLevel(), MetadataVersion.latestTesting().featureLevel(),
"If this test fails, check the default MetadataVersion in the @ClusterTest annotation")
assertEquals(ff.maxVersionLevel(), MetadataVersion.latestTesting().featureLevel())
val admin = clusterInstance.admin()
try {
val describeResult = admin.describeFeatures()
val ff = describeResult.featureMetadata().get().finalizedFeatures().get(MetadataVersion.FEATURE_NAME)
assertEquals(ff.minVersionLevel(), MetadataVersion.latestTesting().featureLevel(),
"If this test fails, check the default MetadataVersion in the @ClusterTest annotation")
assertEquals(ff.maxVersionLevel(), MetadataVersion.latestTesting().featureLevel())
} finally {
admin.close()
}
}
}
Loading

0 comments on commit 6bc7be7

Please sign in to comment.