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


##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -771,12 +772,127 @@ void 
testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc
                         safeLongYammerMetricValue("RemoteLogSizeBytes")));
     }
 
+    @Test
+    void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() 
throws Exception {
+        long oldSegmentStartOffset = 0L;
+        long nextSegmentStartOffset = 150L;
+        int segmentCount = 3;
+        
when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition());
+        when(mockLog.parentDir()).thenReturn("dir1");
+
+        // leader epoch preparation
+        checkpoint.write(totalEpochEntries);
+        LeaderEpochFileCache cache = new 
LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint);
+        when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache));
+        
when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class),
 anyInt()))
+                .thenReturn(Optional.of(0L))
+                .thenReturn(Optional.of(nextSegmentStartOffset - 1));
+
+        File tempFile = TestUtils.tempFile();
+        File mockProducerSnapshotIndex = TestUtils.tempFile();
+        File tempDir = TestUtils.tempDirectory();
+        // create 2 log segments, with 0 and 150 as log start offset
+        LogSegment oldSegment = mock(LogSegment.class);
+        LogSegment activeSegment = mock(LogSegment.class);
+
+        when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset);
+        when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset);
+
+        FileRecords fileRecords = mock(FileRecords.class);
+        when(oldSegment.log()).thenReturn(fileRecords);
+        when(fileRecords.file()).thenReturn(tempFile);
+        when(fileRecords.sizeInBytes()).thenReturn(10);
+        when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset);
+
+        when(mockLog.activeSegment()).thenReturn(activeSegment);
+        when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset);
+        when(mockLog.logSegments(anyLong(), 
anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment,
 activeSegment)));
+
+        ProducerStateManager mockStateManager = 
mock(ProducerStateManager.class);
+        when(mockLog.producerStateManager()).thenReturn(mockStateManager);
+        
when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex));
+        when(mockLog.lastStableOffset()).thenReturn(250L);
+        when(mockLog.logEndOffset()).thenReturn(500L);
+        Map<String, Long> logProps = new HashMap<>();
+        logProps.put("retention.bytes", 100L);
+        logProps.put("retention.ms", -1L);
+        LogConfig logConfig = new LogConfig(logProps);
+        when(mockLog.config()).thenReturn(logConfig);
+
+        OffsetIndex idx = 
LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, 
oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get();
+        TimeIndex timeIdx = 
LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, 
""), oldSegmentStartOffset, 1500).get();
+        File txnFile = UnifiedLog.transactionIndexFile(tempDir, 
oldSegmentStartOffset, "");
+        txnFile.createNewFile();
+        TransactionIndex txnIndex = new 
TransactionIndex(oldSegmentStartOffset, txnFile);
+        when(oldSegment.timeIndex()).thenReturn(timeIdx);
+        when(oldSegment.offsetIndex()).thenReturn(idx);
+        when(oldSegment.txnIndex()).thenReturn(txnIndex);
+
+        CompletableFuture<Void> dummyFuture = new CompletableFuture<>();
+        dummyFuture.complete(null);
+        
when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture);
+        
when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture);
+
+        CountDownLatch copyLogSegmentLatch = new CountDownLatch(1);
+        doAnswer(ans -> {
+            // waiting for verification
+            copyLogSegmentLatch.await(5000, TimeUnit.MILLISECONDS);
+            return Optional.empty();
+        
}).when(remoteStorageManager).copyLogSegmentData(any(RemoteLogSegmentMetadata.class),
 any(LogSegmentData.class));
+
+        Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition);
+        List<RemoteLogSegmentMetadata> list = 
listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, 
RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        // return 3 metadata and then return 0 to simulate all segments are 
deleted

Review Comment:
   ```suggestion
           // return the metadataList 3 times, then return empty list to 
simulate all segments are deleted
   ```



##########
core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java:
##########
@@ -771,12 +772,127 @@ void 
testRemoteLogManagerTasksAvgIdlePercentAndMetadataCountMetrics() throws Exc
                         safeLongYammerMetricValue("RemoteLogSizeBytes")));
     }
 
+    @Test
+    void testRemoteLogTaskUpdateRemoteLogSegmentMetadataAfterLogDirChanged() 
throws Exception {
+        long oldSegmentStartOffset = 0L;
+        long nextSegmentStartOffset = 150L;
+        int segmentCount = 3;
+        
when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition());
+        when(mockLog.parentDir()).thenReturn("dir1");
+
+        // leader epoch preparation
+        checkpoint.write(totalEpochEntries);
+        LeaderEpochFileCache cache = new 
LeaderEpochFileCache(leaderTopicIdPartition.topicPartition(), checkpoint);
+        when(mockLog.leaderEpochCache()).thenReturn(Option.apply(cache));
+        
when(remoteLogMetadataManager.highestOffsetForEpoch(any(TopicIdPartition.class),
 anyInt()))
+                .thenReturn(Optional.of(0L))
+                .thenReturn(Optional.of(nextSegmentStartOffset - 1));
+
+        File tempFile = TestUtils.tempFile();
+        File mockProducerSnapshotIndex = TestUtils.tempFile();
+        File tempDir = TestUtils.tempDirectory();
+        // create 2 log segments, with 0 and 150 as log start offset
+        LogSegment oldSegment = mock(LogSegment.class);
+        LogSegment activeSegment = mock(LogSegment.class);
+
+        when(oldSegment.baseOffset()).thenReturn(oldSegmentStartOffset);
+        when(activeSegment.baseOffset()).thenReturn(nextSegmentStartOffset);
+
+        FileRecords fileRecords = mock(FileRecords.class);
+        when(oldSegment.log()).thenReturn(fileRecords);
+        when(fileRecords.file()).thenReturn(tempFile);
+        when(fileRecords.sizeInBytes()).thenReturn(10);
+        when(oldSegment.readNextOffset()).thenReturn(nextSegmentStartOffset);
+
+        when(mockLog.activeSegment()).thenReturn(activeSegment);
+        when(mockLog.logStartOffset()).thenReturn(oldSegmentStartOffset);
+        when(mockLog.logSegments(anyLong(), 
anyLong())).thenReturn(JavaConverters.collectionAsScalaIterable(Arrays.asList(oldSegment,
 activeSegment)));
+
+        ProducerStateManager mockStateManager = 
mock(ProducerStateManager.class);
+        when(mockLog.producerStateManager()).thenReturn(mockStateManager);
+        
when(mockStateManager.fetchSnapshot(anyLong())).thenReturn(Optional.of(mockProducerSnapshotIndex));
+        when(mockLog.lastStableOffset()).thenReturn(250L);
+        when(mockLog.logEndOffset()).thenReturn(500L);
+        Map<String, Long> logProps = new HashMap<>();
+        logProps.put("retention.bytes", 100L);
+        logProps.put("retention.ms", -1L);
+        LogConfig logConfig = new LogConfig(logProps);
+        when(mockLog.config()).thenReturn(logConfig);
+
+        OffsetIndex idx = 
LazyIndex.forOffset(LogFileUtils.offsetIndexFile(tempDir, 
oldSegmentStartOffset, ""), oldSegmentStartOffset, 1000).get();
+        TimeIndex timeIdx = 
LazyIndex.forTime(LogFileUtils.timeIndexFile(tempDir, oldSegmentStartOffset, 
""), oldSegmentStartOffset, 1500).get();
+        File txnFile = UnifiedLog.transactionIndexFile(tempDir, 
oldSegmentStartOffset, "");
+        txnFile.createNewFile();
+        TransactionIndex txnIndex = new 
TransactionIndex(oldSegmentStartOffset, txnFile);
+        when(oldSegment.timeIndex()).thenReturn(timeIdx);
+        when(oldSegment.offsetIndex()).thenReturn(idx);
+        when(oldSegment.txnIndex()).thenReturn(txnIndex);
+
+        CompletableFuture<Void> dummyFuture = new CompletableFuture<>();
+        dummyFuture.complete(null);
+        
when(remoteLogMetadataManager.addRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadata.class))).thenReturn(dummyFuture);
+        
when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class))).thenReturn(dummyFuture);
+
+        CountDownLatch copyLogSegmentLatch = new CountDownLatch(1);
+        doAnswer(ans -> {
+            // waiting for verification
+            copyLogSegmentLatch.await(5000, TimeUnit.MILLISECONDS);
+            return Optional.empty();
+        
}).when(remoteStorageManager).copyLogSegmentData(any(RemoteLogSegmentMetadata.class),
 any(LogSegmentData.class));
+
+        Partition mockLeaderPartition = mockPartition(leaderTopicIdPartition);
+        List<RemoteLogSegmentMetadata> list = 
listRemoteLogSegmentMetadata(leaderTopicIdPartition, segmentCount, 100, 1024, 
RemoteLogSegmentState.COPY_SEGMENT_FINISHED);

Review Comment:
   can we rename the variable `list` to `metadataList`?



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