diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java new file mode 100644 index 0000000000000..c39cec96aa476 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreUploadIndexPathIT.java @@ -0,0 +1,108 @@ +/* + * 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.remotestore; + +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.index.remote.RemoteIndexPath; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.io.IOException; +import java.util.Locale; +import java.util.concurrent.ExecutionException; + +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteStoreUploadIndexPathIT extends RemoteStoreBaseIntegTestCase { + + private final String INDEX_NAME = "remote-store-test-idx-1"; + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true).build(); + } + + /** + * Checks that the remote index path file gets created for the intended remote store path type and does not get created + * wherever not required. + */ + public void testRemoteIndexPathFileCreation() throws ExecutionException, InterruptedException, IOException { + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + + // Case 1 - Hashed_prefix, we would need the remote index path file to be created. + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_PREFIX) + ) + .get(); + + createIndex(INDEX_NAME, remoteStoreIndexSettings(0, 1)); + validateRemoteIndexPathFile(true); + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + FileSystemUtils.deleteSubDirectories(translogRepoPath); + FileSystemUtils.deleteSubDirectories(segmentRepoPath); + + // Case 2 - Hashed_infix, we would not have the remote index path file created here. + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.HASHED_INFIX) + ) + .get(); + createIndex(INDEX_NAME, remoteStoreIndexSettings(0, 1)); + validateRemoteIndexPathFile(false); + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + + // Case 3 - fixed, we would not have the remote index path file created here either. + client(clusterManagerNode).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING.getKey(), RemoteStoreEnums.PathType.FIXED)) + .get(); + createIndex(INDEX_NAME, remoteStoreIndexSettings(0, 1)); + validateRemoteIndexPathFile(false); + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + + } + + private void validateRemoteIndexPathFile(boolean exists) { + String indexUUID = client().admin() + .indices() + .prepareGetSettings(INDEX_NAME) + .get() + .getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID); + + assertEquals(exists, FileSystemUtils.exists(translogRepoPath.resolve(RemoteIndexPath.DIR))); + assertEquals( + exists, + FileSystemUtils.exists( + translogRepoPath.resolve(RemoteIndexPath.DIR) + .resolve(String.format(Locale.ROOT, RemoteIndexPath.FILE_NAME_FORMAT, indexUUID)) + ) + ); + assertEquals(exists, FileSystemUtils.exists(segmentRepoPath.resolve(RemoteIndexPath.DIR))); + assertEquals( + exists, + FileSystemUtils.exists( + segmentRepoPath.resolve(RemoteIndexPath.DIR) + .resolve(String.format(Locale.ROOT, RemoteIndexPath.FILE_NAME_FORMAT, indexUUID)) + ) + ); + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java index 2047c99d9e13b..6dc01ca76857b 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java @@ -51,7 +51,7 @@ public class RemoteTransferContainer implements Closeable { private final String remoteFileName; private final boolean failTransferIfFileExists; private final WritePriority writePriority; - private final long expectedChecksum; + private final Long expectedChecksum; private final OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier; private final boolean isRemoteDataIntegritySupported; private final AtomicBoolean readBlock = new AtomicBoolean(); @@ -77,7 +77,7 @@ public RemoteTransferContainer( boolean failTransferIfFileExists, WritePriority writePriority, OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier, - long expectedChecksum, + Long expectedChecksum, boolean isRemoteDataIntegritySupported ) { this.fileName = fileName; @@ -190,7 +190,7 @@ private LocalStreamSupplier getMultipartStreamSupplier( } private boolean isRemoteDataIntegrityCheckPossible() { - return isRemoteDataIntegritySupported; + return isRemoteDataIntegritySupported && Objects.nonNull(expectedChecksum); } private void finalizeUpload(boolean uploadSuccessful) throws IOException { @@ -198,7 +198,7 @@ private void finalizeUpload(boolean uploadSuccessful) throws IOException { return; } - if (uploadSuccessful) { + if (uploadSuccessful && Objects.nonNull(expectedChecksum)) { long actualChecksum = getActualChecksum(); if (actualChecksum != expectedChecksum) { throw new CorruptIndexException( diff --git a/server/src/main/java/org/opensearch/gateway/remote/IndexMetadataUploadListener.java b/server/src/main/java/org/opensearch/gateway/remote/IndexMetadataUploadListener.java new file mode 100644 index 0000000000000..f9158c9260747 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/IndexMetadataUploadListener.java @@ -0,0 +1,49 @@ +/* + * 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.gateway.remote; + +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.core.action.ActionListener; +import org.opensearch.threadpool.ThreadPool; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutorService; + +/** + * Hook for running code that needs to be executed before the upload of index metadata. Here we have introduced a hook + * for index creation (also triggerred after enabling the remote cluster statement for the first time). The listener + * is intended to be run in parallel and async with the index metadata upload. + * + * @opensearch.internal + */ +public abstract class IndexMetadataUploadListener { + + private final ExecutorService executorService; + + public IndexMetadataUploadListener(ThreadPool threadPool, String threadPoolName) { + Objects.requireNonNull(threadPool); + Objects.requireNonNull(threadPoolName); + assert ThreadPool.THREAD_POOL_TYPES.containsKey(threadPoolName) && ThreadPool.Names.SAME.equals(threadPoolName) == false; + this.executorService = threadPool.executor(threadPoolName); + } + + /** + * Runs before the new index upload of index metadata (or first time upload). The caller is expected to trigger + * onSuccess or onFailure of the {@code ActionListener}. + * + * @param indexMetadataList list of index metadata of new indexes (or first time index metadata upload). + * @param actionListener listener to be invoked on success or failure. + */ + public final void onNewIndexUpload(List indexMetadataList, ActionListener actionListener) { + executorService.execute(() -> doOnNewIndexUpload(indexMetadataList, actionListener)); + } + + protected abstract void doOnNewIndexUpload(List indexMetadataList, ActionListener actionListener); +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index c892b475d71da..d2f927c827e5b 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -160,6 +160,7 @@ public class RemoteClusterStateService implements Closeable { private final Settings settings; private final LongSupplier relativeTimeNanosSupplier; private final ThreadPool threadpool; + private final List indexMetadataUploadListeners; private BlobStoreRepository blobStoreRepository; private BlobStoreTransferService blobStoreTransferService; private volatile TimeValue slowWriteLoggingThreshold; @@ -177,6 +178,7 @@ public class RemoteClusterStateService implements Closeable { // ToXContent Params with gateway mode. // We are using gateway context mode to persist all custom metadata. public static final ToXContent.Params FORMAT_PARAMS; + static { Map params = new HashMap<>(1); params.put(Metadata.CONTEXT_MODE_PARAM, Metadata.CONTEXT_MODE_GATEWAY); @@ -189,7 +191,8 @@ public RemoteClusterStateService( Settings settings, ClusterSettings clusterSettings, LongSupplier relativeTimeNanosSupplier, - ThreadPool threadPool + ThreadPool threadPool, + List indexMetadataUploadListeners ) { assert isRemoteStoreClusterStateEnabled(settings) : "Remote cluster state is not enabled"; this.nodeId = nodeId; @@ -206,6 +209,7 @@ public RemoteClusterStateService( clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); clusterSettings.addSettingsUpdateConsumer(METADATA_MANIFEST_UPLOAD_TIMEOUT_SETTING, this::setMetadataManifestUploadTimeout); this.remoteStateStats = new RemotePersistenceStats(); + this.indexMetadataUploadListeners = indexMetadataUploadListeners; } private BlobStoreTransferService getBlobStoreTransferService() { @@ -233,10 +237,12 @@ public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState, Stri // Write globalMetadata String globalMetadataFile = writeGlobalMetadata(clusterState); + List toUpload = new ArrayList<>(clusterState.metadata().indices().values()); // any validations before/after upload ? final List allUploadedIndexMetadata = writeIndexMetadataParallel( clusterState, - new ArrayList<>(clusterState.metadata().indices().values()) + toUpload, + ClusterState.UNKNOWN_UUID.equals(previousClusterUUID) ? toUpload : Collections.emptyList() ); final ClusterMetadataManifest manifest = uploadManifest( clusterState, @@ -313,7 +319,7 @@ public ClusterMetadataManifest writeIncrementalMetadata( .collect(Collectors.toMap(UploadedIndexMetadata::getIndexName, Function.identity())); List toUpload = new ArrayList<>(); - + List newIndexMetadataList = new ArrayList<>(); for (final IndexMetadata indexMetadata : clusterState.metadata().indices().values()) { final Long previousVersion = previousStateIndexMetadataVersionByName.get(indexMetadata.getIndex().getName()); if (previousVersion == null || indexMetadata.getVersion() != previousVersion) { @@ -329,9 +335,13 @@ public ClusterMetadataManifest writeIncrementalMetadata( numIndicesUnchanged++; } previousStateIndexMetadataVersionByName.remove(indexMetadata.getIndex().getName()); + // Adding the indexMetadata to newIndexMetadataList if there is no previous version present for the index. + if (previousVersion == null) { + newIndexMetadataList.add(indexMetadata); + } } - List uploadedIndexMetadataList = writeIndexMetadataParallel(clusterState, toUpload); + List uploadedIndexMetadataList = writeIndexMetadataParallel(clusterState, toUpload, newIndexMetadataList); uploadedIndexMetadataList.forEach( uploadedIndexMetadata -> allUploadedIndexMetadata.put(uploadedIndexMetadata.getIndexName(), uploadedIndexMetadata) ); @@ -436,13 +446,18 @@ private String writeGlobalMetadata(ClusterState clusterState) throws IOException * Uploads provided IndexMetadata's to remote store in parallel. The call is blocking so the method waits for upload to finish and then return. * * @param clusterState current ClusterState - * @param toUpload list of IndexMetadata to upload + * @param toUpload list of IndexMetadata to upload * @return {@code List} list of IndexMetadata uploaded to remote */ - private List writeIndexMetadataParallel(ClusterState clusterState, List toUpload) - throws IOException { - List exceptionList = Collections.synchronizedList(new ArrayList<>(toUpload.size())); - final CountDownLatch latch = new CountDownLatch(toUpload.size()); + private List writeIndexMetadataParallel( + ClusterState clusterState, + List toUpload, + List newIndexMetadataList + ) throws IOException { + assert Objects.nonNull(indexMetadataUploadListeners) : "indexMetadataUploadListeners can not be null"; + int latchCount = toUpload.size() + indexMetadataUploadListeners.size(); + List exceptionList = Collections.synchronizedList(new ArrayList<>(latchCount)); + final CountDownLatch latch = new CountDownLatch(latchCount); List result = new ArrayList<>(toUpload.size()); LatchedActionListener latchedActionListener = new LatchedActionListener<>( @@ -467,6 +482,8 @@ private List writeIndexMetadataParallel(ClusterState clus writeIndexMetadataAsync(clusterState, indexMetadata, latchedActionListener); } + invokeIndexMetadataUploadListeners(newIndexMetadataList, latch, exceptionList); + try { if (latch.await(getIndexMetadataUploadTimeout().millis(), TimeUnit.MILLISECONDS) == false) { RemoteStateTransferException ex = new RemoteStateTransferException( @@ -506,11 +523,63 @@ private List writeIndexMetadataParallel(ClusterState clus return result; } + /** + * Invokes the index metadata upload listener but does not wait for the execution to complete. + */ + private void invokeIndexMetadataUploadListeners( + List newIndexMetadataList, + CountDownLatch latch, + List exceptionList + ) { + for (IndexMetadataUploadListener listener : indexMetadataUploadListeners) { + String listenerName = listener.getClass().getSimpleName(); + listener.onNewIndexUpload( + newIndexMetadataList, + getIndexMetadataUploadActionListener(newIndexMetadataList, latch, exceptionList, listenerName) + ); + } + + } + + private ActionListener getIndexMetadataUploadActionListener( + List newIndexMetadataList, + CountDownLatch latch, + List exceptionList, + String listenerName + ) { + long startTime = System.nanoTime(); + return new LatchedActionListener<>( + ActionListener.wrap( + ignored -> logger.trace( + new ParameterizedMessage( + "{} : Invoked listener={} successfully tookTimeNs={}", + listenerName, + newIndexMetadataList, + (System.nanoTime() - startTime) + ) + ), + ex -> { + logger.error( + new ParameterizedMessage( + "{} : Exception during invocation of listener={} tookTimeNs={}", + listenerName, + newIndexMetadataList, + (System.nanoTime() - startTime) + ), + ex + ); + exceptionList.add(ex); + } + ), + latch + ); + } + /** * Allows async Upload of IndexMetadata to remote * - * @param clusterState current ClusterState - * @param indexMetadata {@link IndexMetadata} to upload + * @param clusterState current ClusterState + * @param indexMetadata {@link IndexMetadata} to upload * @param latchedActionListener listener to respond back on after upload finishes */ private void writeIndexMetadataAsync( @@ -659,16 +728,6 @@ private void writeMetadataManifest(String clusterName, String clusterUUID, Clust ); } - private String fetchPreviousClusterUUID(String clusterName, String clusterUUID) { - final Optional latestManifest = getLatestClusterMetadataManifest(clusterName, clusterUUID); - if (!latestManifest.isPresent()) { - final String previousClusterUUID = getLastKnownUUIDFromRemote(clusterName); - assert !clusterUUID.equals(previousClusterUUID) : "Last cluster UUID is same current cluster UUID"; - return previousClusterUUID; - } - return latestManifest.get().getPreviousClusterUUID(); - } - private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX return blobStoreRepository.blobStore() @@ -737,7 +796,7 @@ static String getManifestFileName(long term, long version, boolean committed) { (committed ? "C" : "P"), // C for committed and P for published RemoteStoreUtils.invertLong(System.currentTimeMillis()), String.valueOf(MANIFEST_CURRENT_CODEC_VERSION) // Keep the codec version at last place only, during read we reads last place to - // determine codec version. + // determine codec version. ); } @@ -750,7 +809,7 @@ static String indexMetadataFileName(IndexMetadata indexMetadata) { RemoteStoreUtils.invertLong(indexMetadata.getVersion()), RemoteStoreUtils.invertLong(System.currentTimeMillis()), String.valueOf(INDEX_METADATA_CURRENT_CODEC_VERSION) // Keep the codec version at last place only, during read we reads last - // place to determine codec version. + // place to determine codec version. ); } @@ -772,8 +831,8 @@ private BlobPath getManifestFolderPath(String clusterName, String clusterUUID) { /** * Fetch latest index metadata from remote cluster state * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster * @param clusterMetadataManifest manifest file of cluster * @return {@code Map} latest IndexUUID to IndexMetadata map */ @@ -795,8 +854,8 @@ private Map getIndexMetadataMap( /** * Fetch index metadata from remote cluster state * - * @param clusterUUID uuid of cluster state to refer to in remote - * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster * @param uploadedIndexMetadata {@link UploadedIndexMetadata} contains details about remote location of index metadata * @return {@link IndexMetadata} */ @@ -825,7 +884,6 @@ private IndexMetadata getIndexMetadata(String clusterName, String clusterUUID, U * @return {@link IndexMetadata} */ public ClusterState getLatestClusterState(String clusterName, String clusterUUID) { - start(); Optional clusterMetadataManifest = getLatestClusterMetadataManifest(clusterName, clusterUUID); if (clusterMetadataManifest.isEmpty()) { throw new IllegalStateException( @@ -989,6 +1047,7 @@ private List createClusterChain(final Map trimClusterUUIDs( @@ -1050,7 +1109,7 @@ private boolean isValidClusterUUID(ClusterMetadataManifest manifest) { * * @param clusterUUID uuid of cluster state to refer to in remote * @param clusterName name of the cluster - * @param limit max no of files to fetch + * @param limit max no of files to fetch * @return all manifest file names */ private List getManifestFileNames(String clusterName, String clusterUUID, int limit) throws IllegalStateException { @@ -1123,7 +1182,7 @@ private int getManifestCodecVersion(String fileName) { if (splitName.length == SPLITED_MANIFEST_FILE_LENGTH) { return Integer.parseInt(splitName[splitName.length - 1]); // Last value would be codec version. } else if (splitName.length < SPLITED_MANIFEST_FILE_LENGTH) { // Where codec is not part of file name, i.e. default codec version 0 - // is used. + // is used. return ClusterMetadataManifest.CODEC_V0; } else { throw new IllegalArgumentException("Manifest file name is corrupted"); @@ -1141,7 +1200,7 @@ public void writeMetadataFailed() { /** * Exception for Remote state transfer. */ - static class RemoteStateTransferException extends RuntimeException { + public static class RemoteStateTransferException extends RuntimeException { public RemoteStateTransferException(String errorDesc) { super(errorDesc); @@ -1155,7 +1214,7 @@ public RemoteStateTransferException(String errorDesc, Throwable cause) { /** * Purges all remote cluster state against provided cluster UUIDs * - * @param clusterName name of the cluster + * @param clusterName name of the cluster * @param clusterUUIDs clusteUUIDs for which the remote state needs to be purged */ void deleteStaleUUIDsClusterMetadata(String clusterName, List clusterUUIDs) { @@ -1188,8 +1247,8 @@ public void onFailure(Exception e) { /** * Deletes older than last {@code versionsToRetain} manifests. Also cleans up unreferenced IndexMetadata associated with older manifests * - * @param clusterName name of the cluster - * @param clusterUUID uuid of cluster state to refer to in remote + * @param clusterName name of the cluster + * @param clusterUUID uuid of cluster state to refer to in remote * @param manifestsToRetain no of latest manifest files to keep in remote */ // package private for testing @@ -1308,7 +1367,8 @@ private void deleteStalePaths(String clusterName, String clusterUUID, List> TRANSLOG_PATH = Map.of(TRANSLOG, List.of(DATA, METADATA)); + public static final Map> SEGMENT_PATH = Map.of(SEGMENTS, List.of(DataType.values())); + public static final Map> COMBINED_PATH; + + static { + Map> combinedPath = new HashMap<>(); + combinedPath.putAll(TRANSLOG_PATH); + combinedPath.putAll(SEGMENT_PATH); + COMBINED_PATH = Collections.unmodifiableMap(combinedPath); + } + private static final String DEFAULT_VERSION = "1"; + public static final String DIR = "remote-index-path"; + public static final String FILE_NAME_FORMAT = "remote_path_%s"; + static final String KEY_VERSION = "version"; + static final String KEY_INDEX_UUID = "index_uuid"; + static final String KEY_SHARD_COUNT = "shard_count"; + static final String KEY_PATH_CREATION_MAP = "path_creation_map"; + static final String KEY_PATHS = "paths"; + private final String indexUUID; + private final int shardCount; + private final Iterable basePath; + private final PathType pathType; + private final PathHashAlgorithm pathHashAlgorithm; + + /** + * This keeps the map of paths that would be present in the content of the index path file. For eg - It is possible + * that segment and translog repository can be different. For this use case, we have either segment or translog as the + * key, and list of data, metadata, and lock_files (only for segment) as the value. + */ + private final Map> pathCreationMap; + + public RemoteIndexPath( + String indexUUID, + int shardCount, + Iterable basePath, + PathType pathType, + PathHashAlgorithm pathHashAlgorithm, + Map> pathCreationMap + ) { + if (Objects.isNull(pathCreationMap) + || Objects.isNull(pathType) + || isCompatible(pathType, pathHashAlgorithm) == false + || shardCount < 1 + || Objects.isNull(basePath) + || pathCreationMap.isEmpty() + || pathCreationMap.keySet().stream().anyMatch(k -> pathCreationMap.get(k).isEmpty())) { + ParameterizedMessage parameterizedMessage = new ParameterizedMessage( + "Invalid input in RemoteIndexPath constructor indexUUID={} shardCount={} basePath={} pathType={}" + + " pathHashAlgorithm={} pathCreationMap={}", + indexUUID, + shardCount, + basePath, + pathType, + pathHashAlgorithm, + pathCreationMap + ); + throw new IllegalArgumentException(parameterizedMessage.getFormattedMessage()); + } + boolean validMap = pathCreationMap.keySet() + .stream() + .allMatch(k -> pathCreationMap.get(k).stream().allMatch(k::isSupportedDataType)); + if (validMap == false) { + throw new IllegalArgumentException( + new ParameterizedMessage("pathCreationMap={} is having illegal combination of category and type", pathCreationMap) + .getFormattedMessage() + ); + } + this.indexUUID = indexUUID; + this.shardCount = shardCount; + this.basePath = basePath; + this.pathType = pathType; + this.pathHashAlgorithm = pathHashAlgorithm; + this.pathCreationMap = pathCreationMap; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(KEY_VERSION, DEFAULT_VERSION); + builder.field(KEY_INDEX_UUID, indexUUID); + builder.field(KEY_SHARD_COUNT, shardCount); + builder.field(PathType.NAME, pathType.name()); + if (Objects.nonNull(pathHashAlgorithm)) { + builder.field(PathHashAlgorithm.NAME, pathHashAlgorithm.name()); + } + + Map> pathMap = new HashMap<>(); + for (Map.Entry> entry : pathCreationMap.entrySet()) { + pathMap.put(entry.getKey().getName(), entry.getValue().stream().map(DataType::getName).collect(Collectors.toList())); + } + builder.field(KEY_PATH_CREATION_MAP); + builder.map(pathMap); + builder.startArray(KEY_PATHS); + for (Map.Entry> entry : pathCreationMap.entrySet()) { + DataCategory dataCategory = entry.getKey(); + for (DataType type : entry.getValue()) { + for (int shardNo = 0; shardNo < shardCount; shardNo++) { + PathInput pathInput = PathInput.builder() + .basePath(new BlobPath().add(basePath)) + .indexUUID(indexUUID) + .shardId(Integer.toString(shardNo)) + .dataCategory(dataCategory) + .dataType(type) + .build(); + builder.value(pathType.path(pathInput, pathHashAlgorithm).buildAsString()); + } + } + } + builder.endArray(); + return builder; + } + + public static RemoteIndexPath fromXContent(XContentParser ignored) { + throw new UnsupportedOperationException("RemoteIndexPath.fromXContent() is not supported"); + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java new file mode 100644 index 0000000000000..1ac7e41014d23 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteIndexPathUploader.java @@ -0,0 +1,281 @@ +/* + * 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.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +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.core.index.Index; +import org.opensearch.gateway.remote.IndexMetadataUploadListener; +import org.opensearch.gateway.remote.RemoteClusterStateService.RemoteStateTransferException; +import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.blobstore.ConfigBlobStoreFormat; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.opensearch.gateway.remote.RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING; +import static org.opensearch.index.remote.RemoteIndexPath.COMBINED_PATH; +import static org.opensearch.index.remote.RemoteIndexPath.SEGMENT_PATH; +import static org.opensearch.index.remote.RemoteIndexPath.TRANSLOG_PATH; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled; + +/** + * Uploads the remote store path for all possible combinations of {@link org.opensearch.index.remote.RemoteStoreEnums.DataCategory} + * and {@link org.opensearch.index.remote.RemoteStoreEnums.DataType} for each shard of an index. + * + * @opensearch.internal + */ +@ExperimentalApi +public class RemoteIndexPathUploader extends IndexMetadataUploadListener { + + public static final ConfigBlobStoreFormat REMOTE_INDEX_PATH_FORMAT = new ConfigBlobStoreFormat<>( + RemoteIndexPath.FILE_NAME_FORMAT + ); + + private static final String TIMEOUT_EXCEPTION_MSG = "Timed out waiting while uploading remote index path file for indexes=%s"; + private static final String UPLOAD_EXCEPTION_MSG = "Exception occurred while uploading remote index paths for indexes=%s"; + static final String TRANSLOG_REPO_NAME_KEY = Node.NODE_ATTRIBUTES.getKey() + + RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; + static final String SEGMENT_REPO_NAME_KEY = Node.NODE_ATTRIBUTES.getKey() + + RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; + + private static final Logger logger = LogManager.getLogger(RemoteIndexPathUploader.class); + + private final Settings settings; + private final boolean isRemoteDataAttributePresent; + private final boolean isTranslogSegmentRepoSame; + private final Supplier repositoriesService; + private volatile TimeValue indexMetadataUploadTimeout; + + private BlobStoreRepository translogRepository; + private BlobStoreRepository segmentRepository; + + public RemoteIndexPathUploader( + ThreadPool threadPool, + Settings settings, + Supplier repositoriesService, + ClusterSettings clusterSettings + ) { + super(threadPool, ThreadPool.Names.GENERIC); + this.settings = Objects.requireNonNull(settings); + this.repositoriesService = Objects.requireNonNull(repositoriesService); + isRemoteDataAttributePresent = isRemoteDataAttributePresent(settings); + // If the remote data attributes are not present, then there is no effect of translog and segment being same or different or null. + isTranslogSegmentRepoSame = isTranslogSegmentRepoSame(); + Objects.requireNonNull(clusterSettings); + indexMetadataUploadTimeout = clusterSettings.get(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING); + clusterSettings.addSettingsUpdateConsumer(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, this::setIndexMetadataUploadTimeout); + } + + @Override + protected void doOnNewIndexUpload(List indexMetadataList, ActionListener actionListener) { + if (isRemoteDataAttributePresent == false) { + logger.trace("Skipping beforeNewIndexUpload as there are no remote indexes"); + actionListener.onResponse(null); + return; + } + + long startTime = System.nanoTime(); + boolean success = false; + List eligibleList = indexMetadataList.stream().filter(this::requiresPathUpload).collect(Collectors.toList()); + String indexNames = eligibleList.stream().map(IndexMetadata::getIndex).map(Index::toString).collect(Collectors.joining(",")); + int latchCount = eligibleList.size() * (isTranslogSegmentRepoSame ? 1 : 2); + CountDownLatch latch = new CountDownLatch(latchCount); + List exceptionList = Collections.synchronizedList(new ArrayList<>(latchCount)); + try { + for (IndexMetadata indexMetadata : eligibleList) { + writeIndexPathAsync(indexMetadata, latch, exceptionList); + } + + logger.trace(new ParameterizedMessage("Remote index path upload started for {}", indexNames)); + + try { + if (latch.await(indexMetadataUploadTimeout.millis(), TimeUnit.MILLISECONDS) == false) { + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, TIMEOUT_EXCEPTION_MSG, indexNames) + ); + exceptionList.forEach(ex::addSuppressed); + actionListener.onFailure(ex); + return; + } + } catch (InterruptedException exception) { + exceptionList.forEach(exception::addSuppressed); + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, TIMEOUT_EXCEPTION_MSG, indexNames), + exception + ); + actionListener.onFailure(ex); + return; + } + if (exceptionList.size() > 0) { + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, UPLOAD_EXCEPTION_MSG, indexNames) + ); + exceptionList.forEach(ex::addSuppressed); + actionListener.onFailure(ex); + return; + } + success = true; + actionListener.onResponse(null); + } catch (Exception exception) { + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, UPLOAD_EXCEPTION_MSG, indexNames), + exception + ); + exceptionList.forEach(ex::addSuppressed); + actionListener.onFailure(ex); + } finally { + long tookTimeNs = System.nanoTime() - startTime; + logger.trace(new ParameterizedMessage("executed beforeNewIndexUpload status={} tookTimeNs={}", success, tookTimeNs)); + } + + } + + private void writeIndexPathAsync(IndexMetadata idxMD, CountDownLatch latch, List exceptionList) { + if (isTranslogSegmentRepoSame) { + // If the repositories are same, then we need to upload a single file containing paths for both translog and segments. + writePathToRemoteStore(idxMD, translogRepository, latch, exceptionList, COMBINED_PATH); + } else { + // If the repositories are different, then we need to upload one file per segment and translog containing their individual + // paths. + writePathToRemoteStore(idxMD, translogRepository, latch, exceptionList, TRANSLOG_PATH); + writePathToRemoteStore(idxMD, segmentRepository, latch, exceptionList, SEGMENT_PATH); + } + } + + private void writePathToRemoteStore( + IndexMetadata idxMD, + BlobStoreRepository repository, + CountDownLatch latch, + List exceptionList, + Map> pathCreationMap + ) { + Map remoteCustomData = idxMD.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.valueOf(remoteCustomData.get(RemoteStoreEnums.PathType.NAME)); + RemoteStoreEnums.PathHashAlgorithm hashAlgorithm = RemoteStoreEnums.PathHashAlgorithm.valueOf( + remoteCustomData.get(RemoteStoreEnums.PathHashAlgorithm.NAME) + ); + String indexUUID = idxMD.getIndexUUID(); + int shardCount = idxMD.getNumberOfShards(); + BlobPath basePath = repository.basePath(); + BlobContainer blobContainer = repository.blobStore().blobContainer(basePath.add(RemoteIndexPath.DIR)); + ActionListener actionListener = getUploadPathLatchedActionListener(idxMD, latch, exceptionList, pathCreationMap); + try { + REMOTE_INDEX_PATH_FORMAT.writeAsyncWithUrgentPriority( + new RemoteIndexPath(indexUUID, shardCount, basePath, pathType, hashAlgorithm, pathCreationMap), + blobContainer, + indexUUID, + actionListener + ); + } catch (IOException ioException) { + RemoteStateTransferException ex = new RemoteStateTransferException( + String.format(Locale.ROOT, UPLOAD_EXCEPTION_MSG, List.of(idxMD.getIndex().getName())) + ); + actionListener.onFailure(ioException); + } + } + + private Repository validateAndGetRepository(String repoSetting) { + final String repo = settings.get(repoSetting); + assert repo != null : "Remote " + repoSetting + " repository is not configured"; + final Repository repository = repositoriesService.get().repository(repo); + assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; + return repository; + } + + public void start() { + assert isRemoteStoreClusterStateEnabled(settings) == true : "Remote cluster state is not enabled"; + if (isRemoteDataAttributePresent == false) { + // If remote store data attributes are not present than we skip this. + return; + } + translogRepository = (BlobStoreRepository) validateAndGetRepository(TRANSLOG_REPO_NAME_KEY); + segmentRepository = (BlobStoreRepository) validateAndGetRepository(SEGMENT_REPO_NAME_KEY); + } + + private boolean isTranslogSegmentRepoSame() { + String translogRepoName = settings.get(TRANSLOG_REPO_NAME_KEY); + String segmentRepoName = settings.get(SEGMENT_REPO_NAME_KEY); + return Objects.equals(translogRepoName, segmentRepoName); + } + + private LatchedActionListener getUploadPathLatchedActionListener( + IndexMetadata indexMetadata, + CountDownLatch latch, + List exceptionList, + Map> pathCreationMap + ) { + return new LatchedActionListener<>( + ActionListener.wrap( + resp -> logger.trace( + new ParameterizedMessage("Index path uploaded for {} indexMetadata={}", pathCreationMap, indexMetadata) + ), + ex -> { + logger.error( + new ParameterizedMessage( + "Exception during Index path upload for {} indexMetadata={}", + pathCreationMap, + indexMetadata + ), + ex + ); + exceptionList.add(ex); + } + ), + latch + ); + } + + /** + * This method checks if the index metadata has attributes that calls for uploading the index path for remote store + * uploads. It checks if the remote store path type is {@code HASHED_PREFIX} and returns true if so. + */ + private boolean requiresPathUpload(IndexMetadata indexMetadata) { + // A cluster will have remote custom metadata only if the cluster is remote store enabled from data side. + Map remoteCustomData = indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY); + if (Objects.isNull(remoteCustomData) || remoteCustomData.isEmpty()) { + return false; + } + String pathTypeStr = remoteCustomData.get(RemoteStoreEnums.PathType.NAME); + if (Objects.isNull(pathTypeStr)) { + return false; + } + // We need to upload the path only if the path type for an index is hashed_prefix + return RemoteStoreEnums.PathType.HASHED_PREFIX == RemoteStoreEnums.PathType.parseString(pathTypeStr); + } + + private void setIndexMetadataUploadTimeout(TimeValue newIndexMetadataUploadTimeout) { + this.indexMetadataUploadTimeout = newIndexMetadataUploadTimeout; + } +} diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java index 775f8fe19e4ef..c58f6c3faac84 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategy.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.index.remote.RemoteStoreEnums.DataCategory; @@ -25,6 +26,7 @@ * @opensearch.internal */ @PublicApi(since = "2.14.0") +@ExperimentalApi public class RemoteStorePathStrategy { private final PathType type; @@ -74,6 +76,7 @@ public BlobPath generatePath(PathInput pathInput) { * @opensearch.internal */ @PublicApi(since = "2.14.0") + @ExperimentalApi public static class PathInput { private final BlobPath basePath; private final String indexUUID; @@ -122,6 +125,7 @@ public static Builder builder() { * @opensearch.internal */ @PublicApi(since = "2.14.0") + @ExperimentalApi public static class Builder { private BlobPath basePath; private String indexUUID; diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java index a33f7522daaae..5cd69dfa679a5 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePathStrategyResolver.java @@ -9,6 +9,7 @@ package org.opensearch.index.remote; import org.opensearch.Version; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.indices.RemoteStoreSettings; @@ -20,6 +21,7 @@ * * @opensearch.internal */ +@ExperimentalApi public class RemoteStorePathStrategyResolver { private final RemoteStoreSettings remoteStoreSettings; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 5b5208053cf1d..a8bc25b8c3935 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -145,6 +145,7 @@ import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.engine.EngineFactory; import org.opensearch.index.recovery.RemoteStoreRestoreService; +import org.opensearch.index.remote.RemoteIndexPathUploader; import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.remote.filecache.FileCache; @@ -725,17 +726,26 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService; + final RemoteIndexPathUploader remoteIndexPathUploader; if (isRemoteStoreClusterStateEnabled(settings)) { + remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + repositoriesServiceReference::get, + clusterService.getClusterSettings() + ); remoteClusterStateService = new RemoteClusterStateService( nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, clusterService.getClusterSettings(), threadPool::preciseRelativeTimeInNanos, - threadPool + threadPool, + List.of(remoteIndexPathUploader) ); } else { remoteClusterStateService = null; + remoteIndexPathUploader = null; } // collect engine factory providers from plugins @@ -1308,6 +1318,7 @@ protected Node( b.bind(SearchRequestSlowLog.class).toInstance(searchRequestSlowLog); b.bind(MetricsRegistry.class).toInstance(metricsRegistry); b.bind(RemoteClusterStateService.class).toProvider(() -> remoteClusterStateService); + b.bind(RemoteIndexPathUploader.class).toProvider(() -> remoteIndexPathUploader); b.bind(PersistedStateRegistry.class).toInstance(persistedStateRegistry); b.bind(SegmentReplicationStatsTracker.class).toInstance(segmentReplicationStatsTracker); b.bind(SearchRequestOperationsCompositeListenerFactory.class).toInstance(searchRequestOperationsCompositeListenerFactory); @@ -1457,6 +1468,10 @@ public Node start() throws NodeValidationException { if (remoteClusterStateService != null) { remoteClusterStateService.start(); } + final RemoteIndexPathUploader remoteIndexPathUploader = injector.getInstance(RemoteIndexPathUploader.class); + if (remoteIndexPathUploader != null) { + remoteIndexPathUploader.start(); + } // Load (and maybe upgrade) the metadata stored on disk final GatewayMetaState gatewayMetaState = injector.getInstance(GatewayMetaState.class); gatewayMetaState.start( diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BaseBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/BaseBlobStoreFormat.java new file mode 100644 index 0000000000000..262a32fa1e74d --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BaseBlobStoreFormat.java @@ -0,0 +1,130 @@ +/* + * 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.repositories.blobstore; + +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.OutputStreamIndexOutput; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.lucene.store.IndexOutputOutputStream; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Locale; +import java.util.Objects; + +/** + * Provides common methods, variables that can be used by the implementors. + * + * @opensearch.internal + */ +public class BaseBlobStoreFormat { + + private static final int BUFFER_SIZE = 4096; + + private final String blobNameFormat; + + private final boolean skipHeaderFooter; + + /** + * @param blobNameFormat format of the blobname in {@link String#format} format + */ + public BaseBlobStoreFormat(String blobNameFormat, boolean skipHeaderFooter) { + this.blobNameFormat = blobNameFormat; + this.skipHeaderFooter = skipHeaderFooter; + } + + protected String blobName(String name) { + return String.format(Locale.ROOT, blobNameFormat, name); + } + + /** + * Writes blob with resolving the blob name using {@link #blobName} method. + *

+ * The blob will optionally by compressed. + * + * @param obj object to be serialized + * @param blobContainer blob container + * @param name blob name + * @param compressor whether to use compression + * @param params ToXContent params + * @param codec codec used + * @param version version used + */ + protected void write( + final T obj, + final BlobContainer blobContainer, + final String name, + final Compressor compressor, + final ToXContent.Params params, + XContentType xContentType, + String codec, + Integer version + ) throws IOException { + final String blobName = blobName(name); + final BytesReference bytes = serialize(obj, blobName, compressor, params, xContentType, codec, version); + blobContainer.writeBlob(blobName, bytes.streamInput(), bytes.length(), false); + } + + public BytesReference serialize( + final T obj, + final String blobName, + final Compressor compressor, + final ToXContent.Params params, + XContentType xContentType, + String codec, + Integer version + ) throws IOException { + assert skipHeaderFooter || (Objects.nonNull(codec) && Objects.nonNull(version)); + try (BytesStreamOutput outputStream = new BytesStreamOutput()) { + try ( + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( + "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", + blobName, + outputStream, + BUFFER_SIZE + ) + ) { + if (skipHeaderFooter == false) { + CodecUtil.writeHeader(indexOutput, codec, version); + } + try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { + @Override + public void close() { + // this is important since some of the XContentBuilders write bytes on close. + // in order to write the footer we need to prevent closing the actual index input. + } + }; + XContentBuilder builder = MediaTypeRegistry.contentBuilder( + xContentType, + compressor.threadLocalOutputStream(indexOutputOutputStream) + ) + ) { + builder.startObject(); + obj.toXContent(builder, params); + builder.endObject(); + } + if (skipHeaderFooter == false) { + CodecUtil.writeFooter(indexOutput); + } + } + return outputStream.bytes(); + } + } + + protected String getBlobNameFormat() { + return blobNameFormat; + } +} diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java index 3e6052a5ef820..e567e1b626c5a 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -38,7 +38,6 @@ import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.store.ByteBuffersIndexInput; import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.OutputStreamIndexOutput; import org.apache.lucene.util.BytesRef; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.CheckedFunction; @@ -48,26 +47,21 @@ import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.io.Streams; -import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.store.ByteArrayIndexInput; -import org.opensearch.common.lucene.store.IndexOutputOutputStream; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.compress.Compressor; -import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.CorruptStateException; import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.snapshots.SnapshotInfo; import java.io.IOException; -import java.io.OutputStream; import java.util.Arrays; import java.util.HashMap; import java.util.Locale; @@ -80,7 +74,7 @@ * * @opensearch.internal */ -public final class ChecksumBlobStoreFormat { +public final class ChecksumBlobStoreFormat extends BaseBlobStoreFormat { // Serialization parameters to specify correct context for metadata serialization public static final ToXContent.Params SNAPSHOT_ONLY_FORMAT_PARAMS; @@ -98,12 +92,8 @@ public final class ChecksumBlobStoreFormat { // The format version public static final int VERSION = 1; - private static final int BUFFER_SIZE = 4096; - private final String codec; - private final String blobNameFormat; - private final CheckedFunction reader; /** @@ -112,8 +102,8 @@ public final class ChecksumBlobStoreFormat { * @param reader prototype object that can deserialize T from XContent */ public ChecksumBlobStoreFormat(String codec, String blobNameFormat, CheckedFunction reader) { + super(blobNameFormat, false); this.reader = reader; - this.blobNameFormat = blobNameFormat; this.codec = codec; } @@ -130,7 +120,7 @@ public T read(BlobContainer blobContainer, String name, NamedXContentRegistry na } public String blobName(String name) { - return String.format(Locale.ROOT, blobNameFormat, name); + return String.format(Locale.ROOT, getBlobNameFormat(), name); } public T deserialize(String blobName, NamedXContentRegistry namedXContentRegistry, BytesReference bytes) throws IOException { @@ -170,30 +160,7 @@ public T deserialize(String blobName, NamedXContentRegistry namedXContentRegistr * @param compressor whether to use compression */ public void write(final T obj, final BlobContainer blobContainer, final String name, final Compressor compressor) throws IOException { - write(obj, blobContainer, name, compressor, SNAPSHOT_ONLY_FORMAT_PARAMS); - } - - /** - * Writes blob with resolving the blob name using {@link #blobName} method. - *

- * The blob will optionally by compressed. - * - * @param obj object to be serialized - * @param blobContainer blob container - * @param name blob name - * @param compressor whether to use compression - * @param params ToXContent params - */ - public void write( - final T obj, - final BlobContainer blobContainer, - final String name, - final Compressor compressor, - final ToXContent.Params params - ) throws IOException { - final String blobName = blobName(name); - final BytesReference bytes = serialize(obj, blobName, compressor, params); - blobContainer.writeBlob(blobName, bytes.streamInput(), bytes.length(), false); + write(obj, blobContainer, name, compressor, SNAPSHOT_ONLY_FORMAT_PARAMS, XContentType.SMILE, codec, VERSION); } /** @@ -251,7 +218,7 @@ private void writeAsyncWithPriority( final ToXContent.Params params ) throws IOException { if (blobContainer instanceof AsyncMultiStreamBlobContainer == false) { - write(obj, blobContainer, name, compressor, params); + write(obj, blobContainer, name, compressor, params, XContentType.SMILE, codec, VERSION); listener.onResponse(null); return; } @@ -290,35 +257,6 @@ private void writeAsyncWithPriority( public BytesReference serialize(final T obj, final String blobName, final Compressor compressor, final ToXContent.Params params) throws IOException { - try (BytesStreamOutput outputStream = new BytesStreamOutput()) { - try ( - OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( - "ChecksumBlobStoreFormat.writeBlob(blob=\"" + blobName + "\")", - blobName, - outputStream, - BUFFER_SIZE - ) - ) { - CodecUtil.writeHeader(indexOutput, codec, VERSION); - try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { - @Override - public void close() throws IOException { - // this is important since some of the XContentBuilders write bytes on close. - // in order to write the footer we need to prevent closing the actual index input. - } - }; - XContentBuilder builder = MediaTypeRegistry.contentBuilder( - XContentType.SMILE, - compressor.threadLocalOutputStream(indexOutputOutputStream) - ) - ) { - builder.startObject(); - obj.toXContent(builder, params); - builder.endObject(); - } - CodecUtil.writeFooter(indexOutput); - } - return outputStream.bytes(); - } + return serialize(obj, blobName, compressor, params, XContentType.SMILE, codec, VERSION); } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java new file mode 100644 index 0000000000000..18c718ca2110e --- /dev/null +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java @@ -0,0 +1,73 @@ +/* + * 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.repositories.blobstore; + +import org.apache.lucene.store.IndexInput; +import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.ToXContent; + +import java.io.IOException; + +/** + * Format for writing short configurations to remote. Read interface does not exist as it not yet required. This format + * should be used for writing data from in-memory to remote store where there is no need for checksum and the client + * library for the remote store has inbuilt checksum capabilities while upload and download both. This format would + * serialise the data in Json format and store it on remote store as is. This does not support compression yet (this + * can be changed as required). In comparison to {@link ChecksumBlobStoreFormat}, this format does not add any additional + * metadata (like header and footer) to the content. Hence, this format does not depend on {@code CodecUtil} from + * Lucene library. + * + * @opensearch.internal + */ +public class ConfigBlobStoreFormat extends BaseBlobStoreFormat { + + /** + * @param blobNameFormat format of the blobname in {@link String#format} format + */ + public ConfigBlobStoreFormat(String blobNameFormat) { + super(blobNameFormat, true); + } + + public void writeAsyncWithUrgentPriority(T obj, BlobContainer blobContainer, String name, ActionListener listener) + throws IOException { + if (blobContainer instanceof AsyncMultiStreamBlobContainer == false) { + write(obj, blobContainer, name, new NoneCompressor(), ToXContent.EMPTY_PARAMS, XContentType.JSON, null, null); + listener.onResponse(null); + return; + } + String blobName = blobName(name); + BytesReference bytes = serialize(obj, blobName, new NoneCompressor(), ToXContent.EMPTY_PARAMS, XContentType.JSON, null, null); + String resourceDescription = "BlobStoreFormat.writeAsyncWithPriority(blob=\"" + blobName + "\")"; + try (IndexInput input = new ByteArrayIndexInput(resourceDescription, BytesReference.toBytes(bytes))) { + try ( + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + blobName, + blobName, + bytes.length(), + true, + WritePriority.URGENT, + (size, position) -> new OffsetRangeIndexInputStream(input, size, position), + null, + false + ) + ) { + ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); + } + } + } +} diff --git a/server/src/test/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainerTests.java b/server/src/test/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainerTests.java index 074f659850c7b..44f6a0e11251c 100644 --- a/server/src/test/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainerTests.java +++ b/server/src/test/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainerTests.java @@ -67,7 +67,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new OffsetRangeFileInputStream(testFile, size, position); } }, - 0, + 0L, false ) ) { @@ -89,7 +89,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new OffsetRangeFileInputStream(testFile, size, position); } }, - 0, + 0L, false ) ) { @@ -155,7 +155,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new OffsetRangeFileInputStream(testFile, size, position); } }, - 0, + 0L, false ) ) { @@ -223,7 +223,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new OffsetRangeFileInputStream(testFile, size, position); } }, - 0, + 0L, isRemoteDataIntegritySupported ) ) { @@ -286,7 +286,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new RateLimitingOffsetRangeInputStream(offsetRangeIndexInputStream, rateLimiterSupplier, null); } }, - 0, + 0L, true ) ) { @@ -347,7 +347,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { return new RateLimitingOffsetRangeInputStream(offsetRangeIndexInputStream, rateLimiterSupplier, null); } }, - 0, + 0L, true ) ) { diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index 74bae7b5eb7cf..3ba98c44f8d3e 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -71,6 +71,7 @@ import org.opensearch.gateway.remote.RemotePersistenceStats; import org.opensearch.index.recovery.RemoteStoreRestoreService; import org.opensearch.index.recovery.RemoteStoreRestoreService.RemoteRestoreResult; +import org.opensearch.index.remote.RemoteIndexPathUploader; import org.opensearch.node.Node; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.fs.FsRepository; @@ -473,20 +474,23 @@ public void testDataOnlyNodePersistence() throws Exception { ); Supplier remoteClusterStateServiceSupplier = () -> { if (isRemoteStoreClusterStateEnabled(settings)) { + Supplier repositoriesServiceSupplier = () -> new RepositoriesService( + settings, + clusterService, + transportService, + Collections.emptyMap(), + Collections.emptyMap(), + transportService.getThreadPool() + ); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); return new RemoteClusterStateService( nodeEnvironment.nodeId(), - () -> new RepositoriesService( - settings, - clusterService, - transportService, - Collections.emptyMap(), - Collections.emptyMap(), - transportService.getThreadPool() - ), + repositoriesServiceSupplier, settings, - new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + clusterSettings, () -> 0L, - threadPool + threadPool, + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) ); } else { return null; diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 65477051cdb30..9f321cd62847c 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -38,6 +38,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import org.opensearch.index.remote.RemoteIndexPathUploader; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.FilterRepository; @@ -154,7 +155,8 @@ public void setup() { settings, clusterSettings, () -> 0L, - threadPool + threadPool, + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) ); } @@ -173,15 +175,17 @@ public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException public void testFailInitializationWhenRemoteStateDisabled() { final Settings settings = Settings.builder().build(); + ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); assertThrows( AssertionError.class, () -> new RemoteClusterStateService( "test-node-id", repositoriesServiceSupplier, settings, - new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + clusterSettings, () -> 0L, - threadPool + threadPool, + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)) ) ); } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java new file mode 100644 index 0000000000000..8ddbd383756e7 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathTests.java @@ -0,0 +1,140 @@ +/* + * 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.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; +import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import org.mockito.Mockito; + +import static org.opensearch.index.remote.RemoteStoreEnums.DataCategory.TRANSLOG; +import static org.opensearch.index.remote.RemoteStoreEnums.DataType.LOCK_FILES; + +public class RemoteIndexPathTests extends OpenSearchTestCase { + + /** + * This checks that the remote path contains paths only for segment and data/metadata/lock_files combination. + */ + public void testToXContentWithSegmentRepo() throws IOException { + RemoteIndexPath indexPath = new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + RemoteIndexPath.SEGMENT_PATH + ); + XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + String expected = + "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\"9BmBinD5HYs/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/data/\",\"ExCNOD8_5ew/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/data/\",\"z8wtf0yr2l4/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/metadata/\",\"VheHVwFlExE/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/metadata/\",\"IgFKbsDeUpQ/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/segments/lock_files/\",\"pA3gy_GZtns/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/segments/lock_files/\"]}"; + assertEquals(expected, xContentBuilder.toString()); + } + + /** + * This checks that the remote path contains paths only for translog and data/metadata combination. + */ + public void testToXContentForTranslogRepoOnly() throws IOException { + RemoteIndexPath indexPath = new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + RemoteIndexPath.TRANSLOG_PATH + ); + XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + String expected = + "{\"version\":\"1\",\"index_uuid\":\"djjsid73he8yd7usduh\",\"shard_count\":2,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"]},\"paths\":[\"2EaVODaKBck/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/data/\",\"dTS2VqEOUNo/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/data/\",\"PVNKNGonmZw/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/0/translog/metadata/\",\"NXmt0Y6NjA8/djsd878ndjh/hcs87cj8/djjsid73he8yd7usduh/1/translog/metadata/\"]}"; + assertEquals(expected, xContentBuilder.toString()); + } + + /** + * This checks that the remote path contains paths only for translog and data/metadata combination. + */ + public void testToXContentForBothRepos() throws IOException { + Map> pathCreationMap = new TreeMap<>(); + pathCreationMap.putAll(RemoteIndexPath.TRANSLOG_PATH); + pathCreationMap.putAll(RemoteIndexPath.SEGMENT_PATH); + RemoteIndexPath indexPath = new RemoteIndexPath( + "csbdqiu8a7sdnjdks", + 3, + new BlobPath().add("nxf9yv0").add("c3ejoi"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + pathCreationMap + ); + XContentBuilder xContentBuilder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); + xContentBuilder.startObject(); + xContentBuilder = indexPath.toXContent(xContentBuilder, ToXContent.EMPTY_PARAMS); + xContentBuilder.endObject(); + String expected = + "{\"version\":\"1\",\"index_uuid\":\"csbdqiu8a7sdnjdks\",\"shard_count\":3,\"path_type\":\"HASHED_PREFIX\",\"path_hash_algorithm\":\"FNV_1A_BASE64\",\"path_creation_map\":{\"translog\":[\"data\",\"metadata\"],\"segments\":[\"data\",\"metadata\",\"lock_files\"]},\"paths\":[\"Cjo0F6kNjYk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/data/\",\"kpayyhxct1I/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/data/\",\"p2RlgnHeIgc/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/data/\",\"gkPIurBtB1w/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/metadata/\",\"Y4YhlbxAB1c/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/metadata/\",\"HYc8fyVPouI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/metadata/\",\"igzyZCz1ysI/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/segments/lock_files/\",\"uEluEiYmptk/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/segments/lock_files/\",\"TfAD8f06_7A/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/segments/lock_files/\",\"QqKEpasbEGs/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/data/\",\"sNyoimoe1Bw/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/data/\",\"d4YQtONfq50/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/data/\",\"zLr4UXjK8T4/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/0/translog/metadata/\",\"_s8i7ZmlXGE/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/1/translog/metadata/\",\"tvtD3-k5ISg/nxf9yv0/c3ejoi/csbdqiu8a7sdnjdks/2/translog/metadata/\"]}"; + assertEquals(expected, xContentBuilder.toString()); + } + + public void testRemoteIndexPathWithInvalidPathCreationMap() throws IOException { + IllegalArgumentException ex = assertThrows( + IllegalArgumentException.class, + () -> new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + new HashMap<>() + ) + ); + assertEquals( + "Invalid input in RemoteIndexPath constructor indexUUID=djjsid73he8yd7usduh shardCount=2 " + + "basePath=[djsd878ndjh][hcs87cj8] pathType=HASHED_PREFIX pathHashAlgorithm=FNV_1A_BASE64 pathCreationMap={}", + ex.getMessage() + ); + } + + public void testFromXContent() { + UnsupportedOperationException ex = assertThrows( + UnsupportedOperationException.class, + () -> RemoteIndexPath.fromXContent(Mockito.mock(XContentParser.class)) + ); + assertEquals("RemoteIndexPath.fromXContent() is not supported", ex.getMessage()); + } + + public void testInvalidPathCreationMap() { + IllegalArgumentException ex = assertThrows( + IllegalArgumentException.class, + () -> new RemoteIndexPath( + "djjsid73he8yd7usduh", + 2, + new BlobPath().add("djsd878ndjh").add("hcs87cj8"), + PathType.HASHED_PREFIX, + PathHashAlgorithm.FNV_1A_BASE64, + Map.of(TRANSLOG, List.of(LOCK_FILES)) + ) + ); + assertEquals("pathCreationMap={TRANSLOG=[LOCK_FILES]} is having illegal combination of category and type", ex.getMessage()); + } +} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java new file mode 100644 index 0000000000000..2e4dd15ccb581 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteIndexPathUploaderTests.java @@ -0,0 +1,314 @@ +/* + * 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.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.SetOnce; +import org.opensearch.common.UUIDs; +import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.compress.DeflateCompressor; +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.RemoteClusterStateService; +import org.opensearch.gateway.remote.RemoteClusterStateService.RemoteStateTransferException; +import org.opensearch.index.remote.RemoteStoreEnums.PathType; +import org.opensearch.node.Node; +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.mockito.Mockito; + +import static org.opensearch.index.remote.RemoteStoreEnums.PathType.FIXED; +import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_INFIX; +import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_PREFIX; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class RemoteIndexPathUploaderTests extends OpenSearchTestCase { + + private static final String CLUSTER_STATE_REPO_KEY = Node.NODE_ATTRIBUTES.getKey() + + RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; + + private static final String TRANSLOG_REPO_NAME = "translog-repo"; + private static final String SEGMENT_REPO_NAME = "segment-repo"; + + private final ThreadPool threadPool = new TestThreadPool(getTestName()); + private Settings settings; + private ClusterSettings clusterSettings; + private RepositoriesService repositoriesService; + private BlobStoreRepository repository; + private BlobStore blobStore; + private BlobContainer blobContainer; + private BlobPath basePath; + private List indexMetadataList; + private final AtomicLong successCount = new AtomicLong(); + private final AtomicLong failureCount = new AtomicLong(); + + @Before + public void setup() { + settings = Settings.builder() + .put(RemoteIndexPathUploader.TRANSLOG_REPO_NAME_KEY, TRANSLOG_REPO_NAME) + .put(RemoteIndexPathUploader.SEGMENT_REPO_NAME_KEY, TRANSLOG_REPO_NAME) + .put(CLUSTER_STATE_REPO_KEY, TRANSLOG_REPO_NAME) + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + basePath = BlobPath.cleanPath().add("test"); + repositoriesService = mock(RepositoriesService.class); + repository = mock(BlobStoreRepository.class); + when(repositoriesService.repository(anyString())).thenReturn(repository); + blobStore = mock(BlobStore.class); + when(repository.blobStore()).thenReturn(blobStore); + when(repositoriesService.repository(TRANSLOG_REPO_NAME)).thenReturn(repository); + when(repository.basePath()).thenReturn(basePath); + when(repository.getCompressor()).thenReturn(new DeflateCompressor()); + blobContainer = mock(BlobContainer.class); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); + + Map remoteCustomData = Map.of( + PathType.NAME, + HASHED_PREFIX.name(), + RemoteStoreEnums.PathHashAlgorithm.NAME, + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64.name() + ); + Settings idxSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, UUIDs.randomBase64UUID()) + .build(); + IndexMetadata indexMetadata = new IndexMetadata.Builder("test").settings(idxSettings) + .numberOfShards(1) + .numberOfReplicas(0) + .putCustom(IndexMetadata.REMOTE_STORE_CUSTOM_KEY, remoteCustomData) + .build(); + indexMetadataList = List.of(indexMetadata); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + terminate(threadPool); + } + + public void testInterceptWithNoRemoteDataAttributes() { + Settings settings = Settings.Builder.EMPTY_SETTINGS; + clusterSettings.applySettings(settings); + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + List indexMetadataList = Mockito.mock(List.class); + ActionListener actionListener = ActionListener.wrap( + res -> successCount.incrementAndGet(), + ex -> failureCount.incrementAndGet() + ); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(1, successCount.get()); + assertEquals(0, failureCount.get()); + verify(indexMetadataList, times(0)).stream(); + } + + public void testInterceptWithEmptyIndexMetadataList() { + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + ActionListener actionListener = ActionListener.wrap( + res -> successCount.incrementAndGet(), + ex -> failureCount.incrementAndGet() + ); + remoteIndexPathUploader.doOnNewIndexUpload(Collections.emptyList(), actionListener); + assertEquals(1, successCount.get()); + assertEquals(0, failureCount.get()); + } + + public void testInterceptWithEmptyEligibleIndexMetadataList() { + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + ActionListener actionListener = ActionListener.wrap( + res -> successCount.incrementAndGet(), + ex -> failureCount.incrementAndGet() + ); + + // Case 1 - Null remoteCustomData + List indexMetadataList = new ArrayList<>(); + IndexMetadata indexMetadata = mock(IndexMetadata.class); + indexMetadataList.add(indexMetadata); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(1, successCount.get()); + assertEquals(0, failureCount.get()); + + // Case 2 - Empty remoteCustomData + when(indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)).thenReturn(new HashMap<>()); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(2, successCount.get()); + assertEquals(0, failureCount.get()); + + // Case 3 - RemoteStoreEnums.PathType.NAME not in remoteCustomData map + Map remoteCustomData = Map.of("test", "test"); + when(indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)).thenReturn(remoteCustomData); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(3, successCount.get()); + assertEquals(0, failureCount.get()); + + // Case 4 - RemoteStoreEnums.PathType.NAME is not HASHED_PREFIX + remoteCustomData = Map.of(PathType.NAME, randomFrom(FIXED, HASHED_INFIX).name()); + when(indexMetadata.getCustomData(IndexMetadata.REMOTE_STORE_CUSTOM_KEY)).thenReturn(remoteCustomData); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(4, successCount.get()); + assertEquals(0, failureCount.get()); + } + + public void testInterceptWithSameRepo() throws IOException { + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + ActionListener actionListener = ActionListener.wrap( + res -> successCount.incrementAndGet(), + ex -> failureCount.incrementAndGet() + ); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(1, successCount.get()); + assertEquals(0, failureCount.get()); + verify(blobContainer, times(1)).writeBlob(anyString(), any(InputStream.class), anyLong(), anyBoolean()); + } + + public void testInterceptWithDifferentRepo() throws IOException { + Settings settings = Settings.builder() + .put(this.settings) + .put(RemoteIndexPathUploader.SEGMENT_REPO_NAME_KEY, SEGMENT_REPO_NAME) + .build(); + when(repositoriesService.repository(SEGMENT_REPO_NAME)).thenReturn(repository); + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + ActionListener actionListener = ActionListener.wrap( + res -> successCount.incrementAndGet(), + ex -> failureCount.incrementAndGet() + ); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(1, successCount.get()); + assertEquals(0, failureCount.get()); + verify(blobContainer, times(2)).writeBlob(anyString(), any(InputStream.class), anyLong(), anyBoolean()); + } + + public void testInterceptWithLatchAwaitTimeout() throws IOException { + blobContainer = mock(AsyncMultiStreamBlobContainer.class); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + + Settings settings = Settings.builder() + .put(this.settings) + .put(RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.ZERO) + .build(); + clusterSettings.applySettings(settings); + SetOnce exceptionSetOnce = new SetOnce<>(); + ActionListener actionListener = ActionListener.wrap(res -> successCount.incrementAndGet(), ex -> { + failureCount.incrementAndGet(); + exceptionSetOnce.set(ex); + }); + remoteIndexPathUploader.doOnNewIndexUpload(indexMetadataList, actionListener); + assertEquals(0, successCount.get()); + assertEquals(1, failureCount.get()); + assertTrue(exceptionSetOnce.get() instanceof RemoteStateTransferException); + assertTrue( + exceptionSetOnce.get().getMessage().contains("Timed out waiting while uploading remote index path file for indexes=[test/") + ); + verify(blobContainer, times(0)).writeBlob(anyString(), any(InputStream.class), anyLong(), anyBoolean()); + } + + public void testInterceptWithInterruptedExceptionDuringLatchAwait() throws Exception { + AsyncMultiStreamBlobContainer asyncMultiStreamBlobContainer = mock(AsyncMultiStreamBlobContainer.class); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(asyncMultiStreamBlobContainer); + RemoteIndexPathUploader remoteIndexPathUploader = new RemoteIndexPathUploader( + threadPool, + settings, + () -> repositoriesService, + clusterSettings + ); + remoteIndexPathUploader.start(); + Settings settings = Settings.builder() + .put(this.settings) + .put(RemoteClusterStateService.INDEX_METADATA_UPLOAD_TIMEOUT_SETTING.getKey(), TimeValue.timeValueSeconds(1)) + .build(); + clusterSettings.applySettings(settings); + SetOnce exceptionSetOnce = new SetOnce<>(); + ActionListener actionListener = ActionListener.wrap(res -> successCount.incrementAndGet(), ex -> { + failureCount.incrementAndGet(); + exceptionSetOnce.set(ex); + }); + Thread thread = new Thread(() -> { + try { + remoteIndexPathUploader.onNewIndexUpload(indexMetadataList, actionListener); + } catch (Exception e) { + assertTrue(e instanceof InterruptedException); + assertEquals("sleep interrupted", e.getMessage()); + } + }); + thread.start(); + Thread.sleep(10); + thread.interrupt(); + + assertBusy(() -> { + assertEquals(0, successCount.get()); + assertEquals(1, failureCount.get()); + }); + } + +}