diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java index 928c9e33e19cb..3038db4583cf4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteSplitIndexIT.java @@ -66,6 +66,7 @@ import org.opensearch.index.seqno.SeqNoStats; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.RemoteStoreSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase; @@ -109,13 +110,16 @@ public void cleanUp() throws Exception { assertAcked( client().admin().indices().prepareDelete("*").setIndicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN).get() ); - assertBusy(() -> { - try { - assertEquals(0, getFileCount(translogRepoPath)); - } catch (IOException e) { - fail(); - } - }, 30, TimeUnit.SECONDS); + // With pinned timestamp, we can have tlog files even after deletion. + if (RemoteStoreSettings.isPinnedTimestampsEnabled() == false) { + assertBusy(() -> { + try { + assertEquals(0, getFileCount(translogRepoPath)); + } catch (IOException e) { + fail(); + } + }, 30, TimeUnit.SECONDS); + } super.teardown(); } 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..f6afc4c508720 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,13 @@ import java.util.function.BooleanSupplier; import java.util.function.LongConsumer; import java.util.function.LongSupplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; 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 @@ -68,6 +81,8 @@ public class RemoteFsTranslog extends Translog { private final Logger logger; private final TranslogTransferManager translogTransferManager; + // This tracker keeps track of local tranlog files that are uploaded to remote store. + // Once tlog files are deleted from local, we remove them from tracker even if the files still exist in remote translog. private final FileTransferTracker fileTransferTracker; private final BooleanSupplier startedPrimarySupplier; private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; @@ -92,6 +107,9 @@ 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; + // For metadata files, with no min generation in the name, we cache generation data to avoid multiple reads. + private final Map> oldFormatMetadataFileGenerationMap; public RemoteFsTranslog( TranslogConfig config, @@ -105,6 +123,36 @@ public RemoteFsTranslog( BooleanSupplier startedPrimarySupplier, RemoteTranslogTransferTracker remoteTranslogTransferTracker, RemoteStoreSettings remoteStoreSettings + ) throws IOException { + this( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + threadPool, + startedPrimarySupplier, + remoteTranslogTransferTracker, + remoteStoreSettings, + 0 + ); + } + + 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 +160,8 @@ public RemoteFsTranslog( this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; fileTransferTracker = new FileTransferTracker(shardId, remoteTranslogTransferTracker); isTranslogMetadataEnabled = indexSettings().isTranslogMetadataEnabled(); + this.metadataFilePinnedTimestampMap = new HashMap<>(); + this.oldFormatMetadataFileGenerationMap = new HashMap<>(); this.translogTransferManager = buildTranslogTransferManager( blobStoreRepository, threadPool, @@ -123,7 +173,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 +224,32 @@ public static void download( Logger logger, boolean seedRemote, boolean isTranslogMetadataEnabled + ) throws IOException { + download( + repository, + shardId, + threadPool, + location, + pathStrategy, + remoteStoreSettings, + logger, + seedRemote, + isTranslogMetadataEnabled, + 0 + ); + } + + 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 +271,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 +290,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 +304,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); @@ -535,8 +617,31 @@ protected Releasable drainSync() { @Override public void trimUnreferencedReaders() throws IOException { - // clean up local translog files and updates readers - super.trimUnreferencedReaders(); + trimUnreferencedReaders(false, true); + } + + // Visible for testing + protected void trimUnreferencedReaders(boolean indexDeleted, boolean trimLocal) throws IOException { + if (trimLocal) { + // 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. @@ -544,6 +649,12 @@ public void trimUnreferencedReaders() throws IOException { return; } + // This is to fail fast and avoid listing md files un-necessarily. + if (indexDeleted == false && 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,34 +662,209 @@ 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()); + + try { + if (metadataFiles.size() <= 1) { + logger.debug("No stale translog metadata files found"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + // Check last fetch status of pinned timestamps. If stale, return. + if (indexDeleted == false && RemoteStoreUtils.isPinnedTimestampStateStale()) { + logger.warn("Skipping remote segment store garbage collection as last fetch of pinned timestamp is stale"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + List metadataFilesToBeDeleted = getMetadataFilesToBeDeleted(metadataFiles); + + // If index is not deleted, make sure to keep latest metadata file + if (indexDeleted == false) { + metadataFilesToBeDeleted.remove(metadataFiles.get(0)); + } + + if (metadataFilesToBeDeleted.isEmpty()) { + logger.debug("No metadata files to delete"); + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + return; + } + + logger.debug("metadataFilesToBeDeleted = {}", metadataFilesToBeDeleted); + // For all the files that we are keeping, fetch min and max generations + List metadataFilesNotToBeDeleted = new ArrayList<>(metadataFiles); + metadataFilesNotToBeDeleted.removeAll(metadataFilesToBeDeleted); + + logger.debug("metadataFilesNotToBeDeleted = {}", metadataFilesNotToBeDeleted); + Set generationsToBeDeleted = getGenerationsToBeDeleted( + metadataFilesNotToBeDeleted, + metadataFilesToBeDeleted, + indexDeleted + ); + + logger.debug("generationsToBeDeleted = {}", generationsToBeDeleted); + if (generationsToBeDeleted.isEmpty() == false) { + // Delete stale generations + translogTransferManager.deleteGenerationAsync( + primaryTermSupplier.getAsLong(), + generationsToBeDeleted, + remoteGenerationDeletionPermits::release + ); + + // Delete stale metadata files + translogTransferManager.deleteMetadataFilesAsync( + metadataFilesToBeDeleted, + remoteGenerationDeletionPermits::release + ); + + // Update cache to keep only those metadata files that are not getting deleted + oldFormatMetadataFileGenerationMap.keySet().retainAll(metadataFilesNotToBeDeleted); + + // Delete stale primary terms + deleteStaleRemotePrimaryTerms(metadataFiles); + } else { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } + } catch (Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + } } - generationsToDelete.add(generation); + + @Override + public void onFailure(Exception e) { + remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + logger.error("Exception while listing translog metadata files", e); + } + }; + translogTransferManager.listTranslogMetadataFilesAsync(listMetadataFilesListener); + } + + // Visible for testing + protected Set getGenerationsToBeDeleted( + List metadataFilesNotToBeDeleted, + List metadataFilesToBeDeleted, + boolean indexDeleted + ) throws IOException { + long maxGenerationToBeDeleted = Long.MAX_VALUE; + + if (indexDeleted == false) { + maxGenerationToBeDeleted = minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep(); } - if (generationsToDelete.isEmpty() == false) { - deleteRemoteGeneration(generationsToDelete); - translogTransferManager.deleteStaleTranslogMetadataFilesAsync(remoteGenerationDeletionPermits::release); - deleteStaleRemotePrimaryTerms(); - } else { - remoteGenerationDeletionPermits.release(REMOTE_DELETION_PERMITS); + + Set generationsFromMetadataFilesToBeDeleted = new HashSet<>(); + for (String mdFile : metadataFilesToBeDeleted) { + Tuple minMaxGen = getMinMaxTranslogGenerationFromMetadataFile(mdFile, translogTransferManager); + generationsFromMetadataFilesToBeDeleted.addAll( + LongStream.rangeClosed(minMaxGen.v1(), minMaxGen.v2()).boxed().collect(Collectors.toList()) + ); + } + + Map> metadataFileNotToBeDeletedGenerationMap = getGenerationForMetadataFiles(metadataFilesNotToBeDeleted); + TreeSet> pinnedGenerations = getOrderedPinnedMetadataGenerations(metadataFileNotToBeDeletedGenerationMap); + Set generationsToBeDeleted = new HashSet<>(); + for (long generation : generationsFromMetadataFilesToBeDeleted) { + // Check if the generation is not referred by metadata file matching pinned timestamps + if (generation <= maxGenerationToBeDeleted && isGenerationPinned(generation, pinnedGenerations) == false) { + generationsToBeDeleted.add(generation); + } } + return generationsToBeDeleted; } - /** - * Deletes remote translog and metadata files asynchronously corresponding to the generations. - * @param generations generations to be deleted. - */ - private void deleteRemoteGeneration(Set generations) { - translogTransferManager.deleteGenerationAsync( - primaryTermSupplier.getAsLong(), - generations, - remoteGenerationDeletionPermits::release + // Visible for testing + protected List getMetadataFilesToBeDeleted(List metadataFiles) { + Tuple> pinnedTimestampsState = RemoteStorePinnedTimestampService.getPinnedTimestamps(); + + // Keep files since last successful run of scheduler + List metadataFilesToBeDeleted = RemoteStoreUtils.filterOutMetadataFilesBasedOnAge( + metadataFiles, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + pinnedTimestampsState.v1() + ); + + logger.trace( + "metadataFiles.size = {}, metadataFilesToBeDeleted based on age based filtering = {}", + metadataFiles.size(), + metadataFilesToBeDeleted.size() ); + + // Get md files matching pinned timestamps + Set implicitLockedFiles = RemoteStoreUtils.getPinnedTimestampLockedFiles( + metadataFilesToBeDeleted, + pinnedTimestampsState.v2(), + metadataFilePinnedTimestampMap, + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen + ); + + // Filter out metadata files matching pinned timestamps + metadataFilesToBeDeleted.removeAll(implicitLockedFiles); + + logger.trace( + "implicitLockedFiles.size = {}, metadataFilesToBeDeleted based on pinned timestamp filtering = {}", + implicitLockedFiles.size(), + metadataFilesToBeDeleted.size() + ); + + return metadataFilesToBeDeleted; + } + + // Visible for testing + protected 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; + } + Tuple floorGenerationRange = pinnedGenerations.floor(new Tuple<>(generation, generation)); + if (floorGenerationRange != null && generation >= floorGenerationRange.v1() && generation <= floorGenerationRange.v2()) { + return true; + } + return false; + } + + private TreeSet> getOrderedPinnedMetadataGenerations(Map> metadataFileGenerationMap) { + 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; + } + + // Visible for testing + protected Map> getGenerationForMetadataFiles(List metadataFiles) throws IOException { + Map> metadataFileGenerationMap = new HashMap<>(); + for (String metadataFile : metadataFiles) { + metadataFileGenerationMap.put(metadataFile, getMinMaxTranslogGenerationFromMetadataFile(metadataFile, translogTransferManager)); + } + return metadataFileGenerationMap; + } + + // Visible for testing + protected Tuple getMinMaxTranslogGenerationFromMetadataFile( + String metadataFile, + TranslogTransferManager translogTransferManager + ) throws IOException { + Tuple minMaxGenerationFromFileName = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(metadataFile); + if (minMaxGenerationFromFileName != null) { + return minMaxGenerationFromFileName; + } else { + if (oldFormatMetadataFileGenerationMap.containsKey(metadataFile)) { + return oldFormatMetadataFileGenerationMap.get(metadataFile); + } else { + TranslogTransferMetadata metadata = translogTransferManager.readMetadata(metadataFile); + Tuple minMaxGenTuple = new Tuple<>(metadata.getMinTranslogGeneration(), metadata.getGeneration()); + oldFormatMetadataFileGenerationMap.put(metadataFile, minMaxGenTuple); + return minMaxGenTuple; + } + } } /** @@ -587,17 +873,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); } } @@ -633,7 +922,15 @@ public static void cleanup( protected void onDelete() { ClusterService.assertClusterOrClusterManagerStateThread(); // clean up all remote translog files - translogTransferManager.delete(); + if (RemoteStoreSettings.isPinnedTimestampsEnabled()) { + try { + trimUnreferencedReaders(true, false); + } catch (IOException e) { + logger.error("Exception while deleting translog files from remote store", e); + } + } else { + translogTransferManager.delete(); + } } // Visible for testing 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..527fb0123d800 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,54 @@ private void deleteFileIfExists(Path filePath) throws IOException { } } + public TranslogTransferMetadata readMetadata(long pinnedTimestamp) throws IOException { + if (pinnedTimestamp <= 0) { + return readMetadata(); + } + return readMetadata((blobMetadataList) -> { + List metadataFiles = blobMetadataList.stream().map(BlobMetadata::name).collect(Collectors.toList()); + Set metadataFilesMatchingTimestamp = RemoteStoreUtils.getPinnedTimestampLockedFiles( + metadataFiles, + Set.of(pinnedTimestamp), + file -> RemoteStoreUtils.invertLong(file.split(METADATA_SEPARATOR)[3]), + TranslogTransferMetadata::getNodeIdByPrimaryTermAndGen, + true + ); + 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 +402,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 +419,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 +592,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 +688,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/index/translog/transfer/TranslogTransferMetadata.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java index acd7574e648a3..745fa9a8a219a 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java @@ -8,6 +8,10 @@ package org.opensearch.index.translog.transfer; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.Version; import org.opensearch.common.SetOnce; import org.opensearch.common.collect.Tuple; import org.opensearch.index.remote.RemoteStoreUtils; @@ -25,6 +29,8 @@ */ public class TranslogTransferMetadata { + public static final Logger logger = LogManager.getLogger(TranslogTransferMetadata.class); + private final long primaryTerm; private final long generation; @@ -128,6 +134,24 @@ public static Tuple getNodeIdByPrimaryTermAndGen(String filename return new Tuple<>(primaryTermAndGen, nodeId); } + public static Tuple getMinMaxTranslogGenerationFromFilename(String filename) { + String[] tokens = filename.split(METADATA_SEPARATOR); + if (tokens.length < 7) { + // For versions < 2.17, we don't have min translog generation. + return null; + } + assert Version.CURRENT.onOrAfter(Version.V_2_17_0); + try { + // instead of direct index, we go backwards to avoid running into same separator in nodeId + String minGeneration = tokens[tokens.length - 2]; + String maxGeneration = tokens[2]; + return new Tuple<>(RemoteStoreUtils.invertLong(minGeneration), RemoteStoreUtils.invertLong(maxGeneration)); + } catch (NumberFormatException e) { + logger.error(() -> new ParameterizedMessage("Exception while getting min and max translog generation from: {}", filename), e); + return null; + } + } + @Override public int hashCode() { return Objects.hash(primaryTerm, generation); 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..163340e8ec7d5 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -121,12 +121,12 @@ public class RemoteFsTranslogTests extends OpenSearchTestCase { private AtomicLong globalCheckpoint; protected Path translogDir; // A default primary term is used by translog instances created in this test. - private final AtomicLong primaryTerm = new AtomicLong(); + protected final AtomicLong primaryTerm = new AtomicLong(); private final AtomicBoolean primaryMode = new AtomicBoolean(true); private final AtomicReference persistedSeqNoConsumer = new AtomicReference<>(); - private ThreadPool threadPool; - private final static String METADATA_DIR = "metadata"; - private final static String DATA_DIR = "data"; + protected ThreadPool threadPool; + protected final static String METADATA_DIR = "metadata"; + protected final static String DATA_DIR = "data"; AtomicInteger writeCalls = new AtomicInteger(); BlobStoreRepository repository; @@ -277,7 +277,7 @@ private Translog.Location addToTranslogAndList(Translog translog, List list, Translog.Operation op) + protected Translog.Location addToTranslogAndListAndUpload(Translog translog, List list, Translog.Operation op) throws IOException { Translog.Location loc = translog.add(op); translog.ensureSynced(loc); @@ -390,7 +390,7 @@ public void testSimpleOperations() throws IOException { } - private TranslogConfig getConfig(int gensToKeep) { + protected TranslogConfig getConfig(int gensToKeep) { Path tempDir = createTempDir(); final TranslogConfig temp = getTranslogConfig(tempDir, gensToKeep); final TranslogConfig config = new TranslogConfig( @@ -405,7 +405,7 @@ private TranslogConfig getConfig(int gensToKeep) { return config; } - private ChannelFactory getChannelFactory() { + protected ChannelFactory getChannelFactory() { writeCalls = new AtomicInteger(); final ChannelFactory channelFactory = (file, openOption) -> { FileChannel delegate = FileChannel.open(file, openOption); @@ -493,8 +493,8 @@ ChannelFactory getChannelFactory() { 6, blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() ); - } + } public void testReadLocation() throws IOException { @@ -671,13 +671,13 @@ public void testSimpleOperationsUpload() throws Exception { assertThat(snapshot.totalOperations(), equalTo(ops.size())); } - assertEquals(4, translog.allUploaded().size()); + assertEquals(2, translog.allUploaded().size()); addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); - assertEquals(6, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); translog.rollGeneration(); - assertEquals(6, translog.allUploaded().size()); + assertEquals(4, translog.allUploaded().size()); Set mdFiles = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); assertEquals(2, mdFiles.size()); @@ -736,7 +736,7 @@ public void testSimpleOperationsUpload() throws Exception { assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); assertEquals(1, translog.readers.size()); assertBusy(() -> { - assertEquals(4, translog.allUploaded().size()); + assertEquals(2, translog.allUploaded().size()); assertEquals( 4, blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() @@ -755,13 +755,12 @@ public void testSimpleOperationsUpload() throws Exception { assertEquals(1, translog.readers.size()); assertEquals(1, translog.stats().estimatedNumberOfOperations()); assertBusy(() -> { - assertEquals(4, translog.allUploaded().size()); + assertEquals(2, translog.allUploaded().size()); assertEquals( 4, blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() ); }); - } public void testMetadataFileDeletion() throws Exception { @@ -775,7 +774,7 @@ public void testMetadataFileDeletion() throws Exception { 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())); int moreDocs = randomIntBetween(3, 10); logger.info("numDocs={} moreDocs={}", numDocs, moreDocs); @@ -873,7 +872,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 +881,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,25 +891,25 @@ 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())); } - private BlobPath getTranslogDirectory() { + protected BlobPath getTranslogDirectory() { return repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG.getName()); } @@ -1688,13 +1687,13 @@ public void testDownloadWithRetries() throws IOException { TranslogTransferManager mockTransfer = mock(TranslogTransferManager.class); RemoteTranslogTransferTracker remoteTranslogTransferTracker = mock(RemoteTranslogTransferTracker.class); - when(mockTransfer.readMetadata()).thenReturn(translogTransferMetadata); + when(mockTransfer.readMetadata(0)).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, 0)); // File not found in first attempt . File found in second attempt. mockTransfer = mock(TranslogTransferManager.class); @@ -1715,7 +1714,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, 0); } // No translog data in local as well as remote, we skip creating empty translog @@ -1728,7 +1727,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, 0); assertEquals(filesBeforeDownload, FileSystemUtils.files(location)); } @@ -1748,7 +1747,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, 0); Path[] filesPostDownload = FileSystemUtils.files(location); assertEquals(2, filesPostDownload.length); @@ -1784,11 +1783,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, 0); 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, 0); Path[] filesPostSecondDownload = FileSystemUtils.files(location); assertArrayEquals(filesPostFirstDownload, filesPostSecondDownload); diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java new file mode 100644 index 0000000000000..e35ab6bcd14c3 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogWithPinnedTimestampTests.java @@ -0,0 +1,840 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.apache.lucene.tests.util.LuceneTestCase; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.lease.Releasable; +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.core.action.ActionListener; +import org.opensearch.gateway.remote.model.RemotePinnedTimestamps; +import org.opensearch.gateway.remote.model.RemoteStorePinnedTimestampsBlobStore; +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; +import org.opensearch.index.translog.transfer.TranslogTransferManager; +import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.indices.DefaultRemoteStoreSettings; +import org.opensearch.indices.RemoteStoreSettings; +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; +import org.opensearch.threadpool.ThreadPool; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.CountDownLatch; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.LongStream; + +import org.mockito.Mockito; + +import static org.opensearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; +import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED; +import static org.junit.Assert.fail; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@LuceneTestCase.SuppressFileSystems("ExtrasFS") +public class RemoteFsTranslogWithPinnedTimestampTests extends RemoteFsTranslogTests { + + Runnable updatePinnedTimstampTask; + BlobStoreTransferService pinnedTimestampBlobStoreTransferService; + RemoteStorePinnedTimestampsBlobStore remoteStorePinnedTimestampsBlobStore; + RemoteStorePinnedTimestampService remoteStorePinnedTimestampServiceSpy; + + @Before + public void setUp() throws Exception { + super.setUp(); + + RemoteStoreSettings remoteStoreSettings = new RemoteStoreSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PINNED_TIMESTAMP_ENABLED.getKey(), true).build(), + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + + 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 threadPool = mock(ThreadPool.class); + when(threadPool.schedule(any(), any(), any())).then(invocationOnMock -> { + Runnable updateTask = invocationOnMock.getArgument(0); + updatePinnedTimstampTask = () -> { + long currentTime = System.currentTimeMillis(); + while (RemoteStorePinnedTimestampService.getPinnedTimestamps().v1() < currentTime) { + updateTask.run(); + } + }; + return null; + }).then(subsequentInvocationsOnMock -> null); + + RepositoryMetadata repositoryMetadata = new RepositoryMetadata(randomAlphaOfLength(10), FsRepository.TYPE, settings); + final ClusterService clusterService = BlobStoreTestUtil.mockClusterService(repositoryMetadata); + + RemoteStorePinnedTimestampService remoteStorePinnedTimestampService = new RemoteStorePinnedTimestampService( + repositoriesServiceSupplier, + settings, + threadPool, + clusterService + ); + remoteStorePinnedTimestampServiceSpy = Mockito.spy(remoteStorePinnedTimestampService); + + remoteStorePinnedTimestampsBlobStore = mock(RemoteStorePinnedTimestampsBlobStore.class); + pinnedTimestampBlobStoreTransferService = mock(BlobStoreTransferService.class); + when(remoteStorePinnedTimestampServiceSpy.pinnedTimestampsBlobStore()).thenReturn(remoteStorePinnedTimestampsBlobStore); + when(remoteStorePinnedTimestampServiceSpy.blobStoreTransferService()).thenReturn(pinnedTimestampBlobStoreTransferService); + + doAnswer(invocationOnMock -> { + ActionListener> actionListener = invocationOnMock.getArgument(3); + actionListener.onResponse(new ArrayList<>()); + return null; + }).when(pinnedTimestampBlobStoreTransferService).listAllInSortedOrder(any(), any(), eq(1), any()); + + remoteStorePinnedTimestampServiceSpy.start(); + } + + public void testGetMinMaxTranslogGenerationFromFilename() throws Exception { + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + ArrayList ops = new ArrayList<>(); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("0", 0, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 2, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 3, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 4, primaryTerm.get(), new byte[] { 1 })); + + CountDownLatch latch = new CountDownLatch(1); + blobStoreTransferService.listAllInSortedOrder( + getTranslogDirectory().add(METADATA_DIR), + "metadata", + Integer.MAX_VALUE, + new LatchedActionListener<>(new ActionListener>() { + @Override + public void onResponse(List blobMetadataList) { + Long minGen = 1L; + Long maxGen = 6L; + for (BlobMetadata blobMetadata : blobMetadataList) { + Tuple minMaxGen = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(blobMetadata.name()); + assertEquals(minGen, minMaxGen.v1()); + assertEquals(maxGen, minMaxGen.v2()); + maxGen -= 1; + } + } + + @Override + public void onFailure(Exception e) { + // This means test failure + fail(); + } + }, latch) + ); + latch.await(); + + // Old format metadata file + String oldFormatMdFilename = "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1"; + assertNull(TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(oldFormatMdFilename)); + + // Node id containing separator + String nodeIdWithSeparator = + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__node__1__9223372036438563958__1"; + Tuple minMaxGen = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(nodeIdWithSeparator); + Long minGen = Long.MAX_VALUE - 9223372036438563958L; + assertEquals(minGen, minMaxGen.v1()); + + // Malformed md filename + String malformedMdFileName = "metadata__9223372036438563903__9223372036854774799__9223370311919910393__node1__xyz__1"; + assertNull(TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(malformedMdFileName)); + } + + public void testIndexDeletionWithNoPinnedTimestampNoRecentMdFiles() throws Exception { + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + ArrayList ops = new ArrayList<>(); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("0", 0, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 2, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 3, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 4, primaryTerm.get(), new byte[] { 1 })); + + assertBusy(() -> { + assertEquals(5, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + assertEquals( + 12, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(true, false); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + + assertBusy(() -> { + assertEquals(0, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + assertEquals( + 0, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + } + + public void testIndexDeletionWithNoPinnedTimestampButRecentFiles() throws Exception { + ArrayList ops = new ArrayList<>(); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("0", 0, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 2, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 3, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 4, primaryTerm.get(), new byte[] { 1 })); + + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(true, false); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertBusy(() -> { + assertEquals(5, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + assertEquals( + 12, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + } + + @Override + public void testSimpleOperationsUpload() throws Exception { + ArrayList ops = new ArrayList<>(); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("0", 0, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); + + // First reader is created at the init of translog + assertEquals(3, translog.readers.size()); + assertEquals(2, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + assertBusy(() -> { + assertEquals(6, translog.allUploaded().size()); + assertEquals( + 6, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 2, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 3, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 4, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("5", 5, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("6", 6, primaryTerm.get(), new byte[] { 1 })); + + assertBusy(() -> { + assertEquals( + 16, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + + translog.setMinSeqNoToKeep(4); + translog.trimUnreferencedReaders(); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("7", 7, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("8", 8, primaryTerm.get(), new byte[] { 1 })); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals(5, translog.readers.size()); + assertBusy(() -> { + assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + assertEquals(10, translog.allUploaded().size()); + assertEquals( + 10, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + }); + } + + @Override + public void testMetadataFileDeletion() throws Exception { + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + ArrayList ops = new ArrayList<>(); + // Test deletion of metadata files + int numDocs = randomIntBetween(6, 10); + for (int i = 0; i < numDocs; i++) { + addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(i), i, primaryTerm.get(), new byte[] { 1 })); + translog.setMinSeqNoToKeep(i); + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals(1, translog.readers.size()); + } + assertBusy(() -> assertEquals(2, translog.allUploaded().size())); + addToTranslogAndListAndUpload( + translog, + ops, + new Translog.Index(String.valueOf(numDocs), numDocs, primaryTerm.get(), new byte[] { 1 }) + ); + addToTranslogAndListAndUpload( + translog, + ops, + new Translog.Index(String.valueOf(numDocs + 1), numDocs + 1, primaryTerm.get(), new byte[] { 1 }) + ); + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> { assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); }); + } + + public void testMetadataFileDeletionWithPinnedTimestamps() throws Exception { + ArrayList ops = new ArrayList<>(); + // Test deletion of metadata files + int numDocs = randomIntBetween(16, 20); + for (int i = 0; i < numDocs; i++) { + addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(i), i, primaryTerm.get(), new byte[] { 1 })); + translog.setMinSeqNoToKeep(i); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + translog.trimUnreferencedReaders(); + // This is just to make sure that each metadata is at least 1ms apart + Thread.sleep(1); + } + + CountDownLatch latch = new CountDownLatch(1); + blobStoreTransferService.listAllInSortedOrder( + getTranslogDirectory().add(METADATA_DIR), + "metadata", + Integer.MAX_VALUE, + new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(List blobMetadataList) { + List pinnedTimestampMatchingMetadataFiles = new ArrayList<>(); + List pinnedTimestamps = new ArrayList<>(); + for (BlobMetadata blobMetadata : blobMetadataList) { + String metadataFilename = blobMetadata.name(); + if (randomBoolean()) { + long timestamp = RemoteStoreUtils.invertLong(metadataFilename.split(METADATA_SEPARATOR)[3]); + pinnedTimestamps.add(timestamp); + pinnedTimestampMatchingMetadataFiles.add(metadataFilename); + } + } + + doAnswer(invocationOnMock -> { + ActionListener> actionListener = invocationOnMock.getArgument(3); + actionListener.onResponse(List.of(new PlainBlobMetadata("pinned_timestamp_123", 1000))); + return null; + }).when(pinnedTimestampBlobStoreTransferService).listAllInSortedOrder(any(), any(), eq(1), any()); + + Map> pinnedTimestampsMap = new HashMap<>(); + pinnedTimestamps.forEach(ts -> pinnedTimestampsMap.put(ts, new ArrayList<>())); + + try { + when(remoteStorePinnedTimestampsBlobStore.read(any())).thenReturn( + new RemotePinnedTimestamps.PinnedTimestamps(pinnedTimestampsMap) + ); + when(remoteStorePinnedTimestampsBlobStore.getBlobPathForUpload(any())).thenReturn(new BlobPath()); + + Set dataFilesBeforeTrim = blobStoreTransferService.listAll( + getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get())) + ); + + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + updatePinnedTimstampTask.run(); + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + + Set metadataFilesAfterTrim = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); + Set dataFilesAfterTrim = blobStoreTransferService.listAll( + getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get())) + ); + + // If non pinned generations are within, minRemoteGenReferenced - 1 - indexSettings().getRemoteTranslogExtraKeep() + // we will not delete them + if (dataFilesAfterTrim.equals(dataFilesBeforeTrim) == false) { + // We check for number of pinned timestamp or +1 due to latest metadata. + assertTrue( + metadataFilesAfterTrim.size() == pinnedTimestamps.size() + || metadataFilesAfterTrim.size() == pinnedTimestamps.size() + 1 + ); + } + + for (String md : pinnedTimestampMatchingMetadataFiles) { + assertTrue(metadataFilesAfterTrim.contains(md)); + Tuple minMaXGen = TranslogTransferMetadata.getMinMaxTranslogGenerationFromFilename(md); + for (long i = minMaXGen.v1(); i <= minMaXGen.v2(); i++) { + assertTrue(dataFilesAfterTrim.contains(Translog.getFilename(i))); + } + } + } catch (Exception e) { + fail(); + } + } + + @Override + public void onFailure(Exception e) { + fail(); + } + }, latch) + ); + + latch.await(); + } + + @Override + public void testDrainSync() throws Exception { + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + + // This test checks following scenarios - + // 1. During ongoing uploads, the available permits are 0. + // 2. During an upload, if drainSync is called, it will wait for it to acquire and available permits are 0. + // 3. After drainSync, if trimUnreferencedReaders is attempted, we do not delete from remote store. + // 4. After drainSync, if an upload is an attempted, we do not upload to remote store. + ArrayList ops = new ArrayList<>(); + assertEquals(0, translog.allUploaded().size()); + assertEquals(1, translog.readers.size()); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(0), 0, primaryTerm.get(), new byte[] { 1 })); + assertEquals(4, translog.allUploaded().size()); + assertEquals(2, translog.readers.size()); + assertBusy(() -> assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size())); + + translog.setMinSeqNoToKeep(0); + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals(1, translog.readers.size()); + + // Case 1 - During ongoing uploads, the available permits are 0. + slowDown.setSleepSeconds(2); + CountDownLatch latch = new CountDownLatch(1); + Thread thread1 = new Thread(() -> { + try { + addToTranslogAndListAndUpload(translog, ops, new Translog.Index(String.valueOf(1), 1, primaryTerm.get(), new byte[] { 1 })); + assertEquals(2, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + latch.countDown(); + } catch (IOException e) { + throw new AssertionError(e); + } + }); + thread1.start(); + assertBusy(() -> assertEquals(0, translog.availablePermits())); + // Case 2 - During an upload, if drainSync is called, it will wait for it to acquire and available permits are 0. + Releasable releasable = translog.drainSync(); + assertBusy(() -> assertEquals(0, latch.getCount())); + assertEquals(0, translog.availablePermits()); + slowDown.setSleepSeconds(0); + assertEquals(4, translog.allUploaded().size()); + assertEquals(2, translog.readers.size()); + Set mdFiles = blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)); + + // Case 3 - After drainSync, if trimUnreferencedReaders is attempted, we do not delete from remote store. + translog.setMinSeqNoToKeep(1); + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals(1, translog.readers.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. + Translog.Location loc = addToTranslogAndListAndUpload( + translog, + ops, + new Translog.Index(String.valueOf(2), 2, primaryTerm.get(), new byte[] { 1 }) + ); + assertEquals(1, translog.readers.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(4, translog.allUploaded().size()); + assertEquals(3, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size()); + + translog.setMinSeqNoToKeep(3); + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals(1, translog.readers.size()); + assertBusy(() -> assertEquals(2, translog.allUploaded().size())); + assertBusy(() -> assertEquals(1, blobStoreTransferService.listAll(getTranslogDirectory().add(METADATA_DIR)).size())); + } + + @Override + public void testExtraGenToKeep() throws Exception { + RemoteStoreSettings.setPinnedTimestampsLookbackInterval(TimeValue.ZERO); + + TranslogConfig config = getConfig(1); + ChannelFactory channelFactory = getChannelFactory(); + final Set persistedSeqNos = new HashSet<>(); + String translogUUID = Translog.createEmptyTranslog( + config.getTranslogPath(), + SequenceNumbers.NO_OPS_PERFORMED, + shardId, + channelFactory, + primaryTerm.get() + ); + TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(config.getIndexSettings()); + ArrayList ops = new ArrayList<>(); + try ( + RemoteFsTranslog translog = new RemoteFsTranslog( + config, + translogUUID, + deletionPolicy, + () -> SequenceNumbers.NO_OPS_PERFORMED, + primaryTerm::get, + persistedSeqNos::add, + repository, + threadPool, + () -> Boolean.TRUE, + new RemoteTranslogTransferTracker(shardId, 10), + DefaultRemoteStoreSettings.INSTANCE + ) { + @Override + ChannelFactory getChannelFactory() { + return channelFactory; + } + } + ) { + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 1, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 1 })); + + // expose the new checkpoint (simulating a commit), before we trim the translog + translog.setMinSeqNoToKeep(2); + + // Trims from local + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("4", 3, primaryTerm.get(), new byte[] { 1 })); + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("5", 4, primaryTerm.get(), new byte[] { 1 })); + // Trims from remote now + // Fetch pinned timestamps so that it won't be stale + updatePinnedTimstampTask.run(); + translog.trimUnreferencedReaders(); + assertBusy(() -> assertTrue(translog.isRemoteGenerationDeletionPermitsAvailable())); + assertEquals( + 8, + blobStoreTransferService.listAll(getTranslogDirectory().add(DATA_DIR).add(String.valueOf(primaryTerm.get()))).size() + ); + } + } + + public void testGetGenerationsToBeDeletedEmptyMetadataFilesNotToBeDeleted() throws IOException { + List metadataFilesNotToBeDeleted = new ArrayList<>(); + List metadataFilesToBeDeleted = List.of( + // 4 to 7 + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__9223372036854775803__1", + // 17 to 37 + "metadata__9223372036438563903__9223372036854775770__9223370311919910398__31__9223372036854775790__1", + // 27 to 42 + "metadata__9223372036438563903__9223372036854775765__9223370311919910403__31__9223372036854775780__1" + ); + Set generations = translog.getGenerationsToBeDeleted(metadataFilesNotToBeDeleted, metadataFilesToBeDeleted, true); + Set md1Generations = LongStream.rangeClosed(4, 7).boxed().collect(Collectors.toSet()); + Set md2Generations = LongStream.rangeClosed(17, 37).boxed().collect(Collectors.toSet()); + Set md3Generations = LongStream.rangeClosed(27, 42).boxed().collect(Collectors.toSet()); + assertTrue(generations.containsAll(md1Generations)); + assertTrue(generations.containsAll(md2Generations)); + assertTrue(generations.containsAll(md3Generations)); + + generations.removeAll(md1Generations); + generations.removeAll(md2Generations); + generations.removeAll(md3Generations); + assertTrue(generations.isEmpty()); + } + + public void testGetGenerationsToBeDeleted() throws IOException { + List metadataFilesNotToBeDeleted = List.of( + // 1 to 4 + "metadata__9223372036438563903__9223372036854775803__9223370311919910398__31__9223372036854775806__1", + // 26 to 30 + "metadata__9223372036438563903__9223372036854775777__9223370311919910398__31__9223372036854775781__1", + // 42 to 100 + "metadata__9223372036438563903__9223372036854775707__9223370311919910403__31__9223372036854775765__1" + ); + List metadataFilesToBeDeleted = List.of( + // 4 to 7 + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__9223372036854775803__1", + // 17 to 37 + "metadata__9223372036438563903__9223372036854775770__9223370311919910398__31__9223372036854775790__1", + // 27 to 42 + "metadata__9223372036438563903__9223372036854775765__9223370311919910403__31__9223372036854775780__1" + ); + Set generations = translog.getGenerationsToBeDeleted(metadataFilesNotToBeDeleted, metadataFilesToBeDeleted, true); + Set md1Generations = LongStream.rangeClosed(5, 7).boxed().collect(Collectors.toSet()); + Set md2Generations = LongStream.rangeClosed(17, 25).boxed().collect(Collectors.toSet()); + Set md3Generations = LongStream.rangeClosed(31, 41).boxed().collect(Collectors.toSet()); + assertTrue(generations.containsAll(md1Generations)); + assertTrue(generations.containsAll(md2Generations)); + assertTrue(generations.containsAll(md3Generations)); + + generations.removeAll(md1Generations); + generations.removeAll(md2Generations); + generations.removeAll(md3Generations); + assertTrue(generations.isEmpty()); + } + + public void testGetMetadataFilesToBeDeletedNoExclusion() { + updatePinnedTimstampTask.run(); + + List metadataFiles = List.of( + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__9223372036854775106__1", + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__9223372036854775803__1", + "metadata__9223372036438563903__9223372036854775701__9223370311919910403__31__9223372036854775701__1" + ); + + assertEquals(metadataFiles, translog.getMetadataFilesToBeDeleted(metadataFiles)); + } + + public void testGetMetadataFilesToBeDeletedExclusionBasedOnAgeOnly() { + updatePinnedTimstampTask.run(); + long currentTimeInMillis = System.currentTimeMillis(); + String md1Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 100000); + String md2Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis + 30000); + String md3Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis + 60000); + + List metadataFiles = List.of( + "metadata__9223372036438563903__9223372036854774799__" + md1Timestamp + "__31__9223372036854775106__1", + "metadata__9223372036438563903__9223372036854775800__" + md2Timestamp + "__31__9223372036854775803__1", + "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" + ); + + List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + assertEquals(1, metadataFilesToBeDeleted.size()); + assertEquals(metadataFiles.get(0), metadataFilesToBeDeleted.get(0)); + } + + public void testGetMetadataFilesToBeDeletedExclusionBasedOnPinningOnly() throws IOException { + long currentTimeInMillis = System.currentTimeMillis(); + String md1Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 100000); + String md2Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 300000); + String md3Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 600000); + + doAnswer(invocationOnMock -> { + ActionListener> actionListener = invocationOnMock.getArgument(3); + actionListener.onResponse(List.of(new PlainBlobMetadata("pinned_timestamp_123", 1000))); + return null; + }).when(pinnedTimestampBlobStoreTransferService).listAllInSortedOrder(any(), any(), eq(1), any()); + + long pinnedTimestamp = RemoteStoreUtils.invertLong(md2Timestamp) + 10000; + when(remoteStorePinnedTimestampsBlobStore.read(any())).thenReturn( + new RemotePinnedTimestamps.PinnedTimestamps(Map.of(pinnedTimestamp, List.of("xyz"))) + ); + when(remoteStorePinnedTimestampsBlobStore.getBlobPathForUpload(any())).thenReturn(new BlobPath()); + + updatePinnedTimstampTask.run(); + + List metadataFiles = List.of( + "metadata__9223372036438563903__9223372036854774799__" + md1Timestamp + "__31__9223372036854775106__1", + "metadata__9223372036438563903__9223372036854775600__" + md2Timestamp + "__31__9223372036854775803__1", + "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" + ); + + List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + assertEquals(2, metadataFilesToBeDeleted.size()); + assertEquals(metadataFiles.get(0), metadataFilesToBeDeleted.get(0)); + assertEquals(metadataFiles.get(2), metadataFilesToBeDeleted.get(1)); + } + + public void testGetMetadataFilesToBeDeletedExclusionBasedOnAgeAndPinning() throws IOException { + long currentTimeInMillis = System.currentTimeMillis(); + String md1Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis + 100000); + String md2Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 300000); + String md3Timestamp = RemoteStoreUtils.invertLong(currentTimeInMillis - 600000); + + doAnswer(invocationOnMock -> { + ActionListener> actionListener = invocationOnMock.getArgument(3); + actionListener.onResponse(List.of(new PlainBlobMetadata("pinned_timestamp_123", 1000))); + return null; + }).when(pinnedTimestampBlobStoreTransferService).listAllInSortedOrder(any(), any(), eq(1), any()); + + long pinnedTimestamp = RemoteStoreUtils.invertLong(md2Timestamp) + 10000; + when(remoteStorePinnedTimestampsBlobStore.read(any())).thenReturn( + new RemotePinnedTimestamps.PinnedTimestamps(Map.of(pinnedTimestamp, List.of("xyz"))) + ); + when(remoteStorePinnedTimestampsBlobStore.getBlobPathForUpload(any())).thenReturn(new BlobPath()); + + updatePinnedTimstampTask.run(); + + List metadataFiles = List.of( + "metadata__9223372036438563903__9223372036854774799__" + md1Timestamp + "__31__9223372036854775106__1", + "metadata__9223372036438563903__9223372036854775600__" + md2Timestamp + "__31__9223372036854775803__1", + "metadata__9223372036438563903__9223372036854775701__" + md3Timestamp + "__31__9223372036854775701__1" + ); + + List metadataFilesToBeDeleted = translog.getMetadataFilesToBeDeleted(metadataFiles); + assertEquals(1, metadataFilesToBeDeleted.size()); + assertEquals(metadataFiles.get(2), metadataFilesToBeDeleted.get(0)); + } + + public void testIsGenerationPinned() { + TreeSet> pinnedGenerations = new TreeSet<>(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.add(new Tuple<>(12L, 34L)); + pinnedGenerations.add(new Tuple<>(121L, 140L)); + pinnedGenerations.add(new Tuple<>(142L, 160L)); + pinnedGenerations.add(new Tuple<>(12L, 120L)); + pinnedGenerations.add(new Tuple<>(12L, 78L)); + pinnedGenerations.add(new Tuple<>(142L, 170L)); + pinnedGenerations.add(new Tuple<>(1L, 1L)); + pinnedGenerations.add(new Tuple<>(12L, 56L)); + pinnedGenerations.add(new Tuple<>(142L, 180L)); + pinnedGenerations.add(new Tuple<>(4L, 9L)); + + assertFalse(translog.isGenerationPinned(3, pinnedGenerations)); + assertFalse(translog.isGenerationPinned(10, pinnedGenerations)); + assertFalse(translog.isGenerationPinned(141, pinnedGenerations)); + assertFalse(translog.isGenerationPinned(181, pinnedGenerations)); + assertFalse(translog.isGenerationPinned(5000, pinnedGenerations)); + assertFalse(translog.isGenerationPinned(0, pinnedGenerations)); + + assertTrue(translog.isGenerationPinned(1, pinnedGenerations)); + assertTrue(translog.isGenerationPinned(120, pinnedGenerations)); + assertTrue(translog.isGenerationPinned(121, pinnedGenerations)); + assertTrue(translog.isGenerationPinned(156, pinnedGenerations)); + assertTrue(translog.isGenerationPinned(12, pinnedGenerations)); + } + + public void testGetMinMaxTranslogGenerationFromMetadataFile() throws IOException { + TranslogTransferManager translogTransferManager = mock(TranslogTransferManager.class); + + // Fetch generations directly from the filename + assertEquals( + new Tuple<>(701L, 1008L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__9223372036854775106__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(4L, 7L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__9223372036854775803__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(106L, 106L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854775701__9223370311919910403__31__9223372036854775701__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(4573L, 99964L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854675843__9223370311919910408__31__9223372036854771234__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(1L, 4L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854775803__9223370311919910413__31__9223372036854775806__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(2474L, 3462L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854772345__9223370311919910429__31__9223372036854773333__1", + translogTransferManager + ) + ); + assertEquals( + new Tuple<>(5807L, 7917L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854767890__9223370311919910434__31__9223372036854770000__1", + translogTransferManager + ) + ); + + // For older md filenames, it needs to read the content + TranslogTransferMetadata md1 = mock(TranslogTransferMetadata.class); + when(md1.getMinTranslogGeneration()).thenReturn(701L); + when(md1.getGeneration()).thenReturn(1008L); + when(translogTransferManager.readMetadata("metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1")) + .thenReturn(md1); + assertEquals( + new Tuple<>(701L, 1008L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1", + translogTransferManager + ) + ); + TranslogTransferMetadata md2 = mock(TranslogTransferMetadata.class); + when(md2.getMinTranslogGeneration()).thenReturn(4L); + when(md2.getGeneration()).thenReturn(7L); + when(translogTransferManager.readMetadata("metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1")) + .thenReturn(md2); + assertEquals( + new Tuple<>(4L, 7L), + translog.getMinMaxTranslogGenerationFromMetadataFile( + "metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1", + translogTransferManager + ) + ); + + verify(translogTransferManager).readMetadata("metadata__9223372036438563903__9223372036854774799__9223370311919910393__31__1"); + verify(translogTransferManager).readMetadata("metadata__9223372036438563903__9223372036854775800__9223370311919910398__31__1"); + } +} 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 ffa6a0f456f36..2426a14c0c93b 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); + } }