kamalcph commented on code in PR #14330:
URL: https://github.com/apache/kafka/pull/14330#discussion_r1317068198
##########
core/src/main/java/kafka/log/remote/RemoteLogManager.java:
##########
@@ -853,35 +853,50 @@ public boolean
deleteRetentionTimeBreachedSegments(RemoteLogSegmentMetadata meta
return isSegmentDeleted;
}
- private boolean
deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata, long
startOffset)
+ private boolean
deleteLogStartOffsetBreachedSegments(RemoteLogSegmentMetadata metadata,
+ long
logStartOffset,
+
NavigableMap<Integer, Long> leaderEpochEntries)
throws RemoteStorageException, ExecutionException,
InterruptedException {
- boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x
-> startOffset > x.endOffset());
- if (isSegmentDeleted && retentionSizeData.isPresent()) {
- remainingBreachedSize = Math.max(0, remainingBreachedSize
- metadata.segmentSizeInBytes());
- logger.info("Deleted remote log segment {} due to log
start offset {} breach", metadata.remoteLogSegmentId(), startOffset);
+ boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
ignored -> {
+ if (!leaderEpochEntries.isEmpty()) {
+ // Note that `logStartOffset` and
`leaderEpochEntries.firstEntry().getValue()` should be same
+ Integer firstEpoch = leaderEpochEntries.firstKey();
+ return
metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <=
firstEpoch)
+ && metadata.endOffset() < logStartOffset;
+ }
+ return false;
+ });
+ if (isSegmentDeleted) {
+ remainingBreachedSize = retentionSizeData.isPresent() ?
+ Math.max(0, remainingBreachedSize -
metadata.segmentSizeInBytes()) : 0;
+ logger.info("Deleted remote log segment {} due to
log-start-offset {} breach. " +
+ "Current earliest-epoch-entry: {},
segment-end-offset: {} and segment-epochs: {}",
+ metadata.remoteLogSegmentId(), logStartOffset,
leaderEpochEntries.firstEntry(),
+ metadata.endOffset(),
metadata.segmentLeaderEpochs());
}
-
return isSegmentDeleted;
}
// It removes the segments beyond the current leader's earliest
epoch. Those segments are considered as
// unreferenced because they are not part of the current leader
epoch lineage.
- private boolean
deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry,
RemoteLogSegmentMetadata metadata) throws RemoteStorageException,
ExecutionException, InterruptedException {
- boolean isSegmentDeleted = deleteRemoteLogSegment(metadata, x
->
-
x.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <
earliestEpochEntry.epoch));
+ private boolean
deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earliestEpochEntry,
+
RemoteLogSegmentMetadata metadata)
+ throws RemoteStorageException, ExecutionException,
InterruptedException {
+ boolean isSegmentDeleted = deleteRemoteLogSegment(metadata,
ignored ->
+
metadata.segmentLeaderEpochs().keySet().stream().allMatch(epoch -> epoch <
earliestEpochEntry.epoch));
if (isSegmentDeleted) {
- logger.info("Deleted remote log segment {} due to leader
epoch cache truncation. Current earliest epoch: {}, segmentEndOffset: {} and
segmentEpochs: {}",
+ logger.info("Deleted remote log segment {} due to
leader-epoch-cache truncation. " +
+ "Current earliest-epoch-entry: {},
segment-end-offset: {} and segment-epochs: {}",
metadata.remoteLogSegmentId(), earliestEpochEntry,
metadata.endOffset(), metadata.segmentLeaderEpochs().keySet());
}
-
// No need to update the log-start-offset as these
epochs/offsets are earlier to that value.
return isSegmentDeleted;
}
private boolean deleteRemoteLogSegment(RemoteLogSegmentMetadata
segmentMetadata, Predicate<RemoteLogSegmentMetadata> predicate)
throws RemoteStorageException, ExecutionException,
InterruptedException {
if (predicate.test(segmentMetadata)) {
- logger.info("Deleting remote log segment {}",
segmentMetadata.remoteLogSegmentId());
+ logger.debug("Deleting remote log segment {}",
segmentMetadata.remoteLogSegmentId());
Review Comment:
Reduced the logger level to `debug` as we have one more log statement for
each segment deletion with purpose.
--
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]