diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 20988ff4148a8..fe3345be86c2a 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -90,8 +90,8 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.MapperService.MergeReason; import org.opensearch.index.query.QueryShardContext; -import org.opensearch.index.remote.RemoteStorePathResolver; import org.opensearch.index.remote.RemoteStorePathType; +import org.opensearch.index.remote.RemoteStorePathTypeResolver; import org.opensearch.index.shard.IndexSettingProvider; import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndexCreationException; @@ -115,6 +115,7 @@ 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.concurrent.atomic.AtomicInteger; @@ -174,7 +175,7 @@ public class MetadataCreateIndexService { private AwarenessReplicaBalance awarenessReplicaBalance; @Nullable - private final RemoteStorePathResolver remoteStorePathResolver; + private final RemoteStorePathTypeResolver remoteStorePathTypeResolver; public MetadataCreateIndexService( final Settings settings, @@ -207,8 +208,8 @@ public MetadataCreateIndexService( // Task is onboarded for throttling, it will get retried from associated TransportClusterManagerNodeAction. createIndexTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.CREATE_INDEX_KEY, true); - remoteStorePathResolver = isRemoteDataAttributePresent(settings) - ? new RemoteStorePathResolver(clusterService.getClusterSettings()) + remoteStorePathTypeResolver = isRemoteDataAttributePresent(settings) + ? new RemoteStorePathTypeResolver(clusterService.getClusterSettings()) : null; } @@ -557,7 +558,7 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( tmpImdBuilder.setRoutingNumShards(routingNumShards); tmpImdBuilder.settings(indexSettings); tmpImdBuilder.system(isSystem); - addRemoteCustomData(tmpImdBuilder); + addRemoteStorePathTypeInCustomData(tmpImdBuilder, true); // Set up everything, now locally create the index to see that things are ok, and apply IndexMetadata tempMetadata = tmpImdBuilder.build(); @@ -566,8 +567,14 @@ IndexMetadata buildAndValidateTemporaryIndexMetadata( return tempMetadata; } - public void addRemoteCustomData(IndexMetadata.Builder tmpImdBuilder) { - if (remoteStorePathResolver != null) { + /** + * Adds the remote store path type information in custom data of index metadata. + * + * @param tmpImdBuilder index metadata builder. + * @param assertNullOldType flag to verify that the old remote store path type is null + */ + public void addRemoteStorePathTypeInCustomData(IndexMetadata.Builder tmpImdBuilder, boolean assertNullOldType) { + if (remoteStorePathTypeResolver != null) { // It is possible that remote custom data exists already. In such cases, we need to only update the path type // in the remote store custom data map. Map existingRemoteCustomData = tmpImdBuilder.removeCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); @@ -575,8 +582,9 @@ public void addRemoteCustomData(IndexMetadata.Builder tmpImdBuilder) { ? new HashMap<>() : new HashMap<>(existingRemoteCustomData); // Determine the path type for use using the remoteStorePathResolver. - String newPathType = remoteStorePathResolver.resolveType().toString(); + String newPathType = remoteStorePathTypeResolver.getType().toString(); String oldPathType = remoteCustomData.put(RemoteStorePathType.NAME, newPathType); + assert !assertNullOldType || Objects.isNull(oldPathType); logger.trace(() -> new ParameterizedMessage("Added new path type {}, replaced old path type {}", newPathType, oldPathType)); tmpImdBuilder.putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, remoteCustomData); } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 343152d5e0a47..49e8afaf4d658 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -510,7 +510,8 @@ public synchronized IndexShard createShard( remoteDirectory = ((RemoteSegmentStoreDirectoryFactory) remoteDirectoryFactory).newDirectory( RemoteStoreNodeAttribute.getRemoteStoreSegmentRepo(this.indexSettings.getNodeSettings()), this.indexSettings.getUUID(), - shardId + shardId, + this.indexSettings.getRemoteStorePathType() ); } remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index c7f8199cdef47..f5517b40b3afe 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -49,6 +49,7 @@ import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.Index; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.translog.Translog; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.ingest.IngestService; @@ -60,6 +61,7 @@ import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; @@ -1903,4 +1905,11 @@ public double getDocIdFuzzySetFalsePositiveProbability() { public void setDocIdFuzzySetFalsePositiveProbability(double docIdFuzzySetFalsePositiveProbability) { this.docIdFuzzySetFalsePositiveProbability = docIdFuzzySetFalsePositiveProbability; } + + public RemoteStorePathType getRemoteStorePathType() { + Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + return remoteCustomData != null && remoteCustomData.containsKey(RemoteStorePathType.NAME) + ? RemoteStorePathType.parseString(remoteCustomData.get(RemoteStorePathType.NAME)) + : RemoteStorePathType.FIXED; + } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java new file mode 100644 index 0000000000000..1afd73bf1f1b3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreDataEnums.java @@ -0,0 +1,65 @@ +/* + * 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.remote; + +import java.util.Set; + +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA; + +/** + * This class contains the different enums related to remote store data categories and types. + * + * @opensearch.internal + */ +public class RemoteStoreDataEnums { + + /** + * Categories of the data in Remote store. + */ + public enum DataCategory { + SEGMENTS("segments", Set.of(DataType.values())), + TRANSLOG("translog", Set.of(DATA, METADATA)); + + private final String name; + private final Set supportedDataTypes; + + DataCategory(String name, Set supportedDataTypes) { + this.name = name; + this.supportedDataTypes = supportedDataTypes; + } + + public boolean isSupportedDataType(DataType dataType) { + return supportedDataTypes.contains(dataType); + } + + public String getName() { + return name; + } + } + + /** + * Types of data in remote store. + */ + public enum DataType { + DATA("data"), + METADATA("metadata"), + LOCK_FILES("lock_files"); + + private final String name; + + DataType(String name) { + this.name = name; + } + + public String getName() { + return name; + } + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java index a64e07ab1f66f..d7d7a8cdfd644 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathType.java @@ -8,6 +8,10 @@ package org.opensearch.index.remote; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory; +import org.opensearch.index.remote.RemoteStoreDataEnums.DataType; + import java.util.Locale; /** @@ -18,13 +22,46 @@ */ public enum RemoteStorePathType { - FIXED, - HASHED_PREFIX; + FIXED { + @Override + public BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType) { + return basePath.add(indexUUID).add(shardId).add(dataCategory).add(dataType); + } + }, + HASHED_PREFIX { + @Override + public BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType) { + // TODO - We need to implement this, keeping the same path as Fixed for sake of multiple tests that can fail otherwise. + // throw new UnsupportedOperationException("Not implemented"); --> Not using this for unblocking couple of tests. + return basePath.add(indexUUID).add(shardId).add(dataCategory).add(dataType); + } + }; + + /** + * @param basePath base path of the underlying blob store repository + * @param indexUUID of the index + * @param shardId shard id + * @param dataCategory is either translog or segment + * @param dataType can be one of data, metadata or lock_files. + * @return the blob path for the underlying remote store path type. + */ + public BlobPath path(BlobPath basePath, String indexUUID, String shardId, DataCategory dataCategory, DataType dataType) { + assert dataCategory.isSupportedDataType(dataType) : "category:" + + dataCategory + + " type:" + + dataType + + " are not supported together"; + return generatePath(basePath, indexUUID, shardId, dataCategory.getName(), dataType.getName()); + } + + abstract BlobPath generatePath(BlobPath basePath, String indexUUID, String shardId, String dataCategory, String dataType); public static RemoteStorePathType parseString(String remoteStoreBlobPathType) { try { return RemoteStorePathType.valueOf(remoteStoreBlobPathType.toUpperCase(Locale.ROOT)); - } catch (IllegalArgumentException e) { + } catch (IllegalArgumentException | NullPointerException e) { + // IllegalArgumentException is thrown when the input does not match any enum name + // NullPointerException is thrown when the input is null throw new IllegalArgumentException("Could not parse RemoteStorePathType for [" + remoteStoreBlobPathType + "]"); } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java similarity index 50% rename from server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java rename to server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java index 6e8126fcce0ca..5d014c9862d45 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathResolver.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathTypeResolver.java @@ -16,15 +16,20 @@ * * @opensearch.internal */ -public class RemoteStorePathResolver { +public class RemoteStorePathTypeResolver { - private final ClusterSettings clusterSettings; + private volatile RemoteStorePathType type; - public RemoteStorePathResolver(ClusterSettings clusterSettings) { - this.clusterSettings = clusterSettings; + public RemoteStorePathTypeResolver(ClusterSettings clusterSettings) { + type = clusterSettings.get(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING); + clusterSettings.addSettingsUpdateConsumer(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING, this::setType); } - public RemoteStorePathType resolveType() { - return clusterSettings.get(IndicesService.CLUSTER_REMOTE_STORE_PATH_PREFIX_TYPE_SETTING); + public RemoteStorePathType getType() { + return type; + } + + public void setType(RemoteStorePathType type) { + this.type = type; } } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 8cdabe936a4d4..0e0189eba206d 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4988,7 +4988,7 @@ public void deleteTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); - RemoteFsTranslog.cleanup(repository, shardId, getThreadPool()); + RemoteFsTranslog.cleanup(repository, shardId, getThreadPool(), indexSettings.getRemoteStorePathType()); } /* @@ -5005,7 +5005,14 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); - RemoteFsTranslog.download(repository, shardId, getThreadPool(), shardPath().resolveTranslog(), logger); + RemoteFsTranslog.download( + repository, + shardId, + getThreadPool(), + shardPath().resolveTranslog(), + indexSettings.getRemoteStorePathType(), + logger + ); } /** diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index d038e50412964..dfcffd8b6de40 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -58,6 +58,7 @@ import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.EngineException; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; @@ -402,7 +403,8 @@ void recoverFromSnapshotAndRemoteStore( RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( remoteStoreRepository, indexUUID, - shardId + shardId, + RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot ); sourceRemoteDirectory.initializeToSpecificCommit( primaryTerm, diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index c9a238c6e3350..999625e0e579f 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -31,6 +31,7 @@ import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; @@ -897,13 +898,15 @@ public static void remoteDirectoryCleanup( RemoteSegmentStoreDirectoryFactory remoteDirectoryFactory, String remoteStoreRepoForIndex, String indexUUID, - ShardId shardId + ShardId shardId, + RemoteStorePathType pathType ) { try { RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) remoteDirectoryFactory.newDirectory( remoteStoreRepoForIndex, indexUUID, - shardId + shardId, + pathType ); remoteSegmentStoreDirectory.deleteStaleSegments(0); remoteSegmentStoreDirectory.deleteIfEmpty(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index d6d3f1fca833c..f0ecd96bcf1f7 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -13,6 +13,7 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexSettings; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManagerFactory; @@ -24,8 +25,13 @@ import org.opensearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.Objects; import java.util.function.Supplier; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA; + /** * Factory for a remote store directory * @@ -33,8 +39,6 @@ */ @PublicApi(since = "2.3.0") public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.DirectoryFactory { - private static final String SEGMENTS = "segments"; - private final Supplier repositoriesService; private final ThreadPool threadPool; @@ -48,29 +52,38 @@ public RemoteSegmentStoreDirectoryFactory(Supplier reposito public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throws IOException { String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); - return newDirectory(repositoryName, indexUUID, path.getShardId()); + return newDirectory(repositoryName, indexUUID, path.getShardId(), indexSettings.getRemoteStorePathType()); } - public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId) throws IOException { + public Directory newDirectory(String repositoryName, String indexUUID, ShardId shardId, RemoteStorePathType pathType) + throws IOException { + assert Objects.nonNull(pathType); try (Repository repository = repositoriesService.get().repository(repositoryName)) { + assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository); - BlobPath commonBlobPath = blobStoreRepository.basePath(); - commonBlobPath = commonBlobPath.add(indexUUID).add(String.valueOf(shardId.id())).add(SEGMENTS); + BlobPath repositoryBasePath = blobStoreRepository.basePath(); + String shardIdStr = String.valueOf(shardId.id()); + // Derive the path for data directory of SEGMENTS + BlobPath dataPath = pathType.path(repositoryBasePath, indexUUID, shardIdStr, SEGMENTS, DATA); RemoteDirectory dataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore().blobContainer(commonBlobPath.add("data")), + blobStoreRepository.blobStore().blobContainer(dataPath), blobStoreRepository::maybeRateLimitRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers ); - RemoteDirectory metadataDirectory = new RemoteDirectory( - blobStoreRepository.blobStore().blobContainer(commonBlobPath.add("metadata")) - ); + + // Derive the path for metadata directory of SEGMENTS + BlobPath mdPath = pathType.path(repositoryBasePath, indexUUID, shardIdStr, SEGMENTS, METADATA); + RemoteDirectory metadataDirectory = new RemoteDirectory(blobStoreRepository.blobStore().blobContainer(mdPath)); + + // The path for lock is derived within the RemoteStoreLockManagerFactory RemoteStoreLockManager mdLockManager = RemoteStoreLockManagerFactory.newLockManager( repositoriesService.get(), repositoryName, indexUUID, - String.valueOf(shardId.id()) + shardIdStr, + pathType ); return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, mdLockManager, threadPool, shardId); diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java index 00666ada11983..c033e4f7ad0aa 100644 --- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java +++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactory.java @@ -11,15 +11,18 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.store.RemoteBufferedOutputDirectory; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.repositories.blobstore.BlobStoreRepository; -import java.io.IOException; import java.util.function.Supplier; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.LOCK_FILES; + /** * Factory for remote store lock manager * @@ -27,34 +30,29 @@ */ @PublicApi(since = "2.8.0") public class RemoteStoreLockManagerFactory { - private static final String SEGMENTS = "segments"; - private static final String LOCK_FILES = "lock_files"; private final Supplier repositoriesService; public RemoteStoreLockManagerFactory(Supplier repositoriesService) { this.repositoriesService = repositoriesService; } - public RemoteStoreLockManager newLockManager(String repositoryName, String indexUUID, String shardId) throws IOException { - return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId); + public RemoteStoreLockManager newLockManager(String repositoryName, String indexUUID, String shardId, RemoteStorePathType pathType) { + return newLockManager(repositoriesService.get(), repositoryName, indexUUID, shardId, pathType); } public static RemoteStoreMetadataLockManager newLockManager( RepositoriesService repositoriesService, String repositoryName, String indexUUID, - String shardId - ) throws IOException { + String shardId, + RemoteStorePathType pathType + ) { try (Repository repository = repositoriesService.repository(repositoryName)) { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; - BlobPath shardLevelBlobPath = ((BlobStoreRepository) repository).basePath().add(indexUUID).add(shardId).add(SEGMENTS); - RemoteBufferedOutputDirectory shardMDLockDirectory = createRemoteBufferedOutputDirectory( - repository, - shardLevelBlobPath, - LOCK_FILES - ); - - return new RemoteStoreMetadataLockManager(shardMDLockDirectory); + BlobPath repositoryBasePath = ((BlobStoreRepository) repository).basePath(); + BlobPath lockDirectoryPath = pathType.path(repositoryBasePath, indexUUID, shardId, SEGMENTS, LOCK_FILES); + BlobContainer lockDirectoryBlobContainer = ((BlobStoreRepository) repository).blobStore().blobContainer(lockDirectoryPath); + return new RemoteStoreMetadataLockManager(new RemoteBufferedOutputDirectory(lockDirectoryBlobContainer)); } catch (RepositoryMissingException e) { throw new IllegalArgumentException("Repository should be present to acquire/release lock", e); } @@ -65,14 +63,4 @@ public static RemoteStoreMetadataLockManager newLockManager( public Supplier getRepositoriesService() { return repositoriesService; } - - private static RemoteBufferedOutputDirectory createRemoteBufferedOutputDirectory( - Repository repository, - BlobPath commonBlobPath, - String extention - ) { - BlobPath extendedPath = commonBlobPath.add(extention); - BlobContainer dataBlobContainer = ((BlobStoreRepository) repository).blobStore().blobContainer(extendedPath); - return new RemoteBufferedOutputDirectory(dataBlobContainer); - } } 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 9b6f18a06f496..a7029b41670ff 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.SetOnce; +import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lease.Releasables; import org.opensearch.common.logging.Loggers; @@ -18,6 +19,7 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.remote.RemoteTranslogTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.translog.transfer.BlobStoreTransferService; @@ -39,6 +41,7 @@ import java.util.HashSet; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; @@ -48,6 +51,10 @@ import java.util.function.LongConsumer; import java.util.function.LongSupplier; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA; + /** * A Translog implementation which syncs local FS with a remote store * The current impl uploads translog , ckp and metadata to remote store @@ -75,7 +82,6 @@ public class RemoteFsTranslog extends Translog { private static final int REMOTE_DELETION_PERMITS = 2; private static final int DOWNLOAD_RETRIES = 2; - public static final String TRANSLOG = "translog"; // Semaphore used to allow only single remote generation to happen at a time private final Semaphore remoteGenerationDeletionPermits = new Semaphore(REMOTE_DELETION_PERMITS); @@ -107,7 +113,8 @@ public RemoteFsTranslog( threadPool, shardId, fileTransferTracker, - remoteTranslogTransferTracker + remoteTranslogTransferTracker, + indexSettings().getRemoteStorePathType() ); try { download(translogTransferManager, location, logger); @@ -151,8 +158,14 @@ RemoteTranslogTransferTracker getRemoteTranslogTracker() { return remoteTranslogTransferTracker; } - public static void download(Repository repository, ShardId shardId, ThreadPool threadPool, Path location, Logger logger) - throws IOException { + public static void download( + Repository repository, + ShardId shardId, + ThreadPool threadPool, + Path location, + RemoteStorePathType pathType, + Logger logger + ) throws IOException { assert repository instanceof BlobStoreRepository : String.format( Locale.ROOT, "%s repository should be instance of BlobStoreRepository", @@ -168,7 +181,8 @@ public static void download(Repository repository, ShardId shardId, ThreadPool t threadPool, shardId, fileTransferTracker, - remoteTranslogTransferTracker + remoteTranslogTransferTracker, + pathType ); RemoteFsTranslog.download(translogTransferManager, location, logger); logger.trace(remoteTranslogTransferTracker.toString()); @@ -267,15 +281,16 @@ public static TranslogTransferManager buildTranslogTransferManager( ThreadPool threadPool, ShardId shardId, FileTransferTracker fileTransferTracker, - RemoteTranslogTransferTracker remoteTranslogTransferTracker + RemoteTranslogTransferTracker tracker, + RemoteStorePathType pathType ) { - return new TranslogTransferManager( - shardId, - new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool), - blobStoreRepository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG), - fileTransferTracker, - remoteTranslogTransferTracker - ); + assert Objects.nonNull(pathType); + String indexUUID = shardId.getIndex().getUUID(); + String shardIdStr = String.valueOf(shardId.id()); + BlobPath dataPath = pathType.path(blobStoreRepository.basePath(), indexUUID, shardIdStr, TRANSLOG, DATA); + BlobPath mdPath = pathType.path(blobStoreRepository.basePath(), indexUUID, shardIdStr, TRANSLOG, METADATA); + BlobStoreTransferService transferService = new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool); + return new TranslogTransferManager(shardId, transferService, dataPath, mdPath, fileTransferTracker, tracker); } @Override @@ -547,7 +562,8 @@ private void deleteStaleRemotePrimaryTerms() { } } - public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool) throws IOException { + public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool, RemoteStorePathType pathType) + throws IOException { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; // We use a dummy stats tracker to ensure the flow doesn't break. @@ -559,7 +575,8 @@ public static void cleanup(Repository repository, ShardId shardId, ThreadPool th threadPool, shardId, fileTransferTracker, - remoteTranslogTransferTracker + remoteTranslogTransferTracker, + pathType ); // clean up all remote translog files translogTransferManager.deleteTranslogFiles(); 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 2f6055df87804..c9e07ca3ef8c1 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 @@ -58,7 +58,6 @@ public class TranslogTransferManager { private final TransferService transferService; private final BlobPath remoteDataTransferPath; private final BlobPath remoteMetadataTransferPath; - private final BlobPath remoteBaseTransferPath; private final FileTransferTracker fileTransferTracker; private final RemoteTranslogTransferTracker remoteTranslogTransferTracker; @@ -67,8 +66,6 @@ public class TranslogTransferManager { private static final int METADATA_FILES_TO_FETCH = 10; private final Logger logger; - private final static String METADATA_DIR = "metadata"; - private final static String DATA_DIR = "data"; private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( new TranslogTransferMetadataHandler(), @@ -79,15 +76,15 @@ public class TranslogTransferManager { public TranslogTransferManager( ShardId shardId, TransferService transferService, - BlobPath remoteBaseTransferPath, + BlobPath remoteDataTransferPath, + BlobPath remoteMetadataTransferPath, FileTransferTracker fileTransferTracker, RemoteTranslogTransferTracker remoteTranslogTransferTracker ) { this.shardId = shardId; this.transferService = transferService; - this.remoteBaseTransferPath = remoteBaseTransferPath; - this.remoteDataTransferPath = remoteBaseTransferPath.add(DATA_DIR); - this.remoteMetadataTransferPath = remoteBaseTransferPath.add(METADATA_DIR); + this.remoteDataTransferPath = remoteDataTransferPath; + this.remoteMetadataTransferPath = remoteMetadataTransferPath; this.fileTransferTracker = fileTransferTracker; this.logger = Loggers.getLogger(getClass(), shardId); this.remoteTranslogTransferTracker = remoteTranslogTransferTracker; @@ -456,17 +453,27 @@ public void onFailure(Exception e) { ); } + /** + * Deletes all the translog content related to the underlying shard. + */ public void delete() { - // cleans up all the translog contents in async fashion - transferService.deleteAsync(ThreadPool.Names.REMOTE_PURGE, remoteBaseTransferPath, new ActionListener<>() { + // Delete the translog data content from the remote store. + delete(remoteDataTransferPath); + // Delete the translog metadata content from the remote store. + delete(remoteMetadataTransferPath); + } + + private void delete(BlobPath path) { + // cleans up all the translog contents in async fashion for the given path + transferService.deleteAsync(ThreadPool.Names.REMOTE_PURGE, path, new ActionListener<>() { @Override public void onResponse(Void unused) { - logger.info("Deleted all remote translog data"); + logger.info("Deleted all remote translog data at path={}", path); } @Override public void onFailure(Exception e) { - logger.error("Exception occurred while cleaning translog", e); + logger.error(new ParameterizedMessage("Exception occurred while cleaning translog at path={}", path), e); } }); } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index e9241bb8d88a4..dac7bc0da0f9f 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -108,6 +108,7 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; @@ -673,7 +674,8 @@ public void cloneRemoteStoreIndexShardSnapshot( RemoteStoreLockManager remoteStoreMetadataLockManger = remoteStoreLockManagerFactory.newLockManager( remoteStoreRepository, indexUUID, - String.valueOf(shardId.shardId()) + String.valueOf(shardId.shardId()), + RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot ); remoteStoreMetadataLockManger.cloneLock( FileLockInfo.getLockInfoBuilder().withAcquirerId(source.getUUID()).build(), @@ -1170,7 +1172,8 @@ public static void remoteDirectoryCleanupAsync( String remoteStoreRepoForIndex, String indexUUID, ShardId shardId, - String threadPoolName + String threadPoolName, + RemoteStorePathType pathType ) { threadpool.executor(threadPoolName) .execute( @@ -1179,7 +1182,8 @@ public static void remoteDirectoryCleanupAsync( remoteDirectoryFactory, remoteStoreRepoForIndex, indexUUID, - shardId + shardId, + pathType ), indexUUID, shardId @@ -1210,7 +1214,8 @@ protected void releaseRemoteStoreLockAndCleanup( RemoteStoreLockManager remoteStoreMetadataLockManager = remoteStoreLockManagerFactory.newLockManager( remoteStoreRepoForIndex, indexUUID, - shardId + shardId, + RemoteStorePathType.HASHED_PREFIX // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot ); remoteStoreMetadataLockManager.release(FileLockInfo.getLockInfoBuilder().withAcquirerId(shallowSnapshotUUID).build()); logger.debug("Successfully released lock for shard {} of index with uuid {}", shardId, indexUUID); @@ -1232,7 +1237,8 @@ protected void releaseRemoteStoreLockAndCleanup( remoteStoreRepoForIndex, indexUUID, new ShardId(Index.UNKNOWN_INDEX_NAME, indexUUID, Integer.parseInt(shardId)), - ThreadPool.Names.REMOTE_PURGE + ThreadPool.Names.REMOTE_PURGE, + RemoteStorePathType.FIXED // TODO - The path type needs to be obtained from RemoteStoreShardShallowCopySnapshot ); } } diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index a1138e3b803fa..cb9ff04fd0a7e 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -484,7 +484,7 @@ public ClusterState execute(ClusterState currentState) { .put(snapshotIndexMetadata.getSettings()) .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) ); - createIndexService.addRemoteCustomData(indexMdBuilder); + createIndexService.addRemoteStorePathTypeInCustomData(indexMdBuilder, false); shardLimitValidator.validateShardLimit( renamedIndexName, snapshotIndexMetadata.getSettings(), diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java new file mode 100644 index 0000000000000..0f108d1b45f5a --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStorePathTypeTests.java @@ -0,0 +1,111 @@ +/* + * 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.remote; + +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory; +import org.opensearch.index.remote.RemoteStoreDataEnums.DataType; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.SEGMENTS; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.DATA; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.LOCK_FILES; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA; +import static org.opensearch.index.remote.RemoteStorePathType.FIXED; +import static org.opensearch.index.remote.RemoteStorePathType.parseString; + +public class RemoteStorePathTypeTests extends OpenSearchTestCase { + + private static final String SEPARATOR = "/"; + + public void testParseString() { + // Case 1 - Pass values from the enum. + String typeString = FIXED.toString(); + RemoteStorePathType type = parseString(randomFrom(typeString, typeString.toLowerCase(Locale.ROOT))); + assertEquals(FIXED, type); + + typeString = RemoteStorePathType.HASHED_PREFIX.toString(); + type = parseString(randomFrom(typeString, typeString.toLowerCase(Locale.ROOT))); + assertEquals(RemoteStorePathType.HASHED_PREFIX, type); + + // Case 2 - Pass random string + String randomTypeString = randomAlphaOfLength(2); + IllegalArgumentException ex = assertThrows(IllegalArgumentException.class, () -> parseString(randomTypeString)); + assertEquals("Could not parse RemoteStorePathType for [" + randomTypeString + "]", ex.getMessage()); + + // Case 3 - Null string + ex = assertThrows(IllegalArgumentException.class, () -> parseString(null)); + assertEquals("Could not parse RemoteStorePathType for [null]", ex.getMessage()); + } + + public void testGeneratePathForFixedType() { + BlobPath blobPath = new BlobPath(); + List pathList = getPathList(); + for (String path : pathList) { + blobPath = blobPath.add(path); + } + + String indexUUID = randomAlphaOfLength(10); + String shardId = String.valueOf(randomInt(100)); + DataCategory dataCategory = TRANSLOG; + DataType dataType = DATA; + + String basePath = getPath(pathList) + indexUUID + SEPARATOR + shardId + SEPARATOR; + // Translog Data + BlobPath result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType); + assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString()); + + // Translog Metadata + dataType = METADATA; + result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType); + assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString()); + + // Translog Lock files - This is a negative case where the assertion will trip. + BlobPath finalBlobPath = blobPath; + assertThrows(AssertionError.class, () -> FIXED.path(finalBlobPath, indexUUID, shardId, TRANSLOG, LOCK_FILES)); + + // Segment Data + dataCategory = SEGMENTS; + dataType = DATA; + result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType); + assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString()); + + // Segment Metadata + dataType = METADATA; + result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType); + assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString()); + + // Segment Metadata + dataType = LOCK_FILES; + result = FIXED.path(blobPath, indexUUID, shardId, dataCategory, dataType); + assertEquals(basePath + dataCategory.getName() + SEPARATOR + dataType.getName() + SEPARATOR, result.buildAsString()); + } + + private List getPathList() { + List pathList = new ArrayList<>(); + int length = randomIntBetween(0, 5); + for (int i = 0; i < length; i++) { + pathList.add(randomAlphaOfLength(randomIntBetween(2, 5))); + } + return pathList; + } + + private String getPath(List pathList) { + String p = String.join(SEPARATOR, pathList); + if (p.isEmpty() || p.endsWith(SEPARATOR)) { + return p; + } + return p + SEPARATOR; + } +} diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 8b69c15dac9d3..59c8a9d92f07b 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -37,6 +37,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; @@ -697,13 +698,20 @@ public void testCleanupAsync() throws Exception { threadPool, indexShard.shardId() ); - when(remoteSegmentStoreDirectoryFactory.newDirectory(any(), any(), any())).thenReturn(remoteSegmentDirectory); + when(remoteSegmentStoreDirectoryFactory.newDirectory(any(), any(), any(), any())).thenReturn(remoteSegmentDirectory); String repositoryName = "test-repository"; String indexUUID = "test-idx-uuid"; ShardId shardId = new ShardId(Index.UNKNOWN_INDEX_NAME, indexUUID, Integer.parseInt("0")); - - RemoteSegmentStoreDirectory.remoteDirectoryCleanup(remoteSegmentStoreDirectoryFactory, repositoryName, indexUUID, shardId); - verify(remoteSegmentStoreDirectoryFactory).newDirectory(repositoryName, indexUUID, shardId); + RemoteStorePathType pathType = randomFrom(RemoteStorePathType.values()); + + RemoteSegmentStoreDirectory.remoteDirectoryCleanup( + remoteSegmentStoreDirectoryFactory, + repositoryName, + indexUUID, + shardId, + pathType + ); + verify(remoteSegmentStoreDirectoryFactory).newDirectory(repositoryName, indexUUID, shardId, pathType); verify(threadPool, times(0)).executor(ThreadPool.Names.REMOTE_PURGE); verify(remoteMetadataDirectory).delete(); verify(remoteDataDirectory).delete(); diff --git a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java index 897785849cf7b..0fe5557737447 100644 --- a/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManagerFactoryTests.java @@ -11,6 +11,7 @@ import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.index.remote.RemoteStorePathType; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.OpenSearchTestCase; @@ -48,6 +49,7 @@ public void testNewLockManager() throws IOException { String testRepository = "testRepository"; String testIndexUUID = "testIndexUUID"; String testShardId = "testShardId"; + RemoteStorePathType pathType = RemoteStorePathType.FIXED; BlobStoreRepository repository = mock(BlobStoreRepository.class); BlobStore blobStore = mock(BlobStore.class); @@ -59,7 +61,12 @@ public void testNewLockManager() throws IOException { when(repositoriesService.repository(testRepository)).thenReturn(repository); - RemoteStoreLockManager lockManager = remoteStoreLockManagerFactory.newLockManager(testRepository, testIndexUUID, testShardId); + RemoteStoreLockManager lockManager = remoteStoreLockManagerFactory.newLockManager( + testRepository, + testIndexUUID, + testShardId, + pathType + ); assertTrue(lockManager != null); ArgumentCaptor blobPathCaptor = ArgumentCaptor.forClass(BlobPath.class); 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 20818c1748cca..dfac716ae5f87 100644 --- a/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFsTranslogTests.java @@ -99,7 +99,7 @@ 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.translog.RemoteFsTranslog.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG; import static org.opensearch.index.translog.SnapshotMatchers.containsOperationsInAnyOrder; import static org.opensearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.contains; @@ -907,7 +907,7 @@ public void testDrainSync() throws Exception { } private BlobPath getTranslogDirectory() { - return repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG); + return repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(TRANSLOG.getName()); } private Long populateTranslogOps(boolean withMissingOps) throws IOException { 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 e34bc078896f9..a9502dc051428 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 @@ -48,6 +48,8 @@ import org.mockito.Mockito; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreDataEnums.DataType.METADATA; import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.anyMap; @@ -95,7 +97,8 @@ public void setUp() throws Exception { translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker ); @@ -159,7 +162,8 @@ public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker ); @@ -194,7 +198,8 @@ public void testTransferSnapshotOnUploadTimeout() throws Exception { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker ); @@ -235,7 +240,8 @@ public void testTransferSnapshotOnThreadInterrupt() throws Exception { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), fileTransferTracker, remoteTranslogTransferTracker ); @@ -333,7 +339,8 @@ public void testReadMetadataNoFile() throws IOException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker ); @@ -354,7 +361,8 @@ public void testReadMetadataFile() throws IOException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker ); @@ -390,7 +398,8 @@ public void testReadMetadataReadException() throws IOException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker ); @@ -426,7 +435,8 @@ public void testReadMetadataListException() throws IOException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker ); @@ -499,7 +509,8 @@ public void testDeleteTranslogSuccess() throws Exception { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, blobStoreTransferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker ); @@ -518,7 +529,8 @@ public void testDeleteStaleTranslogMetadata() { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker ); @@ -569,7 +581,8 @@ public void testDeleteTranslogFailure() throws Exception { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, blobStoreTransferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), tracker, remoteTranslogTransferTracker ); @@ -612,7 +625,8 @@ public void testMetadataConflict() throws InterruptedException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( shardId, transferService, - remoteBaseTransferPath, + remoteBaseTransferPath.add(TRANSLOG.getName()), + remoteBaseTransferPath.add(METADATA.getName()), null, remoteTranslogTransferTracker );