showuon commented on code in PR #14049:
URL: https://github.com/apache/kafka/pull/14049#discussion_r1274908196


##########
storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java:
##########
@@ -168,4 +169,64 @@ private void waitUntilConsumerCatchesup(TopicIdPartition 
newLeaderTopicIdPartiti
         }
     }
 
+    @Test
+    public void testRemoteLogSizeCalculationForUnknownTopicIdPartitionThrows() 
{
+        TopicIdPartition topicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition("singleton", 0));
+        Assertions.assertThrows(RemoteResourceNotFoundException.class, () -> 
topicBasedRlmm().remoteLogSize(topicIdPartition, 0));
+    }
+
+    @Test
+    public void testRemoteLogSizeCalculationWithSegmentsOfTheSameEpoch() 
throws RemoteStorageException, TimeoutException {
+        TopicIdPartition topicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition("singleton", 0));
+        TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager 
= topicBasedRlmm();
+
+        RemoteLogSegmentMetadata firstSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, 
Collections.singletonMap(0, 0L));
+        RemoteLogSegmentMetadata secondSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, 
Collections.singletonMap(0, 0L));
+        RemoteLogSegmentMetadata thirdSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, 
Collections.singletonMap(0, 0L));
+
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(firstSegmentMetadata);
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(secondSegmentMetadata);
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(thirdSegmentMetadata);
+
+        
topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges(Collections.singleton(topicIdPartition),
 Collections.emptySet());
+
+        // RemoteLogSegmentMetadata events are already published, and 
topicBasedRlmm's consumer manager will start
+        // fetching those events and build the cache.
+        waitUntilConsumerCatchesup(topicIdPartition, topicIdPartition, 
30_000L);
+
+        Long remoteLogSize = 
topicBasedRemoteLogMetadataManager.remoteLogSize(topicIdPartition, 0);
+
+        Assertions.assertEquals(SEG_SIZE * 6, remoteLogSize);
+    }
+
+    @Test
+    public void testRemoteLogSizeCalculationWithSegmentsOfDifferentEpochs() 
throws RemoteStorageException, TimeoutException {
+        TopicIdPartition topicIdPartition = new 
TopicIdPartition(Uuid.randomUuid(), new TopicPartition("singleton", 0));
+        TopicBasedRemoteLogMetadataManager topicBasedRemoteLogMetadataManager 
= topicBasedRlmm();
+
+        RemoteLogSegmentMetadata firstSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                0, 100, -1L, 0, time.milliseconds(), SEG_SIZE, 
Collections.singletonMap(0, 0L));
+        RemoteLogSegmentMetadata secondSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                100, 200, -1L, 0, time.milliseconds(), SEG_SIZE * 2, 
Collections.singletonMap(1, 100L));
+        RemoteLogSegmentMetadata thirdSegmentMetadata = new 
RemoteLogSegmentMetadata(new RemoteLogSegmentId(topicIdPartition, 
Uuid.randomUuid()),
+                200, 300, -1L, 0, time.milliseconds(), SEG_SIZE * 3, 
Collections.singletonMap(2, 200L));
+
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(firstSegmentMetadata);
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(secondSegmentMetadata);
+        
topicBasedRemoteLogMetadataManager.addRemoteLogSegmentMetadata(thirdSegmentMetadata);
+
+        
topicBasedRemoteLogMetadataManager.onPartitionLeadershipChanges(Collections.singleton(topicIdPartition),
 Collections.emptySet());
+
+        // RemoteLogSegmentMetadata events are already published, and 
topicBasedRlmm's consumer manager will start
+        // fetching those events and build the cache.
+        waitUntilConsumerCatchesup(topicIdPartition, topicIdPartition, 
30_000L);
+
+        Assertions.assertEquals(SEG_SIZE, 
topicBasedRemoteLogMetadataManager.remoteLogSize(topicIdPartition, 0));
+        Assertions.assertEquals(SEG_SIZE * 2, 
topicBasedRemoteLogMetadataManager.remoteLogSize(topicIdPartition, 1));
+        Assertions.assertEquals(SEG_SIZE * 3, 
topicBasedRemoteLogMetadataManager.remoteLogSize(topicIdPartition, 2));

Review Comment:
   Could we also verify what will return when the epoch is not out of range (I 
think it's 0?). We can add here or maybe add a new test?



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