From 6af931ad28e2a9a22a5391a7d6e5eb85c833574a Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Mon, 12 Aug 2024 23:23:39 +0530 Subject: [PATCH] Translog GC changes Signed-off-by: Sachin Kale --- .../index/translog/RemoteFsTranslog.java | 287 ++++++++++++++++-- .../opensearch/index/translog/Translog.java | 4 + .../transfer/TranslogTransferManager.java | 98 ++++-- .../indices/RemoteStoreSettings.java | 5 + .../index/translog/RemoteFsTranslogTests.java | 82 ++++- .../TranslogTransferManagerTests.java | 121 ++++++++ 6 files changed, 537 insertions(+), 60 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index c533a31c310c7..d3ce40a00d101 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -11,15 +11,19 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.SetOnce; +import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.collect.Tuple; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.logging.Loggers; import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.translog.transfer.BlobStoreTransferService; @@ -30,6 +34,7 @@ import org.opensearch.index.translog.transfer.TranslogTransferMetadata; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.indices.RemoteStoreSettings; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.threadpool.ThreadPool; @@ -39,11 +44,16 @@ import java.nio.file.Files; import java.nio.file.NoSuchFileException; import java.nio.file.Path; +import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.TreeSet; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -51,10 +61,12 @@ import java.util.function.BooleanSupplier; import java.util.function.LongConsumer; import java.util.function.LongSupplier; +import java.util.stream.Collectors; import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.DATA; import static org.opensearch.index.remote.RemoteStoreEnums.DataType.METADATA; +import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; /** * A Translog implementation which syncs local FS with a remote store @@ -92,6 +104,8 @@ public class RemoteFsTranslog extends Translog { private final Semaphore syncPermit = new Semaphore(SYNC_PERMIT); private final AtomicBoolean pauseSync = new AtomicBoolean(false); private final boolean isTranslogMetadataEnabled; + private final Map metadataFilePinnedTimestampMap; + private final Map> metadataFileGenerationMap; public RemoteFsTranslog( TranslogConfig config, @@ -105,6 +119,36 @@ public RemoteFsTranslog( BooleanSupplier startedPrimarySupplier, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings + ) throws IOException { + this( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + threadPool, + startedPrimarySupplier, + remoteTranslogTransferTracker, + remoteStoreSettings, + -1 + ); + } + + public RemoteFsTranslog( + TranslogConfig config, + String translogUUID, + TranslogDeletionPolicy deletionPolicy, + LongSupplier globalCheckpointSupplier, + LongSupplier primaryTermSupplier, + LongConsumer persistedSequenceNumberConsumer, + BlobStoreRepository blobStoreRepository, + ThreadPool threadPool, + BooleanSupplier startedPrimarySupplier, + RemoteTranslogTransferTracker remoteTranslogTransferTracker, + RemoteStoreSettings remoteStoreSettings, + long timestamp ) throws IOException { super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); logger = Loggers.getLogger(getClass(), shardId); @@ -112,6 +156,8 @@ public RemoteFsTranslog( this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); + this.metadataFilePinnedTimestampMap = new HashMap<>(); + this.metadataFileGenerationMap = new HashMap<>(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -123,7 +169,7 @@ public RemoteFsTranslog( isTranslogMetadataEnabled ); try { - download(translogTransferManager, location, logger, config.shouldSeedRemote()); + download(translogTransferManager, location, logger, config.shouldSeedRemote(), timestamp); Checkpoint checkpoint = readCheckpoint(location); logger.info("Downloaded data from remote translog till maxSeqNo = {}", checkpoint.maxSeqNo); this.readers.addAll(recoverFromFiles(checkpoint)); @@ -174,6 +220,32 @@ public static void download( Logger logger, boolean seedRemote, boolean isTranslogMetadataEnabled + ) throws IOException { + download( + repository, + shardId, + threadPool, + location, + pathStrategy, + remoteStoreSettings, + logger, + seedRemote, + isTranslogMetadataEnabled, + -1 + ); + } + + public static void download( + Repository repository, + ShardId shardId, + ThreadPool threadPool, + Path location, + RemoteStorePathStrategy pathStrategy, + RemoteStoreSettings remoteStoreSettings, + Logger logger, + boolean seedRemote, + boolean isTranslogMetadataEnabled, + long timestamp ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, @@ -195,11 +267,12 @@ public static void download( remoteStoreSettings, isTranslogMetadataEnabled ); - RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote); + RemoteFsTranslog.download(translogTransferManager, location, logger, seedRemote, timestamp); logger.trace(remoteTranslogTransferTracker.toString()); } - static void download(TranslogTransferManager translogTransferManager, Path location, Logger logger, boolean seedRemote) + // Visible for testing + static void download(TranslogTransferManager translogTransferManager, Path location, Logger logger, boolean seedRemote, long timestamp) throws IOException { /* In Primary to Primary relocation , there can be concurrent upload and download of translog. @@ -213,7 +286,7 @@ static void download(TranslogTransferManager translogTransferManager, Path locat boolean success = false; long startTimeMs = System.currentTimeMillis(); try { - downloadOnce(translogTransferManager, location, logger, seedRemote); + downloadOnce(translogTransferManager, location, logger, seedRemote, timestamp); success = true; return; } catch (FileNotFoundException | NoSuchFileException e) { @@ -227,13 +300,18 @@ static void download(TranslogTransferManager translogTransferManager, Path locat throw ex; } - private static void downloadOnce(TranslogTransferManager translogTransferManager, Path location, Logger logger, boolean seedRemote) - throws IOException { + private static void downloadOnce( + TranslogTransferManager translogTransferManager, + Path location, + Logger logger, + boolean seedRemote, + long timestamp + ) throws IOException { logger.debug("Downloading translog files from remote"); RemoteTranslogTransferTracker statsTracker = translogTransferManager.getRemoteTranslogTransferTracker(); long prevDownloadBytesSucceeded = statsTracker.getDownloadBytesSucceeded(); long prevDownloadTimeInMillis = statsTracker.getTotalDownloadTimeInMillis(); - TranslogTransferMetadata translogMetadata = translogTransferManager.readMetadata(); + TranslogTransferMetadata translogMetadata = translogTransferManager.readMetadata(timestamp); if (translogMetadata != null) { if (Files.notExists(location)) { Files.createDirectories(location); @@ -538,12 +616,34 @@ public void trimUnreferencedReaders() throws IOException { // clean up local translog files and updates readers super.trimUnreferencedReaders(); + // Update file tracker to reflect local translog state + Optional minLiveGeneration = readers.stream().map(BaseTranslogReader::getGeneration).min(Long::compareTo); + if (minLiveGeneration.isPresent()) { + List staleFilesInTracker = new ArrayList<>(); + for (String file : fileTransferTracker.allUploaded()) { + if (file.endsWith(TRANSLOG_FILE_SUFFIX)) { + long generation = Translog.parseIdFromFileName(file); + if (generation < minLiveGeneration.get()) { + staleFilesInTracker.add(file); + staleFilesInTracker.add(Translog.getCommitCheckpointFileName(generation)); + } + } + fileTransferTracker.delete(staleFilesInTracker); + } + } + // This is to ensure that after the permits are acquired during primary relocation, there are no further modification on remote // store. if (startedPrimarySupplier.getAsBoolean() == false || pauseSync.get()) { return; } + // This is to fail fast and avoid listing md files un-necessarily. + if (RemoteStoreUtils.isPinnedTimestampStateStale()) { + logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); + return; + } + // Since remote generation deletion is async, this ensures that only one generation deletion happens at a time. // Remote generations involves 2 async operations - 1) Delete translog generation files 2) Delete metadata files // We try to acquire 2 permits and if we can not, we return from here itself. @@ -551,21 +651,155 @@ public void trimUnreferencedReaders() throws IOException { return; } - // cleans up remote translog files not referenced in latest uploaded metadata. - // This enables us to restore translog from the metadata in case of failover or relocation. - Set generationsToDelete = new HashSet<>(); - for (long generation = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); generation >= 0; generation--) { - if (fileTransferTracker.uploaded(Translog.getFilename(generation)) == false) { - break; + ActionListener> listMetadataFilesListener = new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + List metadataFiles = blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()); + Set generationsToDelete = new HashSet<>(); + try { + if (metadataFiles.size() <= 1) { + logger.debug("No stale translog metadata files found"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + // 1. Check last fetch status of pinned timestamps. If stale, return. + if (RemoteStoreUtils.isPinnedTimestampStateStale()) { + logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + // 2. Get md files matching pinned timestamps + Tuple> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps(); + Set implicitLockedFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( + metadataFiles, + pinnedTimestampsState.v2(), + metadataFilePinnedTimestampMap, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen + ); + + // 3. For new pinned timestamp, read matching file to get min and max generations + try { + readAndCacheGenerationForPinnedTimestamp(implicitLockedFiles); + } catch (IOException e) { + throw new RuntimeException(e); + } + + // 4. we need to keep files since last successful run of scheduler + long lastSuccessfulFetchOfPinnedTimestamps = pinnedTimestampsState.v1(); + long maximumAllowedTimestamp = lastSuccessfulFetchOfPinnedTimestamps - RemoteStoreSettings + .getPinnedTimestampsLookbackInterval() + .getMillis(); + List metadataFilesToBeDeleted = RemoteStoreUtils.filterOutMetadataFilesBasedOnAge( + metadataFiles, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + maximumAllowedTimestamp + ); + + // 5. Filter out metadata files matching pinned timestamps + metadataFilesToBeDeleted.removeAll(implicitLockedFiles); + + if (metadataFilesToBeDeleted.isEmpty()) { + logger.debug("No metadata files to delete"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + if (metadataFiles.size() == metadataFilesToBeDeleted.size() && metadataFiles.equals(metadataFilesToBeDeleted)) { + metadataFilesToBeDeleted = metadataFiles.subList(1, metadataFiles.size()); + } + + // 6. From the remaining files, read oldest and latest + String oldestMetadataFileToBeDeleted = metadataFilesToBeDeleted.get(metadataFilesToBeDeleted.size() - 1); + String latestMetadataFileToBeDeleted = metadataFilesToBeDeleted.get(0); + + // 7. Delete generations + long minGenerationToBeDeleted; + long maxGenerationToBeDeleted; + if (metadataFileGenerationMap.containsKey(latestMetadataFileToBeDeleted) == false) { + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(latestMetadataFileToBeDeleted); + maxGenerationToBeDeleted = metadata.getMinTranslogGeneration() - 1; + } else { + maxGenerationToBeDeleted = metadataFileGenerationMap.get(latestMetadataFileToBeDeleted).v1() - 1; + } + long minGenerationToKeep = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); + maxGenerationToBeDeleted = Math.min(maxGenerationToBeDeleted, minGenerationToKeep); + + if (metadataFileGenerationMap.containsKey(oldestMetadataFileToBeDeleted) == false) { + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(oldestMetadataFileToBeDeleted); + minGenerationToBeDeleted = metadata.getMinTranslogGeneration() - 1; + } else { + minGenerationToBeDeleted = metadataFileGenerationMap.get(oldestMetadataFileToBeDeleted).v1() - 1; + } + + TreeSet> pinnedGenerations = getOrderedPinnedMetadataGenerations(); + for (long generation = maxGenerationToBeDeleted; generation >= minGenerationToBeDeleted; generation--) { + // 8. Check if the generation is not referred by metadata file matching pinned timestamps + if (isGenerationPinned(generation, pinnedGenerations) == false) { + generationsToDelete.add(generation); + } + } + if (generationsToDelete.isEmpty() == false) { + // 9. Delete stale generations + deleteRemoteGenerations(generationsToDelete); + // 10. Delete stale metadata files + translogTransferManager.deleteMetadataFilesAsync( + metadataFilesToBeDeleted, + remoteGenerationDeletionPermits::release + ); + // 11. Delete primary terms + deleteStaleRemotePrimaryTerms(metadataFiles); + } else { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } + } catch (Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } + } + + @Override + public void onFailure(Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + logger.error("Exception while listing translog metadata files", e); } - generationsToDelete.add(generation); + }; + translogTransferManager.listTranslogMetadataFilesAsync(listMetadataFilesListener); + } + + private boolean isGenerationPinned(long generation, TreeSet> pinnedGenerations) { + Tuple ceilingGenerationRange = pinnedGenerations.ceiling(new Tuple<>(generation, generation)); + if (ceilingGenerationRange != null && generation >= ceilingGenerationRange.v1() && generation <= ceilingGenerationRange.v2()) { + return true; } - if (generationsToDelete.isEmpty() == false) { - deleteRemoteGeneration(generationsToDelete); - translogTransferManager.deleteStaleTranslogMetadataFilesAsync(remoteGenerationDeletionPermits::release); - deleteStaleRemotePrimaryTerms(); - } else { - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + Tuple floorGenerationRange = pinnedGenerations.floor(new Tuple<>(generation, generation)); + if (floorGenerationRange != null && generation >= floorGenerationRange.v1() && generation <= floorGenerationRange.v2()) { + return true; + } + return false; + } + + private TreeSet> getOrderedPinnedMetadataGenerations() { + TreeSet> pinnedGenerations = new TreeSet<>((o1, o2) -> { + if (Objects.equals(o1.v1(), o2.v1()) == false) { + return o1.v1().compareTo(o2.v1()); + } else { + return o1.v2().compareTo(o2.v2()); + } + }); + pinnedGenerations.addAll(metadataFileGenerationMap.values()); + return pinnedGenerations; + } + + private void readAndCacheGenerationForPinnedTimestamp(Set implicitLockedFiles) throws IOException { + Set nonCachedMetadataFiles = implicitLockedFiles.stream() + .filter(f -> metadataFileGenerationMap.containsKey(f) == false) + .collect(Collectors.toSet()); + metadataFileGenerationMap.keySet().retainAll(implicitLockedFiles); + for (String metadataFile : nonCachedMetadataFiles) { + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(metadataFile); + metadataFileGenerationMap.put(metadataFile, new Tuple<>(metadata.getMinTranslogGeneration(), metadata.getGeneration())); } } @@ -573,7 +807,7 @@ public void trimUnreferencedReaders() throws IOException { * Deletes remote translog and metadata files asynchronously corresponding to the generations. * @param generations generations to be deleted. */ - private void deleteRemoteGeneration(Set generations) { + private void deleteRemoteGenerations(Set generations) { translogTransferManager.deleteGenerationAsync( primaryTermSupplier.getAsLong(), generations, @@ -587,17 +821,20 @@ private void deleteRemoteGeneration(Set generations) { *
* This will also delete all stale translog metadata files from remote except the latest basis the metadata file comparator. */ - private void deleteStaleRemotePrimaryTerms() { + private void deleteStaleRemotePrimaryTerms(List metadataFiles) { // The deletion of older translog files in remote store is on best-effort basis, there is a possibility that there // are older files that are no longer needed and should be cleaned up. In here, we delete all files that are part // of older primary term. if (olderPrimaryCleaned.trySet(Boolean.TRUE)) { - if (readers.isEmpty()) { - logger.trace("Translog reader list is empty, returning from deleteStaleRemotePrimaryTerms"); + if (metadataFiles.isEmpty()) { + logger.trace("No metadata is uploaded yet, returning from deleteStaleRemotePrimaryTerms"); return; } + Optional minPrimaryTerm = metadataFiles.stream() + .map(file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[1])) + .min(Long::compareTo); // First we delete all stale primary terms folders from remote store - long minimumReferencedPrimaryTerm = readers.stream().map(BaseTranslogReader::getPrimaryTerm).min(Long::compare).get(); + long minimumReferencedPrimaryTerm = minPrimaryTerm.get() - 1; translogTransferManager.deletePrimaryTermsAsync(minimumReferencedPrimaryTerm); } } diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index 87e0c21b8203c..4b4ceb7444471 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -317,6 +317,10 @@ TranslogReader openReader(Path path, Checkpoint checkpoint) throws IOException { */ public static long parseIdFromFileName(Path translogFile) { final String fileName = translogFile.getFileName().toString(); + return parseIdFromFileName(fileName); + } + + public static long parseIdFromFileName(String fileName) { final Matcher matcher = PARSE_STRICT_ID_PATTERN.matcher(fileName); if (matcher.matches()) { try { diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 1cc39cdf442e2..4e6a49fa5a4d6 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -45,10 +45,12 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.stream.Collectors; import static org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import static org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; +import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; /** * The class responsible for orchestrating the transfer of a {@link TransferSnapshot} via a {@link TransferService} @@ -337,35 +339,53 @@ private void deleteFileIfExists(Path filePath) throws IOException { } } + public TranslogTransferMetadata readMetadata(long timestamp) throws IOException { + if (timestamp < 0) { + return readMetadata(); + } + return readMetadata((blobMetadataList) -> { + List metadataFiles = blobMetadataList.stream().map(BlobMetadata::name).collect(Collectors.toList()); + Set metadataFilesMatchingTimestamp = RemoteStoreUtils.getPinnedTimestampLockedFiles( + metadataFiles, + Set.of(timestamp), + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen + ); + if (metadataFilesMatchingTimestamp.isEmpty()) { + return null; + } + assert metadataFilesMatchingTimestamp.size() == 1 : "There should be only 1 metadata file matching given timestamp"; + return metadataFilesMatchingTimestamp.stream().findFirst().get(); + }, Integer.MAX_VALUE); + } + public TranslogTransferMetadata readMetadata() throws IOException { + return readMetadata((blobMetadataList) -> { + RemoteStoreUtils.verifyNoMultipleWriters( + blobMetadataList.stream().map(BlobMetadata::name).collect(Collectors.toList()), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen + ); + return blobMetadataList.get(0).name(); + }, METADATA_FILES_TO_FETCH); + } + + private TranslogTransferMetadata readMetadata(Function, String> getMetadataFileToRead, int numberOfFilesToFetch) + throws IOException { SetOnce metadataSetOnce = new SetOnce<>(); SetOnce exceptionSetOnce = new SetOnce<>(); final CountDownLatch latch = new CountDownLatch(1); LatchedActionListener> latchedActionListener = new LatchedActionListener<>( ActionListener.wrap(blobMetadataList -> { if (blobMetadataList.isEmpty()) return; - RemoteStoreUtils.verifyNoMultipleWriters( - blobMetadataList.stream().map(BlobMetadata::name).collect(Collectors.toList()), - TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen - ); - String filename = blobMetadataList.get(0).name(); - boolean downloadStatus = false; - long downloadStartTime = System.nanoTime(), bytesToRead = 0; - try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, filename)) { - // Capture number of bytes for stats before reading - bytesToRead = inputStream.available(); - IndexInput indexInput = new ByteArrayIndexInput("metadata file", inputStream.readAllBytes()); - metadataSetOnce.set(metadataStreamWrapper.readStream(indexInput)); - downloadStatus = true; + String filename = getMetadataFileToRead.apply(blobMetadataList); + if (filename == null) { + return; + } + try { + metadataSetOnce.set(readMetadata(filename)); } catch (IOException e) { logger.error(() -> new ParameterizedMessage("Exception while reading metadata file: {}", filename), e); exceptionSetOnce.set(e); - } finally { - remoteTranslogTransferTracker.addDownloadTimeInMillis((System.nanoTime() - downloadStartTime) / 1_000_000L); - logger.debug("translogMetadataDownloadStatus={}", downloadStatus); - if (downloadStatus) { - remoteTranslogTransferTracker.addDownloadBytesSucceeded(bytesToRead); - } } }, e -> { if (e instanceof RuntimeException) { @@ -381,12 +401,14 @@ public TranslogTransferMetadata readMetadata() throws IOException { transferService.listAllInSortedOrder( remoteMetadataTransferPath, TranslogTransferMetadata.METADATA_PREFIX, - METADATA_FILES_TO_FETCH, + numberOfFilesToFetch, latchedActionListener ); - latch.await(); + if (latch.await(remoteStoreSettings.getClusterRemoteTranslogTransferTimeout().millis(), TimeUnit.MILLISECONDS) == false) { + throw new RuntimeException("Timed out reading metadata file"); + } } catch (InterruptedException e) { - throw new IOException("Exception while reading/downloading metadafile", e); + throw new IOException("Exception while reading/downloading metadata file", e); } if (exceptionSetOnce.get() != null) { @@ -396,6 +418,26 @@ public TranslogTransferMetadata readMetadata() throws IOException { return metadataSetOnce.get(); } + public TranslogTransferMetadata readMetadata(String metadataFilename) throws IOException { + boolean downloadStatus = false; + TranslogTransferMetadata translogTransferMetadata = null; + long downloadStartTime = System.nanoTime(), bytesToRead = 0; + try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, metadataFilename)) { + // Capture number of bytes for stats before reading + bytesToRead = inputStream.available(); + IndexInput indexInput = new ByteArrayIndexInput("metadata file", inputStream.readAllBytes()); + translogTransferMetadata = metadataStreamWrapper.readStream(indexInput); + downloadStatus = true; + } finally { + remoteTranslogTransferTracker.addDownloadTimeInMillis((System.nanoTime() - downloadStartTime) / 1_000_000L); + logger.debug("translogMetadataDownloadStatus={}", downloadStatus); + if (downloadStatus) { + remoteTranslogTransferTracker.addDownloadBytesSucceeded(bytesToRead); + } + } + return translogTransferMetadata; + } + private TransferFileSnapshot prepareMetadata(TransferSnapshot transferSnapshot) throws IOException { Map generationPrimaryTermMap = transferSnapshot.getTranslogFileSnapshots().stream().map(s -> { assert s instanceof TranslogFileSnapshot; @@ -549,6 +591,16 @@ public void onFailure(Exception e) { }); } + public void listTranslogMetadataFilesAsync(ActionListener> listener) { + transferService.listAllInSortedOrderAsync( + ThreadPool.Names.REMOTE_PURGE, + remoteMetadataTransferPath, + TranslogTransferMetadata.METADATA_PREFIX, + Integer.MAX_VALUE, + listener + ); + } + public void deleteStaleTranslogMetadataFilesAsync(Runnable onCompletion) { try { transferService.listAllInSortedOrderAsync( @@ -635,7 +687,7 @@ public void onFailure(Exception e) { * @param files list of metadata files to be deleted. * @param onCompletion runnable to run on completion of deletion regardless of success/failure. */ - private void deleteMetadataFilesAsync(List files, Runnable onCompletion) { + public void deleteMetadataFilesAsync(List files, Runnable onCompletion) { try { transferService.deleteBlobsAsync(ThreadPool.Names.REMOTE_PURGE, remoteMetadataTransferPath, files, new ActionListener<>() { @Override diff --git a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java index 55280ca5c96d6..00d9ffdac6086 100644 --- a/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java +++ b/server/src/main/java/org/opensearch/indices/RemoteStoreSettings.java @@ -292,6 +292,11 @@ public static TimeValue getPinnedTimestampsLookbackInterval() { return pinnedTimestampsLookbackInterval; } + // Visible for testing + public static void setPinnedTimestampsLookbackInterval(TimeValue pinnedTimestampsLookbackInterval) { + RemoteStoreSettings.pinnedTimestampsLookbackInterval = pinnedTimestampsLookbackInterval; + } + public static boolean isPinnedTimestampsEnabled() { return isPinnedTimestampsEnabled; } diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java index 6bf35cc1eac9b..306de11757b0c 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -15,11 +15,15 @@ import org.apache.lucene.store.DataOutput; import org.apache.lucene.tests.mockfile.FilterFileChannel; import org.apache.lucene.tests.util.LuceneTestCase; +import org.mockito.Mockito; import org.opensearch.OpenSearchException; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.fs.FsBlobContainer; @@ -29,9 +33,11 @@ import org.opensearch.common.lease.Releasables; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; @@ -40,6 +46,7 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.env.TestEnvironment; +import org.opensearch.gateway.remote.model.RemoteStorePinnedTimestampsBlobStore; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.MissingHistoryOperationsException; import org.opensearch.index.remote.RemoteTranslogTransferTracker; @@ -51,8 +58,13 @@ import org.opensearch.index.translog.transfer.TranslogTransferMetadata; import org.opensearch.index.translog.transfer.TranslogUploadFailedException; import org.opensearch.indices.DefaultRemoteStoreSettings; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.blobstore.BlobStoreTestUtil; import org.opensearch.repositories.fs.FsRepository; @@ -95,9 +107,11 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.LongConsumer; +import java.util.function.Supplier; import java.util.zip.CRC32; import java.util.zip.CheckedInputStream; +import static org.mockito.ArgumentMatchers.eq; import static org.opensearch.common.util.BigArrays.NON_RECYCLING_INSTANCE; import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_KEEP_EXTRA_GEN_SETTING; import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; @@ -153,6 +167,50 @@ public void setUp() throws Exception { // if a previous test failed we clean up things here translogDir = createTempDir(); translog = create(translogDir); + + Supplier repositoriesServiceSupplier = mock(Supplier.class); + Settings settings = Settings.builder() + .put(Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "remote-repo") + .build(); + RepositoriesService repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + BlobStoreRepository blobStoreRepository = mock(BlobStoreRepository.class); + when(repositoriesService.repository("remote-repo")).thenReturn(blobStoreRepository); + + ThreadPool mockThreadPool = mock(ThreadPool.class); + when(mockThreadPool.schedule(any(), any(), any())).then(invocationOnMock -> { + Runnable runnable = invocationOnMock.getArgument(0); + runnable.run(); + return null; + }).then(subsequentInvocationsOnMock -> null); + + ClusterService clusterService = mock(ClusterService.class); + ClusterState clusterState = mock(ClusterState.class); + when(clusterService.state()).thenReturn(clusterState); + Metadata metadata = mock(Metadata.class); + when(clusterState.metadata()).thenReturn(metadata); + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService = new RemoteStorePinnedTimestampService( + repositoriesServiceSupplier, + settings, + mockThreadPool, + clusterService + ); + RemoteStorePinnedTimestampService remoteStorePinnedTimestampServiceSpy = Mockito.spy(remoteStorePinnedTimestampService); + + RemoteStorePinnedTimestampsBlobStore remoteStorePinnedTimestampsBlobStore = mock(RemoteStorePinnedTimestampsBlobStore.class); + BlobStoreTransferService blobStoreTransferService = mock(BlobStoreTransferService.class); + when(remoteStorePinnedTimestampServiceSpy.pinnedTimestampsBlobStore()).thenReturn(remoteStorePinnedTimestampsBlobStore); + when(remoteStorePinnedTimestampServiceSpy.blobStoreTransferService()).thenReturn(blobStoreTransferService); + + doAnswer(invocationOnMock -> { + ActionListener> actionListener = invocationOnMock.getArgument(3); + actionListener.onResponse(new ArrayList<>()); + return null; + }).when(blobStoreTransferService).listAllInSortedOrder(any(), any(), eq(1), any()); + + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + + remoteStorePinnedTimestampServiceSpy.start(); } @Override @@ -873,7 +931,7 @@ public void testDrainSync() throws Exception { assertBusy(() -> assertEquals(0, latch.getCount())); assertEquals(0, translog.availablePermits()); slowDown.setSleepSeconds(0); - assertEquals(6, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); assertEquals(2, translog.readers.size()); Set mdFiles = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); @@ -882,7 +940,7 @@ public void testDrainSync() throws Exception { translog.trimUnreferencedReaders(); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); - assertEquals(6, translog.allUploaded().size()); + assertEquals(2, translog.allUploaded().size()); assertEquals(mdFiles, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR))); // Case 4 - After drainSync, if an upload is an attempted, we do not upload to remote store. @@ -892,21 +950,21 @@ public void testDrainSync() throws Exception { new Translog.Index(String.valueOf(2), 2, primaryTerm.get(), new byte[] { 1 }) ); assertEquals(1, translog.readers.size()); - assertEquals(6, translog.allUploaded().size()); + assertEquals(2, translog.allUploaded().size()); assertEquals(mdFiles, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR))); // Refill the permits back Releasables.close(releasable); addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(3), 3, primaryTerm.get(), new byte[] { 1 })); assertEquals(2, translog.readers.size()); - assertEquals(8, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); assertEquals(3, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); translog.setMinSeqNoToKeep(3); translog.trimUnreferencedReaders(); assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); - assertBusy(() -> assertEquals(4, translog.allUploaded().size())); + assertBusy(() -> assertEquals(2, translog.allUploaded().size())); assertBusy(() -> assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size())); } @@ -1688,13 +1746,13 @@ public void testDownloadWithRetries() throws IOException { TranslogTransferManager mockTransfer = mock(TranslogTransferManager.class); RemoteTranslogTransferTracker remoteTranslogTransferTracker = mock(RemoteTranslogTransferTracker.class); - when(mockTransfer.readMetadata()).thenReturn(translogTransferMetadata); + when(mockTransfer.readMetadata(-1)).thenReturn(translogTransferMetadata); when(mockTransfer.getRemoteTranslogTransferTracker()).thenReturn(remoteTranslogTransferTracker); // Always File not found when(mockTransfer.downloadTranslog(any(), any(), any())).thenThrow(new NoSuchFileException("File not found")); TranslogTransferManager finalMockTransfer = mockTransfer; - assertThrows(NoSuchFileException.class, () -> RemoteFsTranslog.download(finalMockTransfer, location, logger, false)); + assertThrows(NoSuchFileException.class, () -> RemoteFsTranslog.download(finalMockTransfer, location, logger, false, -1)); // File not found in first attempt . File found in second attempt. mockTransfer = mock(TranslogTransferManager.class); @@ -1715,7 +1773,7 @@ public void testDownloadWithRetries() throws IOException { }).when(mockTransfer).downloadTranslog(any(), any(), any()); // no exception thrown - RemoteFsTranslog.download(mockTransfer, location, logger, false); + RemoteFsTranslog.download(mockTransfer, location, logger, false, -1); } // No translog data in local as well as remote, we skip creating empty translog @@ -1728,7 +1786,7 @@ public void testDownloadWithNoTranslogInLocalAndRemote() throws IOException { when(mockTransfer.getRemoteTranslogTransferTracker()).thenReturn(remoteTranslogTransferTracker); Path[] filesBeforeDownload = FileSystemUtils.files(location); - RemoteFsTranslog.download(mockTransfer, location, logger, false); + RemoteFsTranslog.download(mockTransfer, location, logger, false, -1); assertEquals(filesBeforeDownload, FileSystemUtils.files(location)); } @@ -1748,7 +1806,7 @@ public void testDownloadWithTranslogOnlyInLocal() throws IOException { Checkpoint existingCheckpoint = Translog.readCheckpoint(location); TranslogTransferManager finalMockTransfer = mockTransfer; - RemoteFsTranslog.download(finalMockTransfer, location, logger, false); + RemoteFsTranslog.download(finalMockTransfer, location, logger, false, -1); Path[] filesPostDownload = FileSystemUtils.files(location); assertEquals(2, filesPostDownload.length); @@ -1784,11 +1842,11 @@ public void testDownloadWithEmptyTranslogOnlyInLocal() throws IOException { TranslogTransferManager finalMockTransfer = mockTransfer; // download first time will ensure creating empty translog - RemoteFsTranslog.download(finalMockTransfer, location, logger, false); + RemoteFsTranslog.download(finalMockTransfer, location, logger, false, -1); Path[] filesPostFirstDownload = FileSystemUtils.files(location); // download on empty translog should be a no-op - RemoteFsTranslog.download(finalMockTransfer, location, logger, false); + RemoteFsTranslog.download(finalMockTransfer, location, logger, false, -1); Path[] filesPostSecondDownload = FileSystemUtils.files(location); assertArrayEquals(filesPostFirstDownload, filesPostSecondDownload); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index c6f9838ad2d52..eb1a5e8f68397 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -23,6 +23,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; @@ -51,7 +52,10 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import org.mockito.Mockito; @@ -753,4 +757,121 @@ private void assertTlogCkpDownloadStatsWithMetadata() { // Expect delay for both tlog and ckp file assertTrue(remoteTranslogTransferTracker.getTotalDownloadTimeInMillis() >= delayForBlobDownload); } + + public void testlistTranslogMetadataFilesAsync() throws Exception { + String tm1 = new TranslogTransferMetadata(1, 1, 1, 2).getFileName(); + String tm2 = new TranslogTransferMetadata(1, 2, 1, 2).getFileName(); + String tm3 = new TranslogTransferMetadata(2, 3, 1, 2).getFileName(); + doAnswer(invocation -> { + ActionListener> actionListener = invocation.getArgument(4); + List bmList = new LinkedList<>(); + bmList.add(new PlainBlobMetadata(tm1, 1)); + bmList.add(new PlainBlobMetadata(tm2, 1)); + bmList.add(new PlainBlobMetadata(tm3, 1)); + actionListener.onResponse(bmList); + return null; + }).when(transferService) + .listAllInSortedOrderAsync( + eq(ThreadPool.Names.REMOTE_PURGE), + any(BlobPath.class), + eq(TranslogTransferMetadata.METADATA_PREFIX), + anyInt(), + any(ActionListener.class) + ); + AtomicBoolean fetchCompleted = new AtomicBoolean(false); + translogTransferManager.listTranslogMetadataFilesAsync(new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + assertEquals(3, blobMetadata.size()); + assertEquals(blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()), List.of(tm1, tm2, tm3)); + fetchCompleted.set(true); + } + + @Override + public void onFailure(Exception e) { + fetchCompleted.set(true); + throw new RuntimeException(e); + } + }); + assertBusy(() -> assertTrue(fetchCompleted.get())); + } + + public void testReadMetadataForGivenTimestampNoFile() throws IOException { + doAnswer(invocation -> { + LatchedActionListener> latchedActionListener = invocation.getArgument(3); + List bmList = new LinkedList<>(); + latchedActionListener.onResponse(bmList); + return null; + }).when(transferService) + .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); + + assertNull(translogTransferManager.readMetadata(1234L)); + assertNoDownloadStats(false); + } + + public void testReadMetadataForGivenTimestampNoMatchingFile() throws IOException { + doAnswer(invocation -> { + LatchedActionListener> latchedActionListener = invocation.getArgument(3); + String timestamp1 = RemoteStoreUtils.invertLong(2345L); + BlobMetadata bm1 = new PlainBlobMetadata("metadata__1__12__" + timestamp1 + "__node1__1", 1); + String timestamp2 = RemoteStoreUtils.invertLong(3456L); + BlobMetadata bm2 = new PlainBlobMetadata("metadata__1__12__" + timestamp2 + "__node1__1", 1); + List bmList = List.of(bm1, bm2); + latchedActionListener.onResponse(bmList); + return null; + }).when(transferService) + .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); + + assertNull(translogTransferManager.readMetadata(1234L)); + assertNoDownloadStats(false); + } + + public void testReadMetadataForGivenTimestampFile() throws IOException { + AtomicReference mdFilename1 = new AtomicReference<>(); + String timestamp1 = RemoteStoreUtils.invertLong(2345L); + mdFilename1.set("metadata__1__12__" + timestamp1 + "__node1__1"); + doAnswer(invocation -> { + LatchedActionListener> latchedActionListener = invocation.getArgument(3); + BlobMetadata bm1 = new PlainBlobMetadata(mdFilename1.get(), 1); + String timestamp2 = RemoteStoreUtils.invertLong(3456L); + BlobMetadata bm2 = new PlainBlobMetadata("metadata__1__12__" + timestamp2 + "__node1__1", 1); + List bmList = List.of(bm1, bm2); + latchedActionListener.onResponse(bmList); + return null; + }).when(transferService) + .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); + + TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); + long delayForMdDownload = 1; + when(transferService.downloadBlob(any(BlobPath.class), eq(mdFilename1.get()))).thenAnswer(invocation -> { + Thread.sleep(delayForMdDownload); + return new ByteArrayInputStream(translogTransferManager.getMetadataBytes(metadata)); + }); + + assertEquals(metadata, translogTransferManager.readMetadata(3000L)); + + assertEquals(translogTransferManager.getMetadataBytes(metadata).length, remoteTranslogTransferTracker.getDownloadBytesSucceeded()); + assertTrue(remoteTranslogTransferTracker.getTotalDownloadTimeInMillis() >= delayForMdDownload); + } + + public void testReadMetadataForGivenTimestampException() throws IOException { + AtomicReference mdFilename1 = new AtomicReference<>(); + String timestamp1 = RemoteStoreUtils.invertLong(2345L); + mdFilename1.set("metadata__1__12__" + timestamp1 + "__node1__1"); + doAnswer(invocation -> { + LatchedActionListener> latchedActionListener = invocation.getArgument(3); + BlobMetadata bm1 = new PlainBlobMetadata(mdFilename1.get(), 1); + String timestamp2 = RemoteStoreUtils.invertLong(3456L); + BlobMetadata bm2 = new PlainBlobMetadata("metadata__1__12__" + timestamp2 + "__node1__1", 1); + List bmList = List.of(bm1, bm2); + latchedActionListener.onResponse(bmList); + return null; + }).when(transferService) + .listAllInSortedOrder(any(BlobPath.class), eq(TranslogTransferMetadata.METADATA_PREFIX), anyInt(), any(ActionListener.class)); + + when(transferService.downloadBlob(any(BlobPath.class), eq(mdFilename1.get()))).thenThrow(new IOException("Something went wrong")); + + assertThrows(IOException.class, () -> translogTransferManager.readMetadata(3000L)); + assertNoDownloadStats(true); + } }