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


##########
storage/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogManager.java:
##########
@@ -1044,6 +1049,9 @@ private void copyLogSegment(UnifiedLog log, LogSegment 
segment, RemoteLogSegment
             
             try {
                 customMetadata = 
remoteStorageManagerPlugin.get().copyLogSegmentData(copySegmentStartedRlsm, 
segmentData);
+            } catch (RetriableRemoteStorageException e) {
+                logger.info("Copy failed with retriable error for segment {}", 
copySegmentStartedRlsm.remoteLogSegmentId());
+                throw e;
             } catch (RemoteStorageException e) {
                 logger.info("Copy failed, cleaning segment {}", 
copySegmentStartedRlsm.remoteLogSegmentId());
                 try {

Review Comment:
   In L1058 below, when `deleteRemoteLogSegment` throws 
`RetriableRemoteStorageException`, we don't catch it. Then, the exception 
thrown will be the one from `deleteRemoteLogSegment`, not the one from 
`copyLogSegmentData`. With that, the `RetriableRemoteStorageException` from 
`deleteRemoteLogSegment` could be caught in 
[L1009](https://github.com/apache/kafka/pull/21150/files#diff-b5803060ec32fdfea597965205edfaa9949566a097f7017f4800ea4dee86ea9bR1009)
 above and not update the metrics as expected. Maybe we should catch all 
exceptions thrown from L1058 because we want to retry later?



##########
storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogManagerTest.java:
##########
@@ -2840,6 +2926,61 @@ public void testFailedDeleteExpiredSegments(long 
retentionSize,
         verify(remoteStorageManager).deleteLogSegmentData(metadataList.get(0));
     }
 
+    @ParameterizedTest(name = 
"testDeleteSegmentFailureWithRetriableExceptionShouldNotUpdateMetrics 
retentionSize={0} retentionMs={1}")
+    @CsvSource(value = {"0, -1", "-1, 0"})
+    public void 
testDeleteSegmentFailureWithRetriableExceptionShouldNotUpdateMetrics(long 
retentionSize,
+                                                                               
      long retentionMs) throws RemoteStorageException, ExecutionException, 
InterruptedException {
+        Map<String, Long> logProps = new HashMap<>();
+        logProps.put("retention.bytes", retentionSize);
+        logProps.put("retention.ms", retentionMs);
+        LogConfig mockLogConfig = new LogConfig(logProps);
+        when(mockLog.config()).thenReturn(mockLogConfig);
+
+        List<EpochEntry> epochEntries = List.of(epochEntry0);
+        checkpoint.write(epochEntries);
+        LeaderEpochFileCache cache = new LeaderEpochFileCache(tp, checkpoint, 
scheduler);
+        when(mockLog.leaderEpochCache()).thenReturn(cache);
+
+        
when(mockLog.topicPartition()).thenReturn(leaderTopicIdPartition.topicPartition());
+        when(mockLog.logEndOffset()).thenReturn(200L);
+
+        List<RemoteLogSegmentMetadata> metadataList =
+            listRemoteLogSegmentMetadata(leaderTopicIdPartition, 1, 100, 1024, 
RemoteLogSegmentState.COPY_SEGMENT_FINISHED);
+        
when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition))
+            .thenReturn(metadataList.iterator());
+        
when(remoteLogMetadataManager.listRemoteLogSegments(leaderTopicIdPartition, 0))
+            .thenAnswer(ans -> metadataList.iterator());
+        
when(remoteLogMetadataManager.updateRemoteLogSegmentMetadata(any(RemoteLogSegmentMetadataUpdate.class)))
+            .thenReturn(CompletableFuture.runAsync(() -> { }));
+
+        // Verify the metrics for remote deletes and for failures is zero 
before attempt to delete segments
+        assertEquals(0, 
brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).remoteDeleteRequestRate().count());
+        assertEquals(0, 
brokerTopicStats.topicStats(leaderTopicIdPartition.topic()).failedRemoteDeleteRequestRate().count());
+        // Verify aggregate metrics
+        assertEquals(0, 
brokerTopicStats.allTopicsStats().remoteDeleteRequestRate().count());
+        assertEquals(0, 
brokerTopicStats.allTopicsStats().failedRemoteDeleteRequestRate().count());
+
+        RemoteLogManager.RLMExpirationTask task = remoteLogManager.new 
RLMExpirationTask(leaderTopicIdPartition);
+        doThrow(new RetriableRemoteStorageException("Failed to delete segment 
with retriable 
exception")).when(remoteStorageManager).deleteLogSegmentData(any());
+        assertThrows(RemoteStorageException.class, 
task::cleanupExpiredRemoteLogSegments);

Review Comment:
   Why do we throw `RemoteStorageException` here, instead of 
`RetriableRemoteStorageException`?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to