|
8 | 8 |
|
9 | 9 | package org.opensearch.index.translog;
|
10 | 10 |
|
| 11 | +import org.apache.logging.log4j.LogManager; |
11 | 12 | import org.apache.logging.log4j.Logger;
|
12 | 13 | import org.opensearch.cluster.service.ClusterService;
|
13 | 14 | import org.opensearch.common.blobstore.BlobMetadata;
|
|
33 | 34 | import java.util.Optional;
|
34 | 35 | import java.util.Set;
|
35 | 36 | import java.util.TreeSet;
|
| 37 | +import java.util.concurrent.atomic.AtomicLong; |
36 | 38 | import java.util.function.BooleanSupplier;
|
37 | 39 | import java.util.function.LongConsumer;
|
38 | 40 | import java.util.function.LongSupplier;
|
|
52 | 54 | */
|
53 | 55 | public class RemoteFsTimestampAwareTranslog extends RemoteFsTranslog {
|
54 | 56 |
|
| 57 | + private static Logger staticLogger = LogManager.getLogger(RemoteFsTimestampAwareTranslog.class); |
55 | 58 | private final Logger logger;
|
56 | 59 | private final Map<Long, String> metadataFilePinnedTimestampMap;
|
57 | 60 | // For metadata files, with no min generation in the name, we cache generation data to avoid multiple reads.
|
58 | 61 | private final Map<String, Tuple<Long, Long>> oldFormatMetadataFileGenerationMap;
|
| 62 | + private final Map<String, Tuple<Long, Long>> oldFormatMetadataFilePrimaryTermMap; |
| 63 | + private final AtomicLong minPrimaryTermInRemote = new AtomicLong(Long.MAX_VALUE); |
59 | 64 |
|
60 | 65 | public RemoteFsTimestampAwareTranslog(
|
61 | 66 | TranslogConfig config,
|
@@ -86,6 +91,7 @@ public RemoteFsTimestampAwareTranslog(
|
86 | 91 | logger = Loggers.getLogger(getClass(), shardId);
|
87 | 92 | this.metadataFilePinnedTimestampMap = new HashMap<>();
|
88 | 93 | this.oldFormatMetadataFileGenerationMap = new HashMap<>();
|
| 94 | + this.oldFormatMetadataFilePrimaryTermMap = new HashMap<>(); |
89 | 95 | }
|
90 | 96 |
|
91 | 97 | @Override
|
@@ -165,7 +171,11 @@ public void onResponse(List<BlobMetadata> blobMetadata) {
|
165 | 171 | return;
|
166 | 172 | }
|
167 | 173 |
|
168 |
| - List<String> metadataFilesToBeDeleted = getMetadataFilesToBeDeleted(metadataFiles); |
| 174 | + List<String> metadataFilesToBeDeleted = getMetadataFilesToBeDeleted( |
| 175 | + metadataFiles, |
| 176 | + metadataFilePinnedTimestampMap, |
| 177 | + logger |
| 178 | + ); |
169 | 179 |
|
170 | 180 | // If index is not deleted, make sure to keep latest metadata file
|
171 | 181 | if (indexDeleted == false) {
|
@@ -209,7 +219,7 @@ public void onResponse(List<BlobMetadata> blobMetadata) {
|
209 | 219 | oldFormatMetadataFileGenerationMap.keySet().retainAll(metadataFilesNotToBeDeleted);
|
210 | 220 |
|
211 | 221 | // Delete stale primary terms
|
212 |
| - deleteStaleRemotePrimaryTerms(metadataFiles); |
| 222 | + deleteStaleRemotePrimaryTerms(metadataFilesNotToBeDeleted); |
213 | 223 | } else {
|
214 | 224 | remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS);
|
215 | 225 | }
|
@@ -259,8 +269,16 @@ protected Set<Long> getGenerationsToBeDeleted(
|
259 | 269 | return generationsToBeDeleted;
|
260 | 270 | }
|
261 | 271 |
|
262 |
| - // Visible for testing |
263 | 272 | protected List<String> getMetadataFilesToBeDeleted(List<String> metadataFiles) {
|
| 273 | + return getMetadataFilesToBeDeleted(metadataFiles, metadataFilePinnedTimestampMap, logger); |
| 274 | + } |
| 275 | + |
| 276 | + // Visible for testing |
| 277 | + protected static List<String> getMetadataFilesToBeDeleted( |
| 278 | + List<String> metadataFiles, |
| 279 | + Map<Long, String> metadataFilePinnedTimestampMap, |
| 280 | + Logger logger |
| 281 | + ) { |
264 | 282 | Tuple<Long, Set<Long>> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps();
|
265 | 283 |
|
266 | 284 | // Keep files since last successful run of scheduler
|
@@ -351,27 +369,153 @@ protected Tuple<Long, Long> getMinMaxTranslogGenerationFromMetadataFile(
|
351 | 369 | }
|
352 | 370 | }
|
353 | 371 |
|
| 372 | + private void deleteStaleRemotePrimaryTerms(List<String> metadataFiles) { |
| 373 | + deleteStaleRemotePrimaryTerms( |
| 374 | + metadataFiles, |
| 375 | + translogTransferManager, |
| 376 | + oldFormatMetadataFilePrimaryTermMap, |
| 377 | + minPrimaryTermInRemote, |
| 378 | + logger |
| 379 | + ); |
| 380 | + } |
| 381 | + |
354 | 382 | /**
|
355 | 383 | * This method must be called only after there are valid generations to delete in trimUnreferencedReaders as it ensures
|
356 | 384 | * implicitly that minimum primary term in latest translog metadata in remote store is the current primary term.
|
357 | 385 | * <br>
|
358 | 386 | * This will also delete all stale translog metadata files from remote except the latest basis the metadata file comparator.
|
359 | 387 | */
|
360 |
| - private void deleteStaleRemotePrimaryTerms(List<String> metadataFiles) { |
| 388 | + protected static void deleteStaleRemotePrimaryTerms( |
| 389 | + List<String> metadataFiles, |
| 390 | + TranslogTransferManager translogTransferManager, |
| 391 | + Map<String, Tuple<Long, Long>> oldFormatMetadataFilePrimaryTermMap, |
| 392 | + AtomicLong minPrimaryTermInRemoteAtomicLong, |
| 393 | + Logger logger |
| 394 | + ) { |
361 | 395 | // The deletion of older translog files in remote store is on best-effort basis, there is a possibility that there
|
362 | 396 | // are older files that are no longer needed and should be cleaned up. In here, we delete all files that are part
|
363 | 397 | // of older primary term.
|
364 |
| - if (olderPrimaryCleaned.trySet(Boolean.TRUE)) { |
365 |
| - if (metadataFiles.isEmpty()) { |
366 |
| - logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); |
367 |
| - return; |
| 398 | + if (metadataFiles.isEmpty()) { |
| 399 | + logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); |
| 400 | + return; |
| 401 | + } |
| 402 | + Optional<Long> minPrimaryTermFromMetadataFiles = metadataFiles.stream().map(file -> { |
| 403 | + try { |
| 404 | + return getMinMaxPrimaryTermFromMetadataFile(file, translogTransferManager, oldFormatMetadataFilePrimaryTermMap).v1(); |
| 405 | + } catch (IOException e) { |
| 406 | + return Long.MAX_VALUE; |
| 407 | + } |
| 408 | + }).min(Long::compareTo); |
| 409 | + // First we delete all stale primary terms folders from remote store |
| 410 | + Long minPrimaryTermInRemote = getMinPrimaryTermInRemote(minPrimaryTermInRemoteAtomicLong, translogTransferManager, logger); |
| 411 | + if (minPrimaryTermFromMetadataFiles.get() > minPrimaryTermInRemote) { |
| 412 | + translogTransferManager.deletePrimaryTermsAsync(minPrimaryTermFromMetadataFiles.get()); |
| 413 | + minPrimaryTermInRemoteAtomicLong.set(minPrimaryTermFromMetadataFiles.get()); |
| 414 | + } else { |
| 415 | + logger.debug( |
| 416 | + "Skipping primary term cleanup. minimumReferencedPrimaryTerm = {}, minPrimaryTermInRemote = {}", |
| 417 | + minPrimaryTermFromMetadataFiles.get(), |
| 418 | + minPrimaryTermInRemote |
| 419 | + ); |
| 420 | + } |
| 421 | + } |
| 422 | + |
| 423 | + private static Long getMinPrimaryTermInRemote( |
| 424 | + AtomicLong minPrimaryTermInRemote, |
| 425 | + TranslogTransferManager translogTransferManager, |
| 426 | + Logger logger |
| 427 | + ) { |
| 428 | + if (minPrimaryTermInRemote.get() == Long.MAX_VALUE) { |
| 429 | + try { |
| 430 | + Set<Long> primaryTermsInRemote = translogTransferManager.listPrimaryTermsInRemote(); |
| 431 | + if (primaryTermsInRemote.isEmpty() == false) { |
| 432 | + Optional<Long> minPrimaryTerm = primaryTermsInRemote.stream().min(Long::compareTo); |
| 433 | + minPrimaryTerm.ifPresent(minPrimaryTermInRemote::set); |
| 434 | + } |
| 435 | + } catch (IOException e) { |
| 436 | + logger.error("Exception while listing primary terms in remote translog", e); |
| 437 | + } |
| 438 | + } |
| 439 | + return minPrimaryTermInRemote.get(); |
| 440 | + } |
| 441 | + |
| 442 | + protected static Tuple<Long, Long> getMinMaxPrimaryTermFromMetadataFile( |
| 443 | + String metadataFile, |
| 444 | + TranslogTransferManager translogTransferManager, |
| 445 | + Map<String, Tuple<Long, Long>> oldFormatMetadataFilePrimaryTermMap |
| 446 | + ) throws IOException { |
| 447 | + Tuple<Long, Long> minMaxPrimaryTermFromFileName = TranslogTransferMetadata.getMinMaxPrimaryTermFromFilename(metadataFile); |
| 448 | + if (minMaxPrimaryTermFromFileName != null) { |
| 449 | + return minMaxPrimaryTermFromFileName; |
| 450 | + } else { |
| 451 | + if (oldFormatMetadataFilePrimaryTermMap.containsKey(metadataFile)) { |
| 452 | + return oldFormatMetadataFilePrimaryTermMap.get(metadataFile); |
| 453 | + } else { |
| 454 | + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(metadataFile); |
| 455 | + long maxPrimaryTem = TranslogTransferMetadata.getPrimaryTermFromFileName(metadataFile); |
| 456 | + long minPrimaryTem = -1; |
| 457 | + if (metadata.getGenerationToPrimaryTermMapper() != null |
| 458 | + && metadata.getGenerationToPrimaryTermMapper().values().isEmpty() == false) { |
| 459 | + Optional<Long> primaryTerm = metadata.getGenerationToPrimaryTermMapper() |
| 460 | + .values() |
| 461 | + .stream() |
| 462 | + .map(s -> Long.parseLong(s)) |
| 463 | + .min(Long::compareTo); |
| 464 | + if (primaryTerm.isPresent()) { |
| 465 | + minPrimaryTem = primaryTerm.get(); |
| 466 | + } |
| 467 | + } |
| 468 | + Tuple<Long, Long> minMaxPrimaryTermTuple = new Tuple<>(minPrimaryTem, maxPrimaryTem); |
| 469 | + oldFormatMetadataFilePrimaryTermMap.put(metadataFile, minMaxPrimaryTermTuple); |
| 470 | + return minMaxPrimaryTermTuple; |
368 | 471 | }
|
369 |
| - Optional<Long> minPrimaryTerm = metadataFiles.stream() |
370 |
| - .map(file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[1])) |
371 |
| - .min(Long::compareTo); |
372 |
| - // First we delete all stale primary terms folders from remote store |
373 |
| - long minimumReferencedPrimaryTerm = minPrimaryTerm.get() - 1; |
374 |
| - translogTransferManager.deletePrimaryTermsAsync(minimumReferencedPrimaryTerm); |
375 | 472 | }
|
376 | 473 | }
|
| 474 | + |
| 475 | + public static void cleanup(TranslogTransferManager translogTransferManager) throws IOException { |
| 476 | + ActionListener<List<BlobMetadata>> listMetadataFilesListener = new ActionListener<>() { |
| 477 | + @Override |
| 478 | + public void onResponse(List<BlobMetadata> blobMetadata) { |
| 479 | + List<String> metadataFiles = blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()); |
| 480 | + |
| 481 | + try { |
| 482 | + if (metadataFiles.isEmpty()) { |
| 483 | + staticLogger.debug("No stale translog metadata files found"); |
| 484 | + return; |
| 485 | + } |
| 486 | + List<String> metadataFilesToBeDeleted = getMetadataFilesToBeDeleted(metadataFiles, new HashMap<>(), staticLogger); |
| 487 | + if (metadataFilesToBeDeleted.isEmpty()) { |
| 488 | + staticLogger.debug("No metadata files to delete"); |
| 489 | + return; |
| 490 | + } |
| 491 | + staticLogger.debug(() -> "metadataFilesToBeDeleted = " + metadataFilesToBeDeleted); |
| 492 | + |
| 493 | + // For all the files that we are keeping, fetch min and max generations |
| 494 | + List<String> metadataFilesNotToBeDeleted = new ArrayList<>(metadataFiles); |
| 495 | + metadataFilesNotToBeDeleted.removeAll(metadataFilesToBeDeleted); |
| 496 | + staticLogger.debug(() -> "metadataFilesNotToBeDeleted = " + metadataFilesNotToBeDeleted); |
| 497 | + |
| 498 | + // Delete stale metadata files |
| 499 | + translogTransferManager.deleteMetadataFilesAsync(metadataFilesToBeDeleted, () -> {}); |
| 500 | + |
| 501 | + // Delete stale primary terms |
| 502 | + deleteStaleRemotePrimaryTerms( |
| 503 | + metadataFilesNotToBeDeleted, |
| 504 | + translogTransferManager, |
| 505 | + new HashMap<>(), |
| 506 | + new AtomicLong(Long.MAX_VALUE), |
| 507 | + staticLogger |
| 508 | + ); |
| 509 | + } catch (Exception e) { |
| 510 | + staticLogger.error("Exception while cleaning up metadata and primary terms", e); |
| 511 | + } |
| 512 | + } |
| 513 | + |
| 514 | + @Override |
| 515 | + public void onFailure(Exception e) { |
| 516 | + staticLogger.error("Exception while cleaning up metadata and primary terms", e); |
| 517 | + } |
| 518 | + }; |
| 519 | + translogTransferManager.listTranslogMetadataFilesAsync(listMetadataFilesListener); |
| 520 | + } |
377 | 521 | }
|
0 commit comments