@@ -984,14 +984,16 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment
984
984
brokerTopicStats .topicStats (log .topicPartition ().topic ()).remoteCopyRequestRate ().mark ();
985
985
brokerTopicStats .allTopicsStats ().remoteCopyRequestRate ().mark ();
986
986
Optional <CustomMetadata > customMetadata = Optional .empty ();
987
+
987
988
try {
988
989
customMetadata = remoteLogStorageManager .copyLogSegmentData (copySegmentStartedRlsm , segmentData );
989
990
} catch (RemoteStorageException e ) {
991
+ logger .info ("Copy failed, cleaning segment {}" , copySegmentStartedRlsm .remoteLogSegmentId ());
990
992
try {
991
- remoteLogStorageManager . deleteLogSegmentData (copySegmentStartedRlsm );
992
- logger .info ("Successfully cleaned segment {} after failing to copy segment " , segmentId );
993
+ deleteRemoteLogSegment (copySegmentStartedRlsm , ignored -> ! isCancelled () );
994
+ LOGGER .info ("Cleanup completed for segment {}" , copySegmentStartedRlsm . remoteLogSegmentId () );
993
995
} catch (RemoteStorageException e1 ) {
994
- logger . error ( "Error while cleaning segment {}, consider cleaning manually " , segmentId , e1 );
996
+ LOGGER . info ( "Cleanup failed, will retry later with segment {}: {} " , copySegmentStartedRlsm . remoteLogSegmentId () , e1 . getMessage () );
995
997
}
996
998
throw e ;
997
999
}
@@ -1003,17 +1005,18 @@ private void copyLogSegment(UnifiedLog log, LogSegment segment, RemoteLogSegment
1003
1005
long customMetadataSize = customMetadata .get ().value ().length ;
1004
1006
if (customMetadataSize > this .customMetadataSizeLimit ) {
1005
1007
CustomMetadataSizeLimitExceededException e = new CustomMetadataSizeLimitExceededException ();
1006
- logger .error ("Custom metadata size {} exceeds configured limit {}." +
1008
+ logger .info ("Custom metadata size {} exceeds configured limit {}." +
1007
1009
" Copying will be stopped and copied segment will be attempted to clean." +
1008
1010
" Original metadata: {}" ,
1009
1011
customMetadataSize , this .customMetadataSizeLimit , copySegmentStartedRlsm , e );
1012
+ // For deletion, we provide back the custom metadata by creating a new metadata object from the update.
1013
+ // However, the update itself will not be stored in this case.
1014
+ RemoteLogSegmentMetadata newMetadata = copySegmentStartedRlsm .createWithUpdates (copySegmentFinishedRlsm );
1010
1015
try {
1011
- // For deletion, we provide back the custom metadata by creating a new metadata object from the update.
1012
- // However, the update itself will not be stored in this case.
1013
- remoteLogStorageManager .deleteLogSegmentData (copySegmentStartedRlsm .createWithUpdates (copySegmentFinishedRlsm ));
1014
- logger .info ("Successfully cleaned segment after custom metadata size exceeded" );
1016
+ deleteRemoteLogSegment (newMetadata , ignored -> !isCancelled ());
1017
+ LOGGER .info ("Cleanup completed for segment {}" , newMetadata .remoteLogSegmentId ());
1015
1018
} catch (RemoteStorageException e1 ) {
1016
- logger . error ( "Error while cleaning segment after custom metadata size exceeded, consider cleaning manually" , e1 );
1019
+ LOGGER . info ( "Cleanup failed, will retry later with segment {}: {}" , newMetadata . remoteLogSegmentId () , e1 . getMessage () );
1017
1020
}
1018
1021
throw e ;
1019
1022
}
@@ -1070,7 +1073,6 @@ public RLMExpirationTask(TopicIdPartition topicIdPartition) {
1070
1073
1071
1074
@ Override
1072
1075
protected void execute (UnifiedLog log ) throws InterruptedException , RemoteStorageException , ExecutionException {
1073
- // Cleanup/delete expired remote log segments
1074
1076
cleanupExpiredRemoteLogSegments ();
1075
1077
}
1076
1078
@@ -1160,8 +1162,8 @@ private boolean isSegmentBreachByLogStartOffset(RemoteLogSegmentMetadata metadat
1160
1162
private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation (EpochEntry earliestEpochEntry ,
1161
1163
RemoteLogSegmentMetadata metadata )
1162
1164
throws RemoteStorageException , ExecutionException , InterruptedException {
1163
- boolean isSegmentDeleted = deleteRemoteLogSegment (metadata , ignored ->
1164
- metadata .segmentLeaderEpochs ().keySet ().stream ().allMatch (epoch -> epoch < earliestEpochEntry .epoch ));
1165
+ boolean isSegmentDeleted = deleteRemoteLogSegment (metadata ,
1166
+ ignored -> metadata .segmentLeaderEpochs ().keySet ().stream ().allMatch (epoch -> epoch < earliestEpochEntry .epoch ));
1165
1167
if (isSegmentDeleted ) {
1166
1168
logger .info ("Deleted remote log segment {} due to leader-epoch-cache truncation. " +
1167
1169
"Current earliest-epoch-entry: {}, segment-end-offset: {} and segment-epochs: {}" ,
@@ -1170,41 +1172,6 @@ private boolean deleteLogSegmentsDueToLeaderEpochCacheTruncation(EpochEntry earl
1170
1172
// No need to update the log-start-offset as these epochs/offsets are earlier to that value.
1171
1173
return isSegmentDeleted ;
1172
1174
}
1173
-
1174
- private boolean deleteRemoteLogSegment (RemoteLogSegmentMetadata segmentMetadata , Predicate <RemoteLogSegmentMetadata > predicate )
1175
- throws RemoteStorageException , ExecutionException , InterruptedException {
1176
- if (predicate .test (segmentMetadata )) {
1177
- logger .debug ("Deleting remote log segment {}" , segmentMetadata .remoteLogSegmentId ());
1178
-
1179
- String topic = segmentMetadata .topicIdPartition ().topic ();
1180
-
1181
- // Publish delete segment started event.
1182
- remoteLogMetadataManager .updateRemoteLogSegmentMetadata (
1183
- new RemoteLogSegmentMetadataUpdate (segmentMetadata .remoteLogSegmentId (), time .milliseconds (),
1184
- segmentMetadata .customMetadata (), RemoteLogSegmentState .DELETE_SEGMENT_STARTED , brokerId )).get ();
1185
-
1186
- brokerTopicStats .topicStats (topic ).remoteDeleteRequestRate ().mark ();
1187
- brokerTopicStats .allTopicsStats ().remoteDeleteRequestRate ().mark ();
1188
-
1189
- // Delete the segment in remote storage.
1190
- try {
1191
- remoteLogStorageManager .deleteLogSegmentData (segmentMetadata );
1192
- } catch (RemoteStorageException e ) {
1193
- brokerTopicStats .topicStats (topic ).failedRemoteDeleteRequestRate ().mark ();
1194
- brokerTopicStats .allTopicsStats ().failedRemoteDeleteRequestRate ().mark ();
1195
- throw e ;
1196
- }
1197
-
1198
- // Publish delete segment finished event.
1199
- remoteLogMetadataManager .updateRemoteLogSegmentMetadata (
1200
- new RemoteLogSegmentMetadataUpdate (segmentMetadata .remoteLogSegmentId (), time .milliseconds (),
1201
- segmentMetadata .customMetadata (), RemoteLogSegmentState .DELETE_SEGMENT_FINISHED , brokerId )).get ();
1202
- logger .debug ("Deleted remote log segment {}" , segmentMetadata .remoteLogSegmentId ());
1203
- return true ;
1204
- }
1205
- return false ;
1206
- }
1207
-
1208
1175
}
1209
1176
1210
1177
private void updateMetadataCountAndLogSizeWith (int metadataCount , long remoteLogSizeBytes ) {
@@ -1221,6 +1188,7 @@ private void updateRemoteDeleteLagWith(int segmentsLeftToDelete, long sizeOfDele
1221
1188
brokerTopicStats .recordRemoteDeleteLagBytes (topic , partition , sizeOfDeletableSegmentsBytes );
1222
1189
}
1223
1190
1191
+ /** Cleanup expired and dangling remote log segments. */
1224
1192
void cleanupExpiredRemoteLogSegments () throws RemoteStorageException , ExecutionException , InterruptedException {
1225
1193
if (isCancelled ()) {
1226
1194
logger .info ("Returning from remote log segments cleanup as the task state is changed" );
@@ -1297,6 +1265,12 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE
1297
1265
canProcess = false ;
1298
1266
continue ;
1299
1267
}
1268
+ // This works as retry mechanism for dangling remote segments that failed the deletion in previous attempts.
1269
+ // Rather than waiting for the retention to kick in, we cleanup early to avoid polluting the cache and possibly waste remote storage.
1270
+ if (RemoteLogSegmentState .DELETE_SEGMENT_STARTED .equals (metadata .state ())) {
1271
+ segmentsToDelete .add (metadata );
1272
+ continue ;
1273
+ }
1300
1274
if (RemoteLogSegmentState .DELETE_SEGMENT_FINISHED .equals (metadata .state ())) {
1301
1275
continue ;
1302
1276
}
@@ -1343,7 +1317,7 @@ void cleanupExpiredRemoteLogSegments() throws RemoteStorageException, ExecutionE
1343
1317
updateRemoteDeleteLagWith (segmentsLeftToDelete , sizeOfDeletableSegmentsBytes );
1344
1318
List <String > undeletedSegments = new ArrayList <>();
1345
1319
for (RemoteLogSegmentMetadata segmentMetadata : segmentsToDelete ) {
1346
- if (!remoteLogRetentionHandler . deleteRemoteLogSegment (segmentMetadata , x -> !isCancelled ())) {
1320
+ if (!deleteRemoteLogSegment (segmentMetadata , ignored -> !isCancelled ())) {
1347
1321
undeletedSegments .add (segmentMetadata .remoteLogSegmentId ().toString ());
1348
1322
} else {
1349
1323
sizeOfDeletableSegmentsBytes -= segmentMetadata .segmentSizeInBytes ();
@@ -1417,13 +1391,11 @@ private Optional<RetentionSizeData> buildRetentionSizeData(long retentionSize,
1417
1391
Iterator <RemoteLogSegmentMetadata > segmentsIterator = remoteLogMetadataManager .listRemoteLogSegments (topicIdPartition , epoch );
1418
1392
while (segmentsIterator .hasNext ()) {
1419
1393
RemoteLogSegmentMetadata segmentMetadata = segmentsIterator .next ();
1420
- // Only count the size of "COPY_SEGMENT_FINISHED" and "DELETE_SEGMENT_STARTED" state segments
1421
- // because "COPY_SEGMENT_STARTED" means copy didn't complete, and "DELETE_SEGMENT_FINISHED" means delete did complete.
1422
- // Note: there might be some "COPY_SEGMENT_STARTED" segments not counted here.
1423
- // Either they are being copied and will be counted next time or they are dangling and will be cleaned elsewhere,
1424
- // either way, this won't cause more segment deletion.
1425
- if (segmentMetadata .state ().equals (RemoteLogSegmentState .COPY_SEGMENT_FINISHED ) ||
1426
- segmentMetadata .state ().equals (RemoteLogSegmentState .DELETE_SEGMENT_STARTED )) {
1394
+ // Count only the size of segments in "COPY_SEGMENT_FINISHED" state because
1395
+ // "COPY_SEGMENT_STARTED" means copy didn't complete and we will count them later,
1396
+ // "DELETE_SEGMENT_STARTED" means deletion failed in the previous attempt and we will retry later,
1397
+ // "DELETE_SEGMENT_FINISHED" means deletion completed, so there is nothing to count.
1398
+ if (segmentMetadata .state ().equals (RemoteLogSegmentState .COPY_SEGMENT_FINISHED )) {
1427
1399
RemoteLogSegmentId segmentId = segmentMetadata .remoteLogSegmentId ();
1428
1400
if (!visitedSegmentIds .contains (segmentId ) && isRemoteSegmentWithinLeaderEpochs (segmentMetadata , logEndOffset , epochEntries )) {
1429
1401
remoteLogSizeBytes += segmentMetadata .segmentSizeInBytes ();
@@ -1463,6 +1435,41 @@ protected void execute(UnifiedLog log) throws InterruptedException, RemoteStorag
1463
1435
log .updateHighestOffsetInRemoteStorage (offsetAndEpoch .offset ());
1464
1436
}
1465
1437
}
1438
+
1439
+ private boolean deleteRemoteLogSegment (
1440
+ RemoteLogSegmentMetadata segmentMetadata ,
1441
+ Predicate <RemoteLogSegmentMetadata > predicate
1442
+ ) throws RemoteStorageException , ExecutionException , InterruptedException {
1443
+ if (predicate .test (segmentMetadata )) {
1444
+ LOGGER .debug ("Deleting remote log segment {}" , segmentMetadata .remoteLogSegmentId ());
1445
+ String topic = segmentMetadata .topicIdPartition ().topic ();
1446
+
1447
+ // Publish delete segment started event.
1448
+ remoteLogMetadataManager .updateRemoteLogSegmentMetadata (
1449
+ new RemoteLogSegmentMetadataUpdate (segmentMetadata .remoteLogSegmentId (), time .milliseconds (),
1450
+ segmentMetadata .customMetadata (), RemoteLogSegmentState .DELETE_SEGMENT_STARTED , brokerId )).get ();
1451
+
1452
+ brokerTopicStats .topicStats (topic ).remoteDeleteRequestRate ().mark ();
1453
+ brokerTopicStats .allTopicsStats ().remoteDeleteRequestRate ().mark ();
1454
+
1455
+ // Delete the segment in remote storage.
1456
+ try {
1457
+ remoteLogStorageManager .deleteLogSegmentData (segmentMetadata );
1458
+ } catch (RemoteStorageException e ) {
1459
+ brokerTopicStats .topicStats (topic ).failedRemoteDeleteRequestRate ().mark ();
1460
+ brokerTopicStats .allTopicsStats ().failedRemoteDeleteRequestRate ().mark ();
1461
+ throw e ;
1462
+ }
1463
+
1464
+ // Publish delete segment finished event.
1465
+ remoteLogMetadataManager .updateRemoteLogSegmentMetadata (
1466
+ new RemoteLogSegmentMetadataUpdate (segmentMetadata .remoteLogSegmentId (), time .milliseconds (),
1467
+ segmentMetadata .customMetadata (), RemoteLogSegmentState .DELETE_SEGMENT_FINISHED , brokerId )).get ();
1468
+ LOGGER .debug ("Deleted remote log segment {}" , segmentMetadata .remoteLogSegmentId ());
1469
+ return true ;
1470
+ }
1471
+ return false ;
1472
+ }
1466
1473
1467
1474
/**
1468
1475
* Returns true if the remote segment's epoch/offsets are within the leader epoch lineage of the partition.
0 commit comments