divijvaidya commented on code in PR #14727:
URL: https://github.com/apache/kafka/pull/14727#discussion_r1391297279


##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -988,30 +997,33 @@ void cleanupExpiredRemoteLogSegments() throws 
RemoteStorageException, ExecutionE
                         return;
                     }
                     RemoteLogSegmentMetadata metadata = 
segmentsIterator.next();
-                    if (segmentsToDelete.contains(metadata)) {
-                        continue;
-                    }
-                    // When the log-start-offset is moved by the user, the 
leader-epoch-checkpoint file gets truncated
-                    // as per the log-start-offset. Until the 
rlm-cleaner-thread runs in the next iteration, those
-                    // remote log segments won't be removed. The 
`isRemoteSegmentWithinLeaderEpoch` validates whether
-                    // the epochs present in the segment lies in the 
checkpoint file. It will always return false
-                    // since the checkpoint file was already truncated.
-                    boolean shouldDeleteSegment = 
remoteLogRetentionHandler.isSegmentBreachByLogStartOffset(
+
+                    if 
(SEGMENT_DELETION_VALID_STATES.contains(metadata.state())) {

Review Comment:
   since there is no "else" part to it, perhaps short circuiting the if 
condition with 
   ```
   if (!SEGMENT_DELETION_VALID_STATES.contains(metadata.state())) {
      continue;
   }
   ```
   would provide more readability. 
   
   (I don't have a strong opinion on this, hence, let me know if you wish to 
keep it as as it)



##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -133,6 +134,10 @@ public class RemoteLogManager implements Closeable {
 
     private static final Logger LOGGER = 
LoggerFactory.getLogger(RemoteLogManager.class);
     private static final String REMOTE_LOG_READER_THREAD_NAME_PREFIX = 
"remote-log-reader";
+    private static final Set<RemoteLogSegmentState> 
SEGMENT_DELETION_VALID_STATES = Collections.unmodifiableSet(EnumSet.of(

Review Comment:
   In this diagram: 
https://github.com/apache/kafka/blob/49d3122d425171b6a59a2b6f02d3fe63d3ac2397/storage/api/src/main/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManager.java#L74
 copy_started -> delete_started is also a valid transition. Although, I can't 
think of a scenario where this would be valid because we always complete copy 
before calling expiration. Are we missing something here by not including 
copy_started?



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