satishd commented on code in PR #14307:
URL: https://github.com/apache/kafka/pull/14307#discussion_r1314468818


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -301,7 +301,7 @@ private Stream<Partition> filterPartitions(Set<Partition> 
partitions) {
 
     private void cacheTopicPartitionIds(TopicIdPartition topicIdPartition) {
         Uuid previousTopicId = 
topicIdByPartitionMap.put(topicIdPartition.topicPartition(), 
topicIdPartition.topicId());
-        if (previousTopicId != null && previousTopicId != 
topicIdPartition.topicId()) {
+        if (previousTopicId != null && 
!previousTopicId.equals(topicIdPartition.topicId())) {

Review Comment:
   Good catch!



##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/DeleteSegmentsByRetentionSizeTest.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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.tiered.storage.integration;
+
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.tiered.storage.TieredStorageTestBuilder;
+import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.kafka.server.log.remote.storage.LocalTieredStorageEvent.EventType.DELETE_SEGMENT;
+
+public class DeleteSegmentsByRetentionSizeTest extends 
TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 1;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final Integer broker0 = 0;
+        final String topicA = "topicA";
+        final Integer p0 = 0;
+        final Integer partitionCount = 1;
+        final Integer replicationFactor = 1;
+        final Integer maxBatchCountPerSegment = 1;
+        final Map<Integer, List<Integer>> replicaAssignment = null;
+        final boolean enableRemoteLogStorage = true;
+        final int beginEpoch = 0;
+        final long startOffset = 3;
+
+        builder.createTopic(topicA, partitionCount, replicationFactor, 
maxBatchCountPerSegment, replicaAssignment,

Review Comment:
   Please add documentation of the test and the behaviour that will be easier 
for readers. This will be easy for folks who are new to this framework and 
trying to understand the test better.  



##########
storage/src/test/java/org/apache/kafka/tiered/storage/integration/PartitionsExpandTest.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.tiered.storage.integration;
+
+import org.apache.kafka.tiered.storage.TieredStorageTestBuilder;
+import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
+import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+
+public final class PartitionsExpandTest extends TieredStorageTestHarness {
+
+    @Override
+    public int brokerCount() {
+        return 2;
+    }
+
+    @Override
+    protected void writeTestSpecifications(TieredStorageTestBuilder builder) {
+        final Integer broker0 = 0;
+        final Integer broker1 = 1;
+        final String topicA = "topicA";
+        final Integer p0 = 0;
+        final Integer p1 = 1;
+        final Integer p2 = 2;
+        final Integer partitionCount = 1;
+        final Integer replicationFactor = 2;
+        final Integer maxBatchCountPerSegment = 1;
+        final boolean enableRemoteLogStorage = true;
+        final List<Integer> p0Assignment = Arrays.asList(broker0, broker1);
+        final List<Integer> p1Assignment = Arrays.asList(broker0, broker1);
+        final List<Integer> p2Assignment = Arrays.asList(broker1, broker0);
+
+        builder
+                .createTopic(topicA, partitionCount, replicationFactor, 
maxBatchCountPerSegment,
+                        Collections.singletonMap(p0, p0Assignment), 
enableRemoteLogStorage)
+                // produce events to partition 0
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 0, new 
KeyValueSpec("k0", "v0"))
+                .expectSegmentToBeOffloaded(broker0, topicA, p0, 1, new 
KeyValueSpec("k1", "v1"))
+                .expectEarliestLocalOffsetInLogDirectory(topicA, p0, 2L)
+                .produce(topicA, p0, new KeyValueSpec("k0", "v0"), new 
KeyValueSpec("k1", "v1"),
+                        new KeyValueSpec("k2", "v2"))
+                // expand the topicA partition-count to 3
+                .createPartitions(topicA, 3, mkMap(mkEntry(p1, p1Assignment), 
mkEntry(p2, p2Assignment)))
+                // consume from the beginning of the topic to read data from 
local and remote storage for partition 0
+                .expectFetchFromTieredStorage(broker0, topicA, p0, 2)
+                .consume(topicA, p0, 0L, 3, 2)
+
+                .expectLeader(topicA, p1, broker0, false)
+                .expectLeader(topicA, p2, broker1, false)
+
+                // produce events to partition 1
+                .expectSegmentToBeOffloaded(broker0, topicA, p1, 0, new 
KeyValueSpec("k0", "v0"))

Review Comment:
   It is good to prefix the key and values with partition info like 
"p0"/"p1"/"p2" so that the test verification checks for the respective messages 
in the partitions. 



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