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


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -1254,7 +1265,10 @@ void cleanupExpiredRemoteLogSegments() throws 
RemoteStorageException, ExecutionE
                         canProcess = false;
                         continue;
                     }
-                    if 
(RemoteLogSegmentState.DELETE_SEGMENT_FINISHED.equals(metadata.state())) {
+                    // skip the COPY_SEGMENT_STARTED segments since they might 
be the dangling segments that failed before
+                    // and blocks the normal segment deletion, ex: it failed 
`isRemoteSegmentWithinLeaderEpochs` check... etc
+                    if 
(RemoteLogSegmentState.DELETE_SEGMENT_FINISHED.equals(metadata.state()) ||
+                            
RemoteLogSegmentState.COPY_SEGMENT_STARTED.equals(metadata.state())) {

Review Comment:
   Ok, got the reason for including the `COPY_SEGMENT_STARTED` state together 
with this check. When the dangling segments gets deleted in the copy-phase, the 
event state is still set to "COPY_SEGMENT_STARTED" and the expiry thread will 
call the RemoteStorageManager#deleteLogSegmentData again. For my understanding, 
why are we not changing the state from `COPY_SEGMENT_STARTED` -> 
`DELETE_SEGMENT_STARTED` -> `DELETE_SEGMENT_FINISHED`?
   
   I think it should be ok, since we mentioned the below in the javadoc 
contract:
   
   
https://sourcegraph.com/github.com/apache/kafka@trunk/-/blob/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteStorageManager.java?L146
   ```
        * This operation is expected to be idempotent. If resources are not 
found, it is not expected to
        * throw {@link RemoteResourceNotFoundException} as it may be already 
removed from a previous attempt
   ```



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