showuon commented on code in PR #15917: URL: https://github.com/apache/kafka/pull/15917#discussion_r1597526821
########## storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java: ########## @@ -171,111 +146,121 @@ private void waitUntilConsumerCatchesUp(TopicIdPartition newLeaderTopicIdPartiti long time = System.currentTimeMillis(); while (true) { - if (System.currentTimeMillis() - time > timeoutMs) { - throw new TimeoutException("Timed out after " + timeoutMs + "ms "); + if (System.currentTimeMillis() - time > 30000L) { + throw new TimeoutException("Timed out after " + 30000L + "ms "); Review Comment: nit: Let's make it as a variable, so that if we want to change the timeout, we can change one place only. ########## storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java: ########## @@ -16,153 +16,128 @@ */ package org.apache.kafka.server.log.remote.metadata.storage; +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.Type; +import kafka.test.junit.ClusterTestExtensions; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException; import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.collection.JavaConverters; -import scala.collection.Seq; import java.io.IOException; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Properties; import java.util.concurrent.TimeoutException; -@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters +@ExtendWith(ClusterTestExtensions.class) +@ClusterTestDefaults(brokers = 3, clusterType = Type.ALL) public class TopicBasedRemoteLogMetadataManagerTest { private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerTest.class); private static final int SEG_SIZE = 1024 * 1024; private final Time time = new MockTime(1); - private final TopicBasedRemoteLogMetadataManagerHarness remoteLogMetadataManagerHarness = new TopicBasedRemoteLogMetadataManagerHarness(); - @BeforeEach - public void setup() { - // Start the cluster and initialize TopicBasedRemoteLogMetadataManager. - remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), true); - } - - @AfterEach - public void teardown() throws IOException { - remoteLogMetadataManagerHarness.close(); - } - - public TopicBasedRemoteLogMetadataManager topicBasedRlmm() { - return remoteLogMetadataManagerHarness.remoteLogMetadataManager(); - } - - @Test - public void testDoesTopicExist() { - Properties adminConfig = remoteLogMetadataManagerHarness.adminClientConfig(); - ListenerName listenerName = remoteLogMetadataManagerHarness.listenerName(); - try (Admin admin = remoteLogMetadataManagerHarness.createAdminClient(listenerName, adminConfig)) { + @ClusterTest + public void testDoesTopicExist(ClusterInstance clusterInstance) throws Exception { + try (TopicBasedRemoteLogMetadataManager remoteLogMetadataManager = RemoteLogMetadataManagerTestUtils.builder() + .bootstrapServers(clusterInstance.bootstrapServers()).startConsumerThread(true).build(); + Admin admin = clusterInstance.createAdminClient()) { String topic = "test-topic-exist"; - remoteLogMetadataManagerHarness.createTopic(topic, 1, 1, new Properties(), - listenerName, adminConfig); - boolean doesTopicExist = topicBasedRlmm().doesTopicExist(admin, topic); + admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).all().get(); + boolean doesTopicExist = remoteLogMetadataManager.doesTopicExist(admin, topic); Assertions.assertTrue(doesTopicExist); } } - @Test - public void testTopicDoesNotExist() { - Properties adminConfig = remoteLogMetadataManagerHarness.adminClientConfig(); - ListenerName listenerName = remoteLogMetadataManagerHarness.listenerName(); - try (Admin admin = remoteLogMetadataManagerHarness.createAdminClient(listenerName, adminConfig)) { + @ClusterTest + public void testTopicDoesNotExist(ClusterInstance clusterInstance) throws IOException { + try (TopicBasedRemoteLogMetadataManager remoteLogMetadataManager = RemoteLogMetadataManagerTestUtils.builder() + .bootstrapServers(clusterInstance.bootstrapServers()).startConsumerThread(true).build(); + Admin admin = clusterInstance.createAdminClient()) { String topic = "dummy-test-topic"; - boolean doesTopicExist = topicBasedRlmm().doesTopicExist(admin, topic); + boolean doesTopicExist = remoteLogMetadataManager.doesTopicExist(admin, topic); Assertions.assertFalse(doesTopicExist); } } - @Test - public void testWithNoAssignedPartitions() throws Exception { - // This test checks simple lifecycle of TopicBasedRemoteLogMetadataManager with out assigning any leader/follower partitions. - // This should close successfully releasing the resources. - log.info("Not assigning any partitions on TopicBasedRemoteLogMetadataManager"); + @ClusterTest + public void testWithNoAssignedPartitions(ClusterInstance clusterInstance) throws IOException { + try (TopicBasedRemoteLogMetadataManager ignored = RemoteLogMetadataManagerTestUtils.builder() + .bootstrapServers(clusterInstance.bootstrapServers()).startConsumerThread(true).build()) { + // This test checks simple lifecycle of TopicBasedRemoteLogMetadataManager without assigning any leader/follower partitions. + // This should close successfully releasing the resources. + log.info("Not assigning any partitions on TopicBasedRemoteLogMetadataManager"); Review Comment: I don't think this INFO log is necessary. ########## storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataManagerTestUtils.java: ########## @@ -0,0 +1,111 @@ +/* + * 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.server.log.remote.metadata.storage; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.TopicIdPartition; +import org.apache.kafka.test.TestUtils; +import org.junit.jupiter.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.BROKER_ID; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP; +import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.REMOTE_LOG_METADATA_TOPIC_RETENTION_MS_PROP; + +class RemoteLogMetadataManagerTestUtils { + private static final Logger log = LoggerFactory.getLogger(RemoteLogMetadataManagerTestUtils.class); + + static final int METADATA_TOPIC_PARTITIONS_COUNT = 3; + static final short METADATA_TOPIC_REPLICATION_FACTOR = 2; + static final long METADATA_TOPIC_RETENTION_MS = 24 * 60 * 60 * 1000L; + + static Builder builder() { + return new Builder(); + } + + static class Builder { + private String bootstrapServers; + private boolean startConsumerThread; + private RemoteLogMetadataTopicPartitioner remoteLogMetadataTopicPartitioner; + private Map<String, Object> overrideRemoteLogMetadataManagerProps = Collections.emptyMap(); + + private Builder() { + } + + public Builder bootstrapServers(String bootstrapServers) { + this.bootstrapServers = Objects.requireNonNull(bootstrapServers); + return this; + } + + public Builder startConsumerThread(boolean startConsumerThread) { + this.startConsumerThread = startConsumerThread; + return this; + } + + public Builder remoteLogMetadataTopicPartitioner(RemoteLogMetadataTopicPartitioner remoteLogMetadataTopicPartitioner) { + this.remoteLogMetadataTopicPartitioner = Objects.requireNonNull(remoteLogMetadataTopicPartitioner); + return this; + } + + public Builder overrideRemoteLogMetadataManagerProps(Map<String, Object> overrideRemoteLogMetadataManagerProps) { + this.overrideRemoteLogMetadataManagerProps = Objects.requireNonNull(overrideRemoteLogMetadataManagerProps); + return this; + } + + public TopicBasedRemoteLogMetadataManager build() { + String logDir = TestUtils.tempDirectory("rlmm_segs_").getAbsolutePath(); + TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager = new TopicBasedRemoteLogMetadataManager(startConsumerThread); + + // Initialize TopicBasedRemoteLogMetadataManager. + Map<String, Object> configs = new HashMap<>(); + configs.put(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers); + configs.put(BROKER_ID, 0); + configs.put(LOG_DIR, logDir); + configs.put(REMOTE_LOG_METADATA_TOPIC_PARTITIONS_PROP, METADATA_TOPIC_PARTITIONS_COUNT); + configs.put(REMOTE_LOG_METADATA_TOPIC_REPLICATION_FACTOR_PROP, METADATA_TOPIC_REPLICATION_FACTOR); + configs.put(REMOTE_LOG_METADATA_TOPIC_RETENTION_MS_PROP, METADATA_TOPIC_RETENTION_MS); + + log.debug("TopicBasedRemoteLogMetadataManager configs before adding overridden properties: {}", configs); + // Add override properties. + configs.putAll(overrideRemoteLogMetadataManagerProps); + log.debug("TopicBasedRemoteLogMetadataManager configs after adding overridden properties: {}", configs); + + topicBasedRemoteLogMetadataManager.configure(configs); + if (remoteLogMetadataTopicPartitioner != null) + topicBasedRemoteLogMetadataManager.setRlmTopicPartitioner(remoteLogMetadataTopicPartitioner); Review Comment: This is duplicated with `TopicBasedRemoteLogMetadataManagerHarness#initializeRemoteLogMetadataManager`, maybe we should extract them as a helper method? ########## storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java: ########## @@ -16,153 +16,128 @@ */ package org.apache.kafka.server.log.remote.metadata.storage; +import kafka.test.ClusterInstance; +import kafka.test.annotation.ClusterTest; +import kafka.test.annotation.ClusterTestDefaults; +import kafka.test.annotation.Type; +import kafka.test.junit.ClusterTestExtensions; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.server.log.remote.storage.RemoteResourceNotFoundException; import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import scala.collection.JavaConverters; -import scala.collection.Seq; import java.io.IOException; -import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Properties; import java.util.concurrent.TimeoutException; -@SuppressWarnings("deprecation") // Added for Scala 2.12 compatibility for usages of JavaConverters +@ExtendWith(ClusterTestExtensions.class) +@ClusterTestDefaults(brokers = 3, clusterType = Type.ALL) public class TopicBasedRemoteLogMetadataManagerTest { private static final Logger log = LoggerFactory.getLogger(TopicBasedRemoteLogMetadataManagerTest.class); private static final int SEG_SIZE = 1024 * 1024; private final Time time = new MockTime(1); - private final TopicBasedRemoteLogMetadataManagerHarness remoteLogMetadataManagerHarness = new TopicBasedRemoteLogMetadataManagerHarness(); - @BeforeEach - public void setup() { - // Start the cluster and initialize TopicBasedRemoteLogMetadataManager. - remoteLogMetadataManagerHarness.initialize(Collections.emptySet(), true); - } - - @AfterEach - public void teardown() throws IOException { - remoteLogMetadataManagerHarness.close(); - } - - public TopicBasedRemoteLogMetadataManager topicBasedRlmm() { - return remoteLogMetadataManagerHarness.remoteLogMetadataManager(); - } - - @Test - public void testDoesTopicExist() { - Properties adminConfig = remoteLogMetadataManagerHarness.adminClientConfig(); - ListenerName listenerName = remoteLogMetadataManagerHarness.listenerName(); - try (Admin admin = remoteLogMetadataManagerHarness.createAdminClient(listenerName, adminConfig)) { + @ClusterTest + public void testDoesTopicExist(ClusterInstance clusterInstance) throws Exception { + try (TopicBasedRemoteLogMetadataManager remoteLogMetadataManager = RemoteLogMetadataManagerTestUtils.builder() + .bootstrapServers(clusterInstance.bootstrapServers()).startConsumerThread(true).build(); + Admin admin = clusterInstance.createAdminClient()) { String topic = "test-topic-exist"; - remoteLogMetadataManagerHarness.createTopic(topic, 1, 1, new Properties(), - listenerName, adminConfig); - boolean doesTopicExist = topicBasedRlmm().doesTopicExist(admin, topic); + admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).all().get(); Review Comment: We used to use `TestUtils.createTopicWithAdmin` to make sure the topic is created and all metadata are propagated. I think we should keep using it, to make the test reliable. Same comments to below createTopic places. ########## storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerHarness.java: ########## @@ -109,27 +108,14 @@ public void onPartitionLeadershipChanges(Set<TopicIdPartition> leaderPartitions, topicBasedRemoteLogMetadataManager.setRlmTopicPartitioner(remoteLogMetadataTopicPartitioner); } try { - waitUntilInitialized(60_000); - } catch (TimeoutException e) { + RemoteLogMetadataManagerTestUtils.waitUntilInitialized(this.remoteLogMetadataManager(), 60_000); + } catch (InterruptedException e) { throw new KafkaException(e); } Review Comment: nit: I don't think we need to catch any exception here because no matter which exception thrown, the test will fail, right? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org