kamalcph commented on code in PR #15917:
URL: https://github.com/apache/kafka/pull/15917#discussion_r1596873472


##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataManagerTestUtils.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+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 {

Review Comment:
   Will the `TopicBasedRemoteLogMetadataManagerHarness` class be removed?



##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataManagerTestUtils.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+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 Set<TopicIdPartition> topicIdPartitions = 
Collections.emptySet();
+        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 topicIdPartitions(Set<TopicIdPartition> 
topicIdPartitions) {
+            this.topicIdPartitions = Objects.requireNonNull(topicIdPartitions);
+            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) {
+                @Override
+                public void onPartitionLeadershipChanges(Set<TopicIdPartition> 
leaderPartitions,
+                                                         Set<TopicIdPartition> 
followerPartitions) {
+                    Set<TopicIdPartition> allReplicas = new 
HashSet<>(leaderPartitions);
+                    allReplicas.addAll(followerPartitions);
+                    // Make sure the topic partition dirs exist as the topics 
might not have been created on this broker.
+                    for (TopicIdPartition topicIdPartition : allReplicas) {
+                        // Create partition directory in the log directory 
created by topicBasedRemoteLogMetadataManager.
+                        File partitionDir = new File(new 
File(config().logDir()), topicIdPartition.topicPartition().topic() + "-" + 
topicIdPartition.topicPartition().partition());
+                        partitionDir.mkdirs();
+                        if (!partitionDir.exists()) {
+                            throw new KafkaException("Partition directory:[" + 
partitionDir + "] could not be created successfully.");
+                        }
+                    }
+
+                    super.onPartitionLeadershipChanges(leaderPartitions, 
followerPartitions);
+                }
+            };
+
+            // 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);
+            }
+            try {
+                waitUntilInitialized(topicBasedRemoteLogMetadataManager, 
60_000);
+            } catch (TimeoutException e) {
+                throw new KafkaException(e);
+            }
+
+            
topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges(topicIdPartitions,
 Collections.emptySet());
+            return topicBasedRemoteLogMetadataManager;
+        }
+    }
+
+    public static void waitUntilInitialized(TopicBasedRemoteLogMetadataManager 
topicBasedRemoteLogMetadataManager,

Review Comment:
   can we use TestUtils.waitForCondition instead?



##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogMetadataManagerTestUtils.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.KafkaException;
+import org.apache.kafka.common.TopicIdPartition;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.TimeoutException;
+
+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 Set<TopicIdPartition> topicIdPartitions = 
Collections.emptySet();
+        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 topicIdPartitions(Set<TopicIdPartition> 
topicIdPartitions) {
+            this.topicIdPartitions = Objects.requireNonNull(topicIdPartitions);
+            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) {
+                @Override
+                public void onPartitionLeadershipChanges(Set<TopicIdPartition> 
leaderPartitions,

Review Comment:
   The `onPartitionLeadershipChanges` override is no longer required post the 
removal of FileBasedRemoteLogMetadataCache in #15636. 



-- 
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

Reply via email to