From cd2f2dcbe81f05c3ad7b9a2f5698b2ccf5da389e Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Wed, 24 May 2023 19:32:40 +0530 Subject: [PATCH 01/18] Add multipart upload integration for translog and segment files Signed-off-by: Raghuvansh Raj --- .../RemoteStoreBaseIntegTestCase.java | 4 +- .../RemoteStoreMultipartFileCorruptionIT.java | 111 +++++++++++ .../multipart/RemoteStoreMultipartIT.java | 38 ++++ .../multipart/mocks/MockFsBlobContainer.java | 126 +++++++++++++ .../multipart/mocks/MockFsBlobStore.java | 36 ++++ .../multipart/mocks/MockFsRepository.java | 46 +++++ .../mocks/MockFsRepositoryPlugin.java | 38 ++++ .../common/blobstore/BlobContainer.java | 30 +++ .../org/opensearch/common/util/ByteUtils.java | 10 + .../opensearch/index/shard/FileUploader.java | 88 +++++++++ .../shard/RemoteStoreRefreshListener.java | 119 +----------- .../index/store/RemoteDirectory.java | 4 + .../store/RemoteSegmentStoreDirectory.java | 30 +++ .../transfer/BlobStoreTransferService.java | 121 +++++++++++- .../translog/transfer/TransferService.java | 28 ++- .../transfer/TranslogTransferManager.java | 15 +- .../org/opensearch/threadpool/ThreadPool.java | 14 +- .../RemoteTransferContainerTests.java | 41 +++++ .../RemoteSegmentStoreDirectoryTests.java | 101 ++++++++++ .../index/store/TestUploadTracker.java | 43 +++++ ...oreTransferServiceMockRepositoryTests.java | 174 ++++++++++++++++++ .../BlobStoreTransferServiceTests.java | 10 +- .../TranslogTransferManagerTests.java | 19 +- 23 files changed, 1103 insertions(+), 143 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartFileCorruptionIT.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepository.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepositoryPlugin.java create mode 100644 server/src/main/java/org/opensearch/index/shard/FileUploader.java create mode 100644 server/src/test/java/org/opensearch/index/store/TestUploadTracker.java create mode 100644 server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 2b3fcadfc645e..10f01749ab4c5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -102,9 +102,7 @@ protected void putRepository(Path path) { protected void setupRepo() { internalCluster().startClusterManagerOnlyNode(); absolutePath = randomRepoPath().toAbsolutePath(); - assertAcked( - clusterAdmin().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(Settings.builder().put("location", absolutePath)) - ); + putRepository(absolutePath); } @After diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartFileCorruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartFileCorruptionIT.java new file mode 100644 index 0000000000000..8f375ca6e2b01 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartFileCorruptionIT.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.remotestore.multipart; + +import org.junit.After; +import org.junit.Before; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.UUIDs; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexModule; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.plugins.Plugin; +import org.opensearch.remotestore.multipart.mocks.MockFsRepository; +import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.nio.file.Path; +import java.util.Collection; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +public class RemoteStoreMultipartFileCorruptionIT extends OpenSearchIntegTestCase { + + protected static final String REPOSITORY_NAME = "test-remore-store-repo"; + private static final String INDEX_NAME = "remote-store-test-idx-1"; + + @Override + protected Collection> nodePlugins() { + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockFsRepositoryPlugin.class)).collect(Collectors.toList()); + } + + @Override + protected Settings featureFlagSettings() { + return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.REMOTE_STORE, "true").build(); + } + + @Before + public void setup() { + internalCluster().startClusterManagerOnlyNode(); + Path absolutePath = randomRepoPath().toAbsolutePath(); + putRepository(absolutePath); + } + + protected void putRepository(Path path) { + assertAcked( + clusterAdmin().preparePutRepository(REPOSITORY_NAME) + .setType(MockFsRepositoryPlugin.TYPE) + .setSettings( + Settings.builder() + .put("location", path) + // custom setting for MockFsRepositoryPlugin + .put(MockFsRepository.TRIGGER_DATA_INTEGRITY_FAILURE.getKey(), true) + ) + ); + } + + @After + public void teardown() { + assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); + } + + protected Settings remoteStoreIndexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put("index.refresh_interval", "300s") + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .build(); + } + + private IndexResponse indexSingleDoc() { + return client().prepareIndex(INDEX_NAME) + .setId(UUIDs.randomBase64UUID()) + .setSource(randomAlphaOfLength(5), randomAlphaOfLength(5)) + .get(); + } + + public void testLocalFileCorruptionDuringUpload() { + internalCluster().startDataOnlyNodes(1); + createIndex(INDEX_NAME, remoteStoreIndexSettings()); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + + indexSingleDoc(); + + client().admin() + .indices() + .prepareRefresh(INDEX_NAME) + .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED) + .execute() + .actionGet(); + + // ensuring red cluster meaning shard has failed and is unassigned + ensureRed(INDEX_NAME); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java new file mode 100644 index 0000000000000..a523d5c0f5470 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/RemoteStoreMultipartIT.java @@ -0,0 +1,38 @@ +/* + * 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.multipart; + +import org.opensearch.common.settings.Settings; +import org.opensearch.plugins.Plugin; +import org.opensearch.remotestore.RemoteStoreIT; +import org.opensearch.remotestore.multipart.mocks.MockFsRepositoryPlugin; + +import java.nio.file.Path; +import java.util.Collection; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +public class RemoteStoreMultipartIT extends RemoteStoreIT { + + @Override + protected Collection> nodePlugins() { + return Stream.concat(super.nodePlugins().stream(), Stream.of(MockFsRepositoryPlugin.class)).collect(Collectors.toList()); + } + + @Override + protected void putRepository(Path path) { + assertAcked( + clusterAdmin().preparePutRepository(REPOSITORY_NAME) + .setType(MockFsRepositoryPlugin.TYPE) + .setSettings(Settings.builder().put("location", path)) + ); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java new file mode 100644 index 0000000000000..a56f0a338d296 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java @@ -0,0 +1,126 @@ +/* + * 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.multipart.mocks; + +import org.apache.lucene.index.CorruptIndexException; +import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.common.StreamContext; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.fs.FsBlobContainer; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.blobstore.stream.write.WriteContext; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +public class MockFsBlobContainer extends FsBlobContainer { + + private static final int TRANSFER_TIMEOUT_MILLIS = 30000; + + private final boolean triggerDataIntegrityFailure; + + public MockFsBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { + super(blobStore, blobPath, path); + this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; + } + + @Override + public boolean isMultiStreamUploadSupported() { + return true; + } + + @Override + public CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException { + CompletableFuture completableFuture = new CompletableFuture<>(); + + int nParts = 10; + long partSize = writeContext.getFileSize() / nParts; + StreamContext streamContext = writeContext.getStreamProvider(partSize); + final Path file = path.resolve(writeContext.getFileName()); + byte[] buffer = new byte[(int) writeContext.getFileSize()]; + AtomicLong totalContentRead = new AtomicLong(); + CountDownLatch latch = new CountDownLatch(streamContext.getNumberOfParts()); + for (int partIdx = 0; partIdx < streamContext.getNumberOfParts(); partIdx++) { + int finalPartIdx = partIdx; + Thread thread = new Thread(() -> { + try { + InputStreamContainer inputStreamContainer = streamContext.provideStream(finalPartIdx); + InputStream inputStream = inputStreamContainer.getInputStream(); + long remainingContentLength = inputStreamContainer.getContentLength(); + long offset = partSize * finalPartIdx; + while (remainingContentLength > 0) { + int readContentLength = inputStream.read(buffer, (int) offset, (int) remainingContentLength); + totalContentRead.addAndGet(readContentLength); + remainingContentLength -= readContentLength; + offset += readContentLength; + } + inputStream.close(); + } catch (IOException e) { + completableFuture.completeExceptionally(e); + } finally { + latch.countDown(); + } + }); + thread.start(); + } + try { + if (!latch.await(TRANSFER_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)) { + throw new IOException("Timed out waiting for file transfer to complete for " + writeContext.getFileName()); + } + } catch (InterruptedException e) { + throw new IOException("Await interrupted on CountDownLatch, transfer failed for " + writeContext.getFileName()); + } + try (OutputStream outputStream = Files.newOutputStream(file, StandardOpenOption.CREATE_NEW)) { + outputStream.write(buffer); + } + if (writeContext.getFileSize() != totalContentRead.get()) { + throw new IOException( + "Incorrect content length read for file " + + writeContext.getFileName() + + ", actual file size: " + + writeContext.getFileSize() + + ", bytes read: " + + totalContentRead.get() + ); + } + + try { + // bulks need to succeed for segment files to be generated + if (isSegmentFile(writeContext.getFileName()) && triggerDataIntegrityFailure) { + completableFuture.completeExceptionally( + new RuntimeException( + new CorruptIndexException( + "Data integrity check failure for file: " + writeContext.getFileName(), + writeContext.getFileName() + ) + ) + ); + } else { + writeContext.getUploadFinalizer().accept(true); + completableFuture.complete(null); + } + } catch (Exception e) { + completableFuture.completeExceptionally(e); + } + + return completableFuture; + } + + private boolean isSegmentFile(String filename) { + return !filename.endsWith(".tlog") && !filename.endsWith(".ckp"); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java new file mode 100644 index 0000000000000..435be3f8e66f2 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java @@ -0,0 +1,36 @@ +/* + * 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.multipart.mocks; + +import org.opensearch.OpenSearchException; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.fs.FsBlobStore; + +import java.io.IOException; +import java.nio.file.Path; + +public class MockFsBlobStore extends FsBlobStore { + + private final boolean triggerDataIntegrityFailure; + + public MockFsBlobStore(int bufferSizeInBytes, Path path, boolean readonly, boolean triggerDataIntegrityFailure) throws IOException { + super(bufferSizeInBytes, path, readonly); + this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; + } + + @Override + public BlobContainer blobContainer(BlobPath path) { + try { + return new MockFsBlobContainer(this, path, buildAndCreate(path), triggerDataIntegrityFailure); + } catch (IOException ex) { + throw new OpenSearchException("failed to create blob container", ex); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepository.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepository.java new file mode 100644 index 0000000000000..15a9853477081 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepository.java @@ -0,0 +1,46 @@ +/* + * 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.multipart.mocks; + +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.settings.Setting; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.Environment; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.repositories.fs.FsRepository; + +public class MockFsRepository extends FsRepository { + + public static Setting TRIGGER_DATA_INTEGRITY_FAILURE = Setting.boolSetting( + "mock_fs_repository.trigger_data_integrity_failure", + false + ); + + private final boolean triggerDataIntegrityFailure; + + public MockFsRepository( + RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings + ) { + super(metadata, environment, namedXContentRegistry, clusterService, recoverySettings); + triggerDataIntegrityFailure = TRIGGER_DATA_INTEGRITY_FAILURE.get(metadata.settings()); + } + + @Override + protected BlobStore createBlobStore() throws Exception { + FsBlobStore fsBlobStore = (FsBlobStore) super.createBlobStore(); + return new MockFsBlobStore(fsBlobStore.bufferSizeInBytes(), fsBlobStore.path(), isReadOnly(), triggerDataIntegrityFailure); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepositoryPlugin.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepositoryPlugin.java new file mode 100644 index 0000000000000..ffd53adf4e29e --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsRepositoryPlugin.java @@ -0,0 +1,38 @@ +/* + * 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.multipart.mocks; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.Environment; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.RepositoryPlugin; +import org.opensearch.repositories.Repository; + +import java.util.Collections; +import java.util.Map; + +public class MockFsRepositoryPlugin extends Plugin implements RepositoryPlugin { + + public static final String TYPE = "fs_multipart_repository"; + + @Override + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings + ) { + return Collections.singletonMap( + "fs_multipart_repository", + metadata -> new MockFsRepository(metadata, env, namedXContentRegistry, clusterService, recoverySettings) + ); + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index e626824e7e271..cfc5238231d8e 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -32,6 +32,8 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.blobstore.stream.write.WriteContext; + import org.opensearch.action.ActionListener; import java.io.IOException; @@ -42,6 +44,7 @@ import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; /** * An interface for managing a repository of blob entries, where each blob entry is just a named group of bytes. @@ -128,6 +131,33 @@ default long readBlobPreferredLength() { */ void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException; + /** + * Used to check whether vendor plugin support for parallel upload of multiple streams is enabled or not. + * Returns false by default + * + * @return If multi-stream parallel uploads are supported + */ + default boolean isMultiStreamUploadSupported() { + return false; + } + + default boolean isRemoteDataIntegritySupported() { + return false; + } + + /** + * Reads blob content from multiple streams, each from a specific part of the file, which is provided by the + * StreamContextSupplier in the WriteContext passed to this method. An {@link IOException} is thrown if reading + * any of the input streams fails, or writing to the target blob fails + * + * @param writeContext A WriteContext object encapsulating all information needed to perform the upload + * @return A {@link CompletableFuture} representing the upload + * @throws IOException if any of the input streams could not be read, or the target blob could not be written to + */ + default CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException { + throw new UnsupportedOperationException(); + } + /** * Reads blob content from the input stream and writes it to the container in a new blob with the given name, * using an atomic write operation if the implementation supports it. diff --git a/server/src/main/java/org/opensearch/common/util/ByteUtils.java b/server/src/main/java/org/opensearch/common/util/ByteUtils.java index 36ae3b1f5bcaa..8c7665d991751 100644 --- a/server/src/main/java/org/opensearch/common/util/ByteUtils.java +++ b/server/src/main/java/org/opensearch/common/util/ByteUtils.java @@ -61,6 +61,16 @@ public static void writeLongLE(long l, byte[] arr, int offset) { assert l == 0; } + /** Convert long to a byte array in big-endian format */ + public static byte[] toByteArrayBE(long l) { + byte[] result = new byte[8]; + for (int i = 7; i >= 0; i--) { + result[i] = (byte) (l & 0xffL); + l >>= 8; + } + return result; + } + /** Write a long in little-endian format. */ public static long readLongLE(byte[] arr, int offset) { long l = arr[offset++] & 0xFFL; diff --git a/server/src/main/java/org/opensearch/index/shard/FileUploader.java b/server/src/main/java/org/opensearch/index/shard/FileUploader.java new file mode 100644 index 0000000000000..98f04f76d1594 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/FileUploader.java @@ -0,0 +1,88 @@ +/* + * 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.shard; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.IOContext; +import org.opensearch.common.CheckedFunction; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; + +import java.io.IOException; +import java.util.Collection; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * This class is a wrapper over the copying of file from local to remote store allowing to decorate the actual copy + * method along with adding hooks of code that can be run before, on success and on failure. + * + * @opensearch.internal + */ +public class FileUploader { + + private static final Logger logger = LogManager.getLogger(FileUploader.class); + + private final UploadTracker uploadTracker; + + private final RemoteSegmentStoreDirectory remoteDirectory; + + private final Directory storeDirectory; + + private final Set excludeFiles; + + private final CheckedFunction checksumProvider; + + public FileUploader( + UploadTracker uploadTracker, + RemoteSegmentStoreDirectory remoteDirectory, + Directory storeDirectory, + Set excludeFiles, + CheckedFunction checksumProvider + ) { + this.uploadTracker = uploadTracker; + this.remoteDirectory = remoteDirectory; + this.storeDirectory = storeDirectory; + this.excludeFiles = excludeFiles; + this.checksumProvider = checksumProvider; + } + + /** + * Calling this method will filter out files that need to be skipped and call + * {@link RemoteSegmentStoreDirectory#copyFilesFrom} + * + * @param files The files that need to be uploaded + * @return A boolean for whether all files were successful or not + * @throws Exception when the underlying upload fails + */ + public boolean uploadFiles(Collection files) throws Exception { + Collection filteredFiles = files.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); + return remoteDirectory.copyFilesFrom(storeDirectory, filteredFiles, IOContext.DEFAULT, uploadTracker); + } + + /** + * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. + * + * @param file that needs to be uploaded. + * @return true if the upload has to be skipped for the file. + */ + private boolean skipUpload(String file) { + try { + // Exclude files that are already uploaded and the exclude files to come up with the list of files to be uploaded. + return excludeFiles.contains(file) || remoteDirectory.containsFile(file, checksumProvider.apply(file)); + } catch (IOException e) { + logger.error( + "Exception while reading checksum of local segment file: {}, ignoring the exception and re-uploading the file", + file + ); + } + return false; + } +} diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 46d52bc8ca5df..1141b9c24c5b5 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.ReferenceManager; @@ -19,7 +20,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.opensearch.action.bulk.BackoffPolicy; -import org.opensearch.common.CheckedFunction; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.logging.Loggers; import org.opensearch.common.unit.TimeValue; @@ -156,7 +156,7 @@ public void onFailure(String file) { // Track upload failure segmentTracker.addUploadBytesFailed(latestFileNameSizeOnLocalMap.get(file)); } - }, remoteDirectory, storeDirectory, this::getChecksumOfLocalFile, logger); + }, remoteDirectory, storeDirectory, EXCLUDE_FILES, this::getChecksumOfLocalFile); } @Override @@ -380,14 +380,15 @@ void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos se private boolean uploadNewSegments(Collection localSegmentsPostRefresh) throws IOException { AtomicBoolean uploadSuccess = new AtomicBoolean(true); - localSegmentsPostRefresh.forEach(file -> { - try { - fileUploader.uploadFile(file); - } catch (IOException e) { - uploadSuccess.set(false); - logger.warn(() -> new ParameterizedMessage("Exception while uploading file {} to the remote segment store", file), e); + try { + uploadSuccess.set(fileUploader.uploadFiles(localSegmentsPostRefresh)); + } catch (Exception e) { + uploadSuccess.set(false); + if (e instanceof CorruptIndexException) { + indexShard.failShard(e.getMessage(), e); } - }); + logger.warn(() -> new ParameterizedMessage("Exception: [{}] while uploading segment files", e), e); + } return uploadSuccess.get(); } @@ -462,104 +463,4 @@ private void updateFinalUploadStatusInSegmentTracker(boolean uploadStatus, long segmentTracker.incrementTotalUploadsFailed(); } } - - /** - * This class is a wrapper over the copying of file from local to remote store allowing to decorate the actual copy - * method along with adding hooks of code that can be run before, on success and on failure. - * - * @opensearch.internal - */ - private static class FileUploader { - - private final Logger logger; - - private final UploadTracker uploadTracker; - - private final RemoteSegmentStoreDirectory remoteDirectory; - - private final Directory storeDirectory; - - private final CheckedFunction checksumProvider; - - public FileUploader( - UploadTracker uploadTracker, - RemoteSegmentStoreDirectory remoteDirectory, - Directory storeDirectory, - CheckedFunction checksumProvider, - Logger logger - ) { - this.uploadTracker = uploadTracker; - this.remoteDirectory = remoteDirectory; - this.storeDirectory = storeDirectory; - this.checksumProvider = checksumProvider; - this.logger = logger; - } - - /** - * Calling this method will lead to before getting executed and then the actual upload. Based on the upload status, - * the onSuccess or onFailure method gets invoked. - * - * @param file the file which is to be uploaded. - * @throws IOException is thrown if the upload fails. - */ - private void uploadFile(String file) throws IOException { - if (skipUpload(file)) { - return; - } - uploadTracker.beforeUpload(file); - boolean success = false; - try { - performUpload(file); - uploadTracker.onSuccess(file); - success = true; - } finally { - if (!success) { - uploadTracker.onFailure(file); - } - } - } - - /** - * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. - * - * @param file that needs to be uploaded. - * @return true if the upload has to be skipped for the file. - */ - private boolean skipUpload(String file) { - try { - // Exclude files that are already uploaded and the exclude files to come up with the list of files to be uploaded. - return EXCLUDE_FILES.contains(file) || remoteDirectory.containsFile(file, checksumProvider.apply(file)); - } catch (IOException e) { - logger.error( - "Exception while reading checksum of local segment file: {}, ignoring the exception and re-uploading the file", - file - ); - } - return false; - } - - /** - * This method does the actual upload. - * - * @param file that needs to be uploaded. - * @throws IOException is thrown if the upload fails. - */ - private void performUpload(String file) throws IOException { - remoteDirectory.copyFrom(storeDirectory, file, file, IOContext.DEFAULT); - } - } - - /** - * A tracker class that is fed to FileUploader. - * - * @opensearch.internal - */ - interface UploadTracker { - - void beforeUpload(String file); - - void onSuccess(String file); - - void onFailure(String file); - } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 8782808c070ab..f7fe7ca62e6ba 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -45,6 +45,10 @@ public class RemoteDirectory extends Directory { protected final BlobContainer blobContainer; + public BlobContainer getBlobContainer() { + return blobContainer; + } + public RemoteDirectory(BlobContainer blobContainer) { this.blobContainer = blobContainer; } 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 e7602203440d2..f69b09e630c1d 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -8,6 +8,7 @@ package org.opensearch.index.store; +import com.jcraft.jzlib.JZlib; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.CodecUtil; @@ -22,7 +23,9 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.common.util.ByteUtils; import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; @@ -44,6 +47,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.zip.CRC32; /** * A RemoteDirectory extension for remote segment store. We need to make sure we don't overwrite a segment file once uploaded. @@ -62,6 +66,11 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement */ public static final String SEGMENT_NAME_UUID_SEPARATOR = "__"; + /** + * Number of bytes in the segment file to store checksum + */ + private static final int SEGMENT_CHECKSUM_BYTES = 8; + /** * remoteDataDirectory is used to store segment files at path: cluster_UUID/index_UUID/shardId/segments/data */ @@ -528,6 +537,27 @@ private String getChecksumOfLocalFile(Directory directory, String file) throws I } } + private long calculateChecksumOfChecksum(Directory directory, String file) throws IOException { + try (IndexInput indexInput = directory.openInput(file, IOContext.DEFAULT)) { + long storedChecksum = CodecUtil.retrieveChecksum(indexInput); + CRC32 checksumOfChecksum = new CRC32(); + checksumOfChecksum.update(ByteUtils.toByteArrayBE(storedChecksum)); + try { + return JZlib.crc32_combine(storedChecksum, checksumOfChecksum.getValue(), SEGMENT_CHECKSUM_BYTES); + } catch (Exception e) { + throw new ChecksumCombinationException( + "Potentially corrupted file: Checksum combination failed while combining stored checksum " + + "and calculated checksum of stored checksum in segment file: " + + file + + ", directory: " + + directory, + file, + e + ); + } + } + } + private String getExistingRemoteFilename(String localFilename) { if (segmentsUploadedToRemoteStore.containsKey(localFilename)) { return segmentsUploadedToRemoteStore.get(localFilename).uploadedFilename; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index d9feb1a832681..82ee1cb9cb319 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -16,13 +16,24 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.stream.write.WriteContext; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeFileInputStream; +import org.opensearch.index.translog.ChannelFactory; +import org.opensearch.index.translog.checked.TranslogCheckedContainer; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.io.InputStream; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; import static org.opensearch.common.blobstore.BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC; @@ -44,18 +55,18 @@ public BlobStoreTransferService(BlobStore blobStore, ThreadPool threadPool) { } @Override - public void uploadBlobAsync( - String threadpoolName, + public void uploadBlobByThreadPool( + String threadPoolName, final TransferFileSnapshot fileSnapshot, Iterable remoteTransferPath, - ActionListener listener + ActionListener listener, + WritePriority writePriority ) { assert remoteTransferPath instanceof BlobPath; BlobPath blobPath = (BlobPath) remoteTransferPath; - threadPool.executor(threadpoolName).execute(ActionRunnable.wrap(listener, l -> { - try (InputStream inputStream = fileSnapshot.inputStream()) { - blobStore.blobContainer(blobPath) - .writeBlobAtomic(fileSnapshot.getName(), inputStream, fileSnapshot.getContentLength(), true); + threadPool.executor(threadPoolName).execute(ActionRunnable.wrap(listener, l -> { + try { + uploadBlob(fileSnapshot, blobPath, writePriority); l.onResponse(fileSnapshot); } catch (Exception e) { logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), e); @@ -65,14 +76,106 @@ public void uploadBlobAsync( } @Override - public void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remoteTransferPath) throws IOException { - assert remoteTransferPath instanceof BlobPath; + public void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remoteTransferPath, WritePriority writePriority) + throws IOException { BlobPath blobPath = (BlobPath) remoteTransferPath; try (InputStream inputStream = fileSnapshot.inputStream()) { blobStore.blobContainer(blobPath).writeBlobAtomic(fileSnapshot.getName(), inputStream, fileSnapshot.getContentLength(), true); } } + @Override + public void uploadBlobs( + Set fileSnapshots, + final Map blobPaths, + ActionListener listener, + WritePriority writePriority + ) { + List> resultFutures = new ArrayList<>(); + fileSnapshots.forEach(fileSnapshot -> { + BlobPath blobPath = blobPaths.get(fileSnapshot.getPrimaryTerm()); + if (!blobStore.blobContainer(blobPath).isMultiStreamUploadSupported()) { + uploadBlobByThreadPool(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); + } else { + CompletableFuture resultFuture = createUploadFuture(fileSnapshot, listener, blobPath, writePriority); + if (resultFuture != null) { + resultFutures.add(resultFuture); + } + } + }); + + if (resultFutures.isEmpty() == false) { + CompletableFuture resultFuture = CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); + try { + resultFuture.get(); + } catch (Exception e) { + logger.warn("Failed to upload blobs", e); + } + } + } + + private CompletableFuture createUploadFuture( + TransferFileSnapshot fileSnapshot, + ActionListener listener, + BlobPath blobPath, + WritePriority writePriority + ) { + + CompletableFuture resultFuture = null; + try { + ChannelFactory channelFactory = FileChannel::open; + long contentLength; + long expectedChecksum; + try (FileChannel channel = channelFactory.open(fileSnapshot.getPath(), StandardOpenOption.READ)) { + contentLength = channel.size(); + TranslogCheckedContainer translogCheckedContainer = new TranslogCheckedContainer( + channel, + 0, + (int) contentLength, + fileSnapshot.getName() + ); + expectedChecksum = translogCheckedContainer.getChecksum(); + } + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + fileSnapshot.getName(), + fileSnapshot.getName(), + contentLength, + true, + writePriority, + (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), + expectedChecksum, + blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported(), + false + ); + WriteContext writeContext = remoteTransferContainer.createWriteContext(); + CompletableFuture uploadFuture = blobStore.blobContainer(blobPath).writeBlobByStreams(writeContext); + resultFuture = uploadFuture.whenComplete((resp, throwable) -> { + try { + remoteTransferContainer.close(); + } catch (Exception e) { + logger.warn("Error occurred while closing streams", e); + } + if (throwable != null) { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), throwable); + listener.onFailure(new FileTransferException(fileSnapshot, throwable)); + } else { + listener.onResponse(fileSnapshot); + } + }); + } catch (Exception e) { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), e); + listener.onFailure(new FileTransferException(fileSnapshot, e)); + } finally { + try { + fileSnapshot.close(); + } catch (IOException e) { + logger.warn("Error while closing TransferFileSnapshot", e); + } + } + + return resultFuture; + } + @Override public InputStream downloadBlob(Iterable path, String fileName) throws IOException { return blobStore.blobContainer((BlobPath) path).readBlob(fileName); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java index 0e6496042e3d8..f74517e992782 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java @@ -10,11 +10,14 @@ import org.opensearch.action.ActionListener; import org.opensearch.common.blobstore.BlobMetadata; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import java.io.IOException; import java.io.InputStream; import java.util.List; +import java.util.Map; import java.util.Set; /** @@ -26,25 +29,40 @@ public interface TransferService { /** * Uploads the {@link TransferFileSnapshot} async, once the upload is complete the callback is invoked - * @param threadpoolName threadpool type which will be used to upload blobs asynchronously + * @param threadPoolName threadpool type which will be used to upload blobs asynchronously * @param fileSnapshot the file snapshot to upload * @param remotePath the remote path where upload should be made * @param listener the callback to be invoked once upload completes successfully/fails */ - void uploadBlobAsync( - String threadpoolName, + void uploadBlobByThreadPool( + String threadPoolName, final TransferFileSnapshot fileSnapshot, Iterable remotePath, - ActionListener listener + ActionListener listener, + WritePriority writePriority ); + /** + * Uploads multiple {@link TransferFileSnapshot}, once the upload is complete the callback is invoked + * @param fileSnapshots the file snapshots to upload + * @param blobPaths Primary term to {@link BlobPath} map + * @param listener the callback to be invoked once uploads complete successfully/fail + */ + void uploadBlobs( + Set fileSnapshots, + final Map blobPaths, + ActionListener listener, + WritePriority writePriority + ) throws Exception; + /** * Uploads the {@link TransferFileSnapshot} blob * @param fileSnapshot the file snapshot to upload * @param remotePath the remote path where upload should be made + * @param writePriority Priority by which content needs to be written. * @throws IOException the exception while transferring the data */ - void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remotePath) throws IOException; + void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remotePath, WritePriority writePriority) throws IOException; void deleteBlobs(Iterable path, List fileNames) throws IOException; 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 54140226e3744..0c63a7ffe4cce 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 @@ -17,6 +17,7 @@ import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -119,14 +120,16 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans }), latch ); + Map blobPathMap = new HashMap<>(); toUpload.forEach( - fileSnapshot -> transferService.uploadBlobAsync( - ThreadPool.Names.TRANSLOG_TRANSFER, - fileSnapshot, - remoteDataTransferPath.add(String.valueOf(fileSnapshot.getPrimaryTerm())), - latchedActionListener + fileSnapshot -> blobPathMap.put( + fileSnapshot.getPrimaryTerm(), + remoteDataTransferPath.add(String.valueOf(fileSnapshot.getPrimaryTerm())) ) ); + + transferService.uploadBlobs(toUpload, blobPathMap, latchedActionListener, WritePriority.HIGH); + try { if (latch.await(TRANSFER_TIMEOUT_IN_MILLIS, TimeUnit.MILLISECONDS) == false) { Exception ex = new TimeoutException("Timed out waiting for transfer of snapshot " + transferSnapshot + " to complete"); @@ -139,7 +142,7 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans throw ex; } if (exceptionList.isEmpty()) { - transferService.uploadBlob(prepareMetadata(transferSnapshot), remoteMetadataTransferPath); + transferService.uploadBlob(prepareMetadata(transferSnapshot), remoteMetadataTransferPath, WritePriority.HIGH); translogTransferListener.onUploadComplete(transferSnapshot); return true; } else { diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index ebc68c288e25a..21659fea90c2f 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -46,10 +46,10 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.concurrent.OpenSearchExecutors; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.concurrent.XRejectedExecutionHandler; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.node.Node; @@ -62,8 +62,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ScheduledExecutorService; @@ -451,6 +453,16 @@ public ScheduledCancellable schedule(Runnable command, TimeValue delay, String e return new ScheduledCancellableAdapter(scheduler.schedule(command, delay.millis(), TimeUnit.MILLISECONDS)); } + public Future executeCallable(Callable command, String executorName) { + ExecutorService executorService = executor(executorName); + return executorService.submit(command); + } + + public boolean isExecutorShutDown(String executorName) { + ExecutorService executorService = executor(executorName); + return executorService.isShutdown(); + } + public void scheduleUnlessShuttingDown(TimeValue delay, String executor, Runnable command) { try { schedule(command, delay, executor); 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 1ebec46042247..48940a0d401fd 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 @@ -9,6 +9,7 @@ package org.opensearch.common.blobstore.transfer; import org.junit.Before; +import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.io.InputStreamContainer; import org.opensearch.common.StreamContext; import org.opensearch.common.blobstore.stream.write.WritePriority; @@ -21,6 +22,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.UUID; public class RemoteTransferContainerTests extends OpenSearchTestCase { @@ -140,6 +142,45 @@ public void testTypeOfProvidedStreamsAllCases() throws IOException { testTypeOfProvidedStreams(false); } + public void testCreateWriteContextAllCases() throws IOException { + testCreateWriteContext(true); + testCreateWriteContext(false); + } + + private void testCreateWriteContext(boolean doRemoteDataIntegrityCheck) throws IOException { + String remoteFileName = testFile.getFileName().toString() + UUID.randomUUID(); + Long expectedChecksum = randomLong(); + try ( + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + testFile.getFileName().toString(), + remoteFileName, + TEST_FILE_SIZE_BYTES, + true, + WritePriority.HIGH, + new RemoteTransferContainer.OffsetRangeInputStreamSupplier() { + @Override + public OffsetRangeInputStream get(long size, long position) throws IOException { + return new OffsetRangeFileInputStream(testFile, size, position); + } + }, + expectedChecksum, + doRemoteDataIntegrityCheck + ) + ) { + WriteContext writeContext = remoteTransferContainer.createWriteContext(); + assertEquals(remoteFileName, writeContext.getFileName()); + assertTrue(writeContext.isFailIfAlreadyExists()); + assertEquals(TEST_FILE_SIZE_BYTES, writeContext.getFileSize()); + assertEquals(WritePriority.HIGH, writeContext.getWritePriority()); + assertEquals(doRemoteDataIntegrityCheck, writeContext.doRemoteDataIntegrityCheck()); + if (doRemoteDataIntegrityCheck) { + assertEquals(expectedChecksum, writeContext.getExpectedChecksum()); + } else { + assertNull(writeContext.getExpectedChecksum()); + } + } + } + private void testTypeOfProvidedStreams(boolean isRemoteDataIntegritySupported) throws IOException { try ( RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( 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 c37893877253e..99bfcfddc0b6a 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -25,6 +25,8 @@ import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.UUIDs; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -48,6 +50,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.HashMap; +import java.util.Collection; import java.util.concurrent.ExecutorService; import static org.mockito.Mockito.mock; @@ -68,6 +74,7 @@ public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { private RemoteStoreMetadataLockManager mdLockManager; private RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; + private TestUploadTracker testUploadTracker; private IndexShard indexShard; private SegmentInfos segmentInfos; private ThreadPool threadPool; @@ -89,6 +96,7 @@ public void setup() throws IOException { mdLockManager, threadPool ); + testUploadTracker = new TestUploadTracker(); Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); @@ -503,6 +511,99 @@ public void testCopyFrom() throws IOException { storeDirectory.close(); } + public void testCopyFilesFromMultipart() throws Exception { + String filename = "_100.si"; + populateMetadata(); + remoteSegmentStoreDirectory.init(); + + Directory storeDirectory = LuceneTestCase.newDirectory(); + IndexOutput indexOutput = storeDirectory.createOutput(filename, IOContext.DEFAULT); + indexOutput.writeString("Hello World!"); + CodecUtil.writeFooter(indexOutput); + indexOutput.close(); + storeDirectory.sync(List.of(filename)); + + assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); + uploadResponseCompletableFuture.complete(null); + when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + + remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker); + + assertTrue(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + assertEquals(TestUploadTracker.UploadStatus.UPLOAD_SUCCESS, testUploadTracker.getUploadStatus(filename)); + + storeDirectory.close(); + } + + public void testCopyFilesFromMultipartIOException() throws Exception { + String filename = "_100.si"; + populateMetadata(); + remoteSegmentStoreDirectory.init(); + + Directory storeDirectory = LuceneTestCase.newDirectory(); + IndexOutput indexOutput = storeDirectory.createOutput(filename, IOContext.DEFAULT); + indexOutput.writeString("Hello World!"); + CodecUtil.writeFooter(indexOutput); + indexOutput.close(); + storeDirectory.sync(List.of(filename)); + + assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); + uploadResponseCompletableFuture.complete(null); + when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenThrow(new IOException()); + + assertThrows( + IOException.class, + () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker) + ); + + assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + assertEquals(TestUploadTracker.UploadStatus.UPLOAD_FAILURE, testUploadTracker.getUploadStatus(filename)); + + storeDirectory.close(); + } + + public void testCopyFilesFromMultipartUploadFutureCompletedExceptionally() throws Exception { + String filename = "_100.si"; + populateMetadata(); + remoteSegmentStoreDirectory.init(); + + Directory storeDirectory = LuceneTestCase.newDirectory(); + IndexOutput indexOutput = storeDirectory.createOutput(filename, IOContext.DEFAULT); + indexOutput.writeString("Hello World!"); + CodecUtil.writeFooter(indexOutput); + indexOutput.close(); + storeDirectory.sync(List.of(filename)); + + assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); + uploadResponseCompletableFuture.completeExceptionally(new IOException()); + when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + + assertThrows( + ExecutionException.class, + () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker) + ); + + assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); + assertEquals(TestUploadTracker.UploadStatus.UPLOAD_FAILURE, testUploadTracker.getUploadStatus(filename)); + + storeDirectory.close(); + } + public void testCopyFromException() throws IOException { String filename = "_100.si"; Directory storeDirectory = LuceneTestCase.newDirectory(); diff --git a/server/src/test/java/org/opensearch/index/store/TestUploadTracker.java b/server/src/test/java/org/opensearch/index/store/TestUploadTracker.java new file mode 100644 index 0000000000000..05250d69b7867 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/TestUploadTracker.java @@ -0,0 +1,43 @@ +/* + * 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.store; + +import org.opensearch.index.shard.UploadTracker; + +import java.util.concurrent.ConcurrentHashMap; + +public class TestUploadTracker implements UploadTracker { + + private final ConcurrentHashMap uploadStatusMap = new ConcurrentHashMap<>(); + + enum UploadStatus { + BEFORE_UPLOAD, + UPLOAD_SUCCESS, + UPLOAD_FAILURE + } + + @Override + public void beforeUpload(String file) { + uploadStatusMap.put(file, UploadStatus.BEFORE_UPLOAD); + } + + @Override + public void onSuccess(String file) { + uploadStatusMap.put(file, UploadStatus.UPLOAD_SUCCESS); + } + + @Override + public void onFailure(String file) { + uploadStatusMap.put(file, UploadStatus.UPLOAD_FAILURE); + } + + public UploadStatus getUploadStatus(String file) { + return uploadStatusMap.get(file); + } +} diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java new file mode 100644 index 0000000000000..3020703cd398f --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -0,0 +1,174 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog.transfer; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.LatchedActionListener; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.stream.write.WriteContext; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.Collections; +import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class BlobStoreTransferServiceMockRepositoryTests extends OpenSearchTestCase { + + private ThreadPool threadPool; + + private BlobStore blobStore; + + @Override + public void setUp() throws Exception { + super.setUp(); + blobStore = mock(BlobStore.class); + threadPool = new TestThreadPool(getClass().getName()); + } + + public void testUploadBlobs() throws Exception { + Path testFile = createTempFile(); + Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); + uploadResponseCompletableFuture.complete(null); + when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); + + TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); + CountDownLatch latch = new CountDownLatch(1); + AtomicBoolean onResponseCalled = new AtomicBoolean(false); + AtomicReference exceptionRef = new AtomicReference<>(); + AtomicReference fileSnapshotRef = new AtomicReference<>(); + transferService.uploadBlobs(Collections.singleton(transferFileSnapshot), new HashMap<>() { + { + put(transferFileSnapshot.getPrimaryTerm(), new BlobPath().add("sample_path")); + } + }, new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { + onResponseCalled.set(true); + fileSnapshotRef.set(fileSnapshot); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }, latch), WritePriority.HIGH); + + assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + assertTrue(onResponseCalled.get()); + assertEquals(transferFileSnapshot.getPrimaryTerm(), fileSnapshotRef.get().getPrimaryTerm()); + assertEquals(transferFileSnapshot.getName(), fileSnapshotRef.get().getName()); + assertNull(exceptionRef.get()); + } + + public void testUploadBlobsIOException() throws Exception { + Path testFile = createTempFile(); + Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + doThrow(new IOException()).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); + + TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); + CountDownLatch latch = new CountDownLatch(1); + AtomicBoolean onResponseCalled = new AtomicBoolean(false); + AtomicReference exceptionRef = new AtomicReference<>(); + transferService.uploadBlobs(Collections.singleton(transferFileSnapshot), new HashMap<>() { + { + put(transferFileSnapshot.getPrimaryTerm(), new BlobPath().add("sample_path")); + } + }, new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { + onResponseCalled.set(true); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }, latch), WritePriority.HIGH); + + assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + assertFalse(onResponseCalled.get()); + assertTrue(exceptionRef.get() instanceof FileTransferException); + } + + public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception { + Path testFile = createTempFile(); + Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + + BlobContainer blobContainer = mock(BlobContainer.class); + when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); + uploadResponseCompletableFuture.completeExceptionally(new IOException()); + when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); + + TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); + CountDownLatch latch = new CountDownLatch(1); + AtomicBoolean onResponseCalled = new AtomicBoolean(false); + AtomicReference exceptionRef = new AtomicReference<>(); + transferService.uploadBlobs(Collections.singleton(transferFileSnapshot), new HashMap<>() { + { + put(transferFileSnapshot.getPrimaryTerm(), new BlobPath().add("sample_path")); + } + }, new LatchedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { + onResponseCalled.set(true); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }, latch), WritePriority.HIGH); + + assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + assertFalse(onResponseCalled.get()); + assertTrue(exceptionRef.get() instanceof FileTransferException); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + ThreadPool.terminate(threadPool, 10, TimeUnit.SECONDS); + } +} diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index 196fbd58c2c20..cc00c49892853 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -12,6 +12,7 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.env.Environment; @@ -51,7 +52,7 @@ public void testUploadBlob() throws IOException { Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); - transferService.uploadBlob(transferFileSnapshot, repository.basePath()); + transferService.uploadBlob(transferFileSnapshot, repository.basePath(), WritePriority.HIGH); } public void testUploadBlobFromByteArray() throws IOException { @@ -61,7 +62,7 @@ public void testUploadBlobFromByteArray() throws IOException { 1 ); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); - transferService.uploadBlob(transferFileSnapshot, repository.basePath()); + transferService.uploadBlob(transferFileSnapshot, repository.basePath(), WritePriority.NORMAL); } public void testUploadBlobAsync() throws IOException, InterruptedException { @@ -71,7 +72,7 @@ public void testUploadBlobAsync() throws IOException, InterruptedException { FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); CountDownLatch latch = new CountDownLatch(1); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); - transferService.uploadBlobAsync( + transferService.uploadBlobByThreadPool( ThreadPool.Names.TRANSLOG_TRANSFER, transferFileSnapshot, repository.basePath(), @@ -87,7 +88,8 @@ public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { public void onFailure(Exception e) { throw new AssertionError("Failed to perform uploadBlobAsync", e); } - }, latch) + }, latch), + WritePriority.HIGH ); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); assertTrue(succeeded.get()); 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 5f8aa64457896..0928c9bf15344 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 @@ -17,6 +17,7 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; import org.opensearch.index.translog.Translog; @@ -41,6 +42,8 @@ import java.util.concurrent.atomic.AtomicInteger; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.anySet; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; @@ -80,20 +83,24 @@ public void tearDown() throws Exception { } @SuppressWarnings("unchecked") - public void testTransferSnapshot() throws IOException { + public void testTransferSnapshot() throws Exception { AtomicInteger fileTransferSucceeded = new AtomicInteger(); AtomicInteger fileTransferFailed = new AtomicInteger(); AtomicInteger translogTransferSucceeded = new AtomicInteger(); AtomicInteger translogTransferFailed = new AtomicInteger(); doNothing().when(transferService) - .uploadBlob(any(TransferFileSnapshot.class), Mockito.eq(remoteBaseTransferPath.add(String.valueOf(primaryTerm)))); + .uploadBlob( + any(TransferFileSnapshot.class), + Mockito.eq(remoteBaseTransferPath.add(String.valueOf(primaryTerm))), + any(WritePriority.class) + ); doAnswer(invocationOnMock -> { - ActionListener listener = (ActionListener) invocationOnMock.getArguments()[3]; - listener.onResponse((TransferFileSnapshot) invocationOnMock.getArguments()[1]); + ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; + Set transferFileSnapshots = (Set) invocationOnMock.getArguments()[0]; + transferFileSnapshots.forEach(listener::onResponse); return null; - }).when(transferService) - .uploadBlobAsync(any(String.class), any(TransferFileSnapshot.class), any(BlobPath.class), any(ActionListener.class)); + }).when(transferService).uploadBlobs(anySet(), anyMap(), any(ActionListener.class), any(WritePriority.class)); FileTransferTracker fileTransferTracker = new FileTransferTracker(new ShardId("index", "indexUUid", 0)) { @Override From 8c13090f9d2ca9343527579a97a3d9cc2cf69a63 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Mon, 5 Jun 2023 19:35:45 +0530 Subject: [PATCH 02/18] Addressing PR comments Signed-off-by: Raghuvansh Raj --- .../org/opensearch/index/shard/FileUploader.java | 7 ++++--- .../index/shard/RemoteStoreRefreshListener.java | 3 ++- .../index/store/RemoteSegmentStoreDirectory.java | 1 + .../store/RemoteSegmentStoreDirectoryTests.java | 16 ++++++++-------- ...ploadTracker.java => TestUploadListener.java} | 4 ++-- 5 files changed, 17 insertions(+), 14 deletions(-) rename server/src/test/java/org/opensearch/index/store/{TestUploadTracker.java => TestUploadListener.java} (89%) diff --git a/server/src/main/java/org/opensearch/index/shard/FileUploader.java b/server/src/main/java/org/opensearch/index/shard/FileUploader.java index 98f04f76d1594..04612dace533e 100644 --- a/server/src/main/java/org/opensearch/index/shard/FileUploader.java +++ b/server/src/main/java/org/opensearch/index/shard/FileUploader.java @@ -13,6 +13,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.opensearch.common.CheckedFunction; +import org.opensearch.common.util.UploadListener; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import java.io.IOException; @@ -30,7 +31,7 @@ public class FileUploader { private static final Logger logger = LogManager.getLogger(FileUploader.class); - private final UploadTracker uploadTracker; + private final UploadListener uploadListener; private final RemoteSegmentStoreDirectory remoteDirectory; @@ -41,13 +42,13 @@ public class FileUploader { private final CheckedFunction checksumProvider; public FileUploader( - UploadTracker uploadTracker, + UploadListener uploadListener, RemoteSegmentStoreDirectory remoteDirectory, Directory storeDirectory, Set excludeFiles, CheckedFunction checksumProvider ) { - this.uploadTracker = uploadTracker; + this.uploadListener = uploadListener; this.remoteDirectory = remoteDirectory; this.storeDirectory = storeDirectory; this.excludeFiles = excludeFiles; diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 1141b9c24c5b5..29b6a73502984 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -23,6 +23,7 @@ import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.logging.Loggers; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.UploadListener; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.index.engine.EngineException; import org.opensearch.index.engine.InternalEngine; @@ -137,7 +138,7 @@ public RemoteStoreRefreshListener( this.segmentTracker = segmentTracker; resetBackOffDelayIterator(); this.checkpointPublisher = checkpointPublisher; - this.fileUploader = new FileUploader(new UploadTracker() { + this.fileUploader = new FileUploader(new UploadListener() { @Override public void beforeUpload(String file) { // Start tracking the upload bytes started 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 f69b09e630c1d..891ed58f1ac3b 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -25,6 +25,7 @@ import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.util.ByteUtils; import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.common.util.UploadListener; import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; 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 99bfcfddc0b6a..0f700314514bd 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -74,7 +74,7 @@ public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { private RemoteStoreMetadataLockManager mdLockManager; private RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; - private TestUploadTracker testUploadTracker; + private TestUploadListener testUploadListener; private IndexShard indexShard; private SegmentInfos segmentInfos; private ThreadPool threadPool; @@ -96,7 +96,7 @@ public void setup() throws IOException { mdLockManager, threadPool ); - testUploadTracker = new TestUploadTracker(); + testUploadListener = new TestUploadListener(); Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); @@ -532,10 +532,10 @@ public void testCopyFilesFromMultipart() throws Exception { uploadResponseCompletableFuture.complete(null); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); - remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker); + remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener); assertTrue(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadTracker.UploadStatus.UPLOAD_SUCCESS, testUploadTracker.getUploadStatus(filename)); + assertEquals(TestUploadListener.UploadStatus.UPLOAD_SUCCESS, testUploadListener.getUploadStatus(filename)); storeDirectory.close(); } @@ -563,11 +563,11 @@ public void testCopyFilesFromMultipartIOException() throws Exception { assertThrows( IOException.class, - () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker) + () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener) ); assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadTracker.UploadStatus.UPLOAD_FAILURE, testUploadTracker.getUploadStatus(filename)); + assertEquals(TestUploadListener.UploadStatus.UPLOAD_FAILURE, testUploadListener.getUploadStatus(filename)); storeDirectory.close(); } @@ -595,11 +595,11 @@ public void testCopyFilesFromMultipartUploadFutureCompletedExceptionally() throw assertThrows( ExecutionException.class, - () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadTracker) + () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener) ); assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadTracker.UploadStatus.UPLOAD_FAILURE, testUploadTracker.getUploadStatus(filename)); + assertEquals(TestUploadListener.UploadStatus.UPLOAD_FAILURE, testUploadListener.getUploadStatus(filename)); storeDirectory.close(); } diff --git a/server/src/test/java/org/opensearch/index/store/TestUploadTracker.java b/server/src/test/java/org/opensearch/index/store/TestUploadListener.java similarity index 89% rename from server/src/test/java/org/opensearch/index/store/TestUploadTracker.java rename to server/src/test/java/org/opensearch/index/store/TestUploadListener.java index 05250d69b7867..a2a61a93371e8 100644 --- a/server/src/test/java/org/opensearch/index/store/TestUploadTracker.java +++ b/server/src/test/java/org/opensearch/index/store/TestUploadListener.java @@ -8,11 +8,11 @@ package org.opensearch.index.store; -import org.opensearch.index.shard.UploadTracker; +import org.opensearch.common.util.UploadListener; import java.util.concurrent.ConcurrentHashMap; -public class TestUploadTracker implements UploadTracker { +public class TestUploadListener implements UploadListener { private final ConcurrentHashMap uploadStatusMap = new ConcurrentHashMap<>(); From a1b78b13533f2f9bb597f19731188d67e95e2539 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Fri, 9 Jun 2023 14:27:20 +0530 Subject: [PATCH 03/18] Removed areStreamsDecorated check from RemoteTransferContainer Signed-off-by: Raghuvansh Raj --- .../index/translog/transfer/BlobStoreTransferService.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 82ee1cb9cb319..c5b6a297624ec 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -144,8 +144,7 @@ private CompletableFuture createUploadFuture( writePriority, (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), expectedChecksum, - blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported(), - false + blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported() ); WriteContext writeContext = remoteTransferContainer.createWriteContext(); CompletableFuture uploadFuture = blobStore.blobContainer(blobPath).writeBlobByStreams(writeContext); From a564200b64600cfdaec16326dc8858d5edb65ee2 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Fri, 9 Jun 2023 14:46:35 +0530 Subject: [PATCH 04/18] Moved TranslogCheckedContainer to org.opensearch.index.translog Signed-off-by: Raghuvansh Raj --- .../store/exception/ChecksumCombinationException.java | 2 +- .../{checked => }/TranslogCheckedContainer.java | 2 +- .../index/translog/checked/package-info.java | 10 ---------- .../translog/transfer/BlobStoreTransferService.java | 2 +- 4 files changed, 3 insertions(+), 13 deletions(-) rename server/src/main/java/org/opensearch/index/translog/{checked => }/TranslogCheckedContainer.java (98%) delete mode 100644 server/src/main/java/org/opensearch/index/translog/checked/package-info.java diff --git a/server/src/main/java/org/opensearch/index/store/exception/ChecksumCombinationException.java b/server/src/main/java/org/opensearch/index/store/exception/ChecksumCombinationException.java index a355473aa2afd..d8e1739fbaa9d 100644 --- a/server/src/main/java/org/opensearch/index/store/exception/ChecksumCombinationException.java +++ b/server/src/main/java/org/opensearch/index/store/exception/ChecksumCombinationException.java @@ -11,7 +11,7 @@ import org.apache.lucene.index.CorruptIndexException; /** - * Exception is raised when combination to two crc checksums fail. + * Exception is raised when combination of two CRC checksums fail. * * @opensearch.internal */ diff --git a/server/src/main/java/org/opensearch/index/translog/checked/TranslogCheckedContainer.java b/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java similarity index 98% rename from server/src/main/java/org/opensearch/index/translog/checked/TranslogCheckedContainer.java rename to server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java index b90794e29d2b1..aac708f74da62 100644 --- a/server/src/main/java/org/opensearch/index/translog/checked/TranslogCheckedContainer.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.index.translog.checked; +package org.opensearch.index.translog; import org.opensearch.common.io.Channels; import org.opensearch.common.util.concurrent.ReleasableLock; diff --git a/server/src/main/java/org/opensearch/index/translog/checked/package-info.java b/server/src/main/java/org/opensearch/index/translog/checked/package-info.java deleted file mode 100644 index ddb235fdbedce..0000000000000 --- a/server/src/main/java/org/opensearch/index/translog/checked/package-info.java +++ /dev/null @@ -1,10 +0,0 @@ -/* - * 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. - */ - -/** Contains checksum related utilities for translog files */ -package org.opensearch.index.translog.checked; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index c5b6a297624ec..4b85f15c7fe33 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -21,7 +21,7 @@ import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeFileInputStream; import org.opensearch.index.translog.ChannelFactory; -import org.opensearch.index.translog.checked.TranslogCheckedContainer; +import org.opensearch.index.translog.TranslogCheckedContainer; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.threadpool.ThreadPool; From 7c1aad70013f98240be06b723074c9169b5b507f Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Fri, 9 Jun 2023 18:52:16 +0530 Subject: [PATCH 05/18] Modified translog checksum calculation to update on every translog write Signed-off-by: Raghuvansh Raj --- .../opensearch/index/translog/Checkpoint.java | 2 +- .../translog/TranslogCheckedContainer.java | 15 ++------ .../index/translog/TranslogHeader.java | 12 ++++-- .../index/translog/TranslogReader.java | 37 +++++++++++++++++-- .../index/translog/TranslogWriter.java | 25 ++++++++++++- .../transfer/BlobStoreTransferService.java | 12 +----- .../index/translog/transfer/FileSnapshot.java | 17 ++++++--- .../TranslogCheckpointTransferSnapshot.java | 10 ++++- ...oreTransferServiceMockRepositoryTests.java | 18 +++++++-- .../BlobStoreTransferServiceTests.java | 12 +++++- .../translog/transfer/FileSnapshotTests.java | 6 +-- .../transfer/FileTransferTrackerTests.java | 12 ++++-- .../TranslogTransferManagerTests.java | 12 ++++-- 13 files changed, 135 insertions(+), 55 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/translog/Checkpoint.java b/server/src/main/java/org/opensearch/index/translog/Checkpoint.java index 56de7e5daf55f..a9f905f52bc3a 100644 --- a/server/src/main/java/org/opensearch/index/translog/Checkpoint.java +++ b/server/src/main/java/org/opensearch/index/translog/Checkpoint.java @@ -233,7 +233,7 @@ public static void write(FileChannel fileChannel, Path checkpointFile, Checkpoin } } - private static byte[] createCheckpointBytes(Path checkpointFile, Checkpoint checkpoint) throws IOException { + public static byte[] createCheckpointBytes(Path checkpointFile, Checkpoint checkpoint) throws IOException { final ByteArrayOutputStream byteOutputStream = new ByteArrayOutputStream(V4_FILE_SIZE) { @Override public synchronized byte[] toByteArray() { diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java b/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java index aac708f74da62..7e2a38559166f 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogCheckedContainer.java @@ -8,11 +8,8 @@ package org.opensearch.index.translog; -import org.opensearch.common.io.Channels; import org.opensearch.common.util.concurrent.ReleasableLock; -import java.io.IOException; -import java.nio.channels.FileChannel; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; import java.util.zip.CRC32; @@ -28,21 +25,15 @@ public class TranslogCheckedContainer { private final Checksum checksum; private final AtomicLong contentLength; private final ReleasableLock updateLock = new ReleasableLock(new ReentrantLock()); - private final String file; /** - * Creates TranslogCheckedContainer from provided channel. + * Create TranslogCheckedContainer from provided bytes * - * @param channel {@link FileChannel} to read from - * @param offset offset of channel from which bytes are to be read. - * @param len Length of bytes to be read. + * @param bytes The byte array to read from */ - public TranslogCheckedContainer(FileChannel channel, int offset, int len, String file) throws IOException { + public TranslogCheckedContainer(byte[] bytes) { this.checksum = new CRC32(); this.contentLength = new AtomicLong(); - this.file = file; - - byte[] bytes = Channels.readFromFileChannel(channel, offset, len); updateFromBytes(bytes, 0, bytes.length); } diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java index af6ebcf7b7c66..8067cccb772a2 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java @@ -46,6 +46,7 @@ import java.io.EOFException; import java.io.IOException; +import java.io.OutputStream; import java.nio.channels.FileChannel; import java.nio.file.Path; @@ -213,12 +214,10 @@ private static void tryReportOldVersionError(final Path path, final FileChannel /** * Writes this header with the latest format into the file channel */ - void write(final FileChannel channel, boolean fsync) throws IOException { + void write(final OutputStream outputStream) throws IOException { // This output is intentionally not closed because closing it will close the FileChannel. @SuppressWarnings({ "IOResourceOpenedButNotSafelyClosed", "resource" }) - final BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput( - new OutputStreamStreamOutput(java.nio.channels.Channels.newOutputStream(channel)) - ); + final BufferedChecksumStreamOutput out = new BufferedChecksumStreamOutput(new OutputStreamStreamOutput(outputStream)); CodecUtil.writeHeader(new OutputStreamDataOutput(out), TRANSLOG_CODEC, CURRENT_VERSION); // Write uuid final BytesRef uuid = new BytesRef(translogUUID); @@ -229,6 +228,11 @@ void write(final FileChannel channel, boolean fsync) throws IOException { // Checksum header out.writeInt((int) out.getChecksum()); out.flush(); + } + + void write(final FileChannel channel, boolean fsync) throws IOException { + OutputStream outputStream = java.nio.channels.Channels.newOutputStream(channel); + write(outputStream); if (fsync == true) { channel.force(true); } diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogReader.java b/server/src/main/java/org/opensearch/index/translog/TranslogReader.java index c4a4fb7a460a0..9ea3328587645 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogReader.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogReader.java @@ -33,6 +33,7 @@ package org.opensearch.index.translog; import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.common.Nullable; import org.opensearch.common.io.Channels; import org.opensearch.common.util.io.IOUtils; import org.opensearch.index.seqno.SequenceNumbers; @@ -59,6 +60,11 @@ public class TranslogReader extends BaseTranslogReader implements Closeable { private final Checkpoint checkpoint; protected final AtomicBoolean closed = new AtomicBoolean(false); + @Nullable + private final Long translogChecksum; + @Nullable + private final Long checkpointChecksum; + /** * Create a translog writer against the specified translog file channel. * @@ -67,11 +73,34 @@ public class TranslogReader extends BaseTranslogReader implements Closeable { * @param path the path to the translog * @param header the header of the translog file */ - TranslogReader(final Checkpoint checkpoint, final FileChannel channel, final Path path, final TranslogHeader header) { + TranslogReader( + final Checkpoint checkpoint, + final FileChannel channel, + final Path path, + final TranslogHeader header, + final Long translogChecksum + ) throws IOException { super(checkpoint.generation, channel, path, header); this.length = checkpoint.offset; this.totalOperations = checkpoint.numOps; this.checkpoint = checkpoint; + this.translogChecksum = translogChecksum; + this.checkpointChecksum = (translogChecksum != null) ? calculateCheckpointChecksum(checkpoint, path) : null; + } + + private static Long calculateCheckpointChecksum(Checkpoint checkpoint, Path path) throws IOException { + TranslogCheckedContainer checkpointCheckedContainer = new TranslogCheckedContainer( + Checkpoint.createCheckpointBytes(path.getParent().resolve(Translog.CHECKPOINT_FILE_NAME), checkpoint) + ); + return checkpointCheckedContainer.getChecksum(); + } + + public Long getTranslogChecksum() { + return translogChecksum; + } + + public Long getCheckpointChecksum() { + return checkpointChecksum; } /** @@ -87,7 +116,7 @@ public class TranslogReader extends BaseTranslogReader implements Closeable { public static TranslogReader open(final FileChannel channel, final Path path, final Checkpoint checkpoint, final String translogUUID) throws IOException { final TranslogHeader header = TranslogHeader.read(translogUUID, path, channel); - return new TranslogReader(checkpoint, channel, path, header); + return new TranslogReader(checkpoint, channel, path, header, null); } /** @@ -115,9 +144,9 @@ TranslogReader closeIntoTrimmedReader(long aboveSeqNo, ChannelFactory channelFac IOUtils.fsync(checkpointFile.getParent(), true); - newReader = new TranslogReader(newCheckpoint, channel, path, header); + newReader = new TranslogReader(newCheckpoint, channel, path, header, translogChecksum); } else { - newReader = new TranslogReader(checkpoint, channel, path, header); + newReader = new TranslogReader(checkpoint, channel, path, header, translogChecksum); } toCloseOnFailure = null; return newReader; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java index e19aece60adc0..a4b5d486e5913 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -37,6 +37,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.opensearch.common.Nullable; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; @@ -54,6 +55,7 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.ShardId; +import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; @@ -110,6 +112,9 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable { private final Map> seenSequenceNumbers; + @Nullable + private final TranslogCheckedContainer translogCheckedContainer; + private final Boolean remoteTranslogEnabled; private TranslogWriter( @@ -126,6 +131,7 @@ private TranslogWriter( final TragicExceptionHolder tragedy, final LongConsumer persistedSequenceNumberConsumer, final BigArrays bigArrays, + TranslogCheckedContainer translogCheckedContainer, Boolean remoteTranslogEnabled ) throws IOException { super(initialCheckpoint.generation, channel, path, header); @@ -151,6 +157,7 @@ private TranslogWriter( this.bigArrays = bigArrays; this.seenSequenceNumbers = Assertions.ENABLED ? new HashMap<>() : null; this.tragedy = tragedy; + this.translogCheckedContainer = translogCheckedContainer; this.remoteTranslogEnabled = remoteTranslogEnabled; } @@ -179,6 +186,12 @@ public static TranslogWriter create( checkpointChannel = channelFactory.open(checkpointFile, StandardOpenOption.WRITE); final TranslogHeader header = new TranslogHeader(translogUUID, primaryTerm); header.write(channel, !Boolean.TRUE.equals(remoteTranslogEnabled)); + TranslogCheckedContainer translogCheckedContainer = null; + if (remoteTranslogEnabled == Boolean.TRUE) { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + header.write(byteArrayOutputStream); + translogCheckedContainer = new TranslogCheckedContainer(byteArrayOutputStream.toByteArray()); + } final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint( header.sizeInBytes(), fileGeneration, @@ -214,6 +227,7 @@ public static TranslogWriter create( tragedy, persistedSequenceNumberConsumer, bigArrays, + translogCheckedContainer, remoteTranslogEnabled ); } catch (Exception exception) { @@ -438,7 +452,13 @@ public TranslogReader closeIntoReader() throws IOException { closeWithTragicEvent(ex); throw ex; } - return new TranslogReader(getLastSyncedCheckpoint(), channel, path, header); + return new TranslogReader( + getLastSyncedCheckpoint(), + channel, + path, + header, + (translogCheckedContainer != null) ? translogCheckedContainer.getChecksum() : null + ); } else { throw new AlreadyClosedException( "translog [" + getGeneration() + "] is already closed (path [" + path + "]", @@ -571,6 +591,9 @@ private void writeAndReleaseOps(ReleasableBytesReference toWrite) throws IOExcep while (currentBytesConsumed != current.length) { int nBytesToWrite = Math.min(current.length - currentBytesConsumed, ioBuffer.remaining()); ioBuffer.put(current.bytes, current.offset + currentBytesConsumed, nBytesToWrite); + if (translogCheckedContainer != null) { + translogCheckedContainer.updateFromBytes(current.bytes, current.offset + currentBytesConsumed, nBytesToWrite); + } currentBytesConsumed += nBytesToWrite; if (ioBuffer.hasRemaining() == false) { ioBuffer.flip(); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 4b85f15c7fe33..15b110f39eff0 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -21,7 +21,6 @@ import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeFileInputStream; import org.opensearch.index.translog.ChannelFactory; -import org.opensearch.index.translog.TranslogCheckedContainer; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import org.opensearch.threadpool.ThreadPool; @@ -32,6 +31,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -125,16 +125,8 @@ private CompletableFuture createUploadFuture( try { ChannelFactory channelFactory = FileChannel::open; long contentLength; - long expectedChecksum; try (FileChannel channel = channelFactory.open(fileSnapshot.getPath(), StandardOpenOption.READ)) { contentLength = channel.size(); - TranslogCheckedContainer translogCheckedContainer = new TranslogCheckedContainer( - channel, - 0, - (int) contentLength, - fileSnapshot.getName() - ); - expectedChecksum = translogCheckedContainer.getChecksum(); } RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( fileSnapshot.getName(), @@ -143,7 +135,7 @@ private CompletableFuture createUploadFuture( true, writePriority, (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), - expectedChecksum, + Objects.requireNonNull(fileSnapshot.getChecksum()), blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported() ); WriteContext writeContext = remoteTransferContainer.createWriteContext(); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java index 239ef7c3c9300..dcec94edd694f 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileSnapshot.java @@ -107,10 +107,12 @@ public void close() throws IOException { public static class TransferFileSnapshot extends FileSnapshot { private final long primaryTerm; + private Long checksum; - public TransferFileSnapshot(Path path, long primaryTerm) throws IOException { + public TransferFileSnapshot(Path path, long primaryTerm, Long checksum) throws IOException { super(path); this.primaryTerm = primaryTerm; + this.checksum = checksum; } public TransferFileSnapshot(String name, byte[] content, long primaryTerm) throws IOException { @@ -118,6 +120,10 @@ public TransferFileSnapshot(String name, byte[] content, long primaryTerm) throw this.primaryTerm = primaryTerm; } + public Long getChecksum() { + return checksum; + } + public long getPrimaryTerm() { return primaryTerm; } @@ -148,8 +154,8 @@ public static final class TranslogFileSnapshot extends TransferFileSnapshot { private final long generation; - public TranslogFileSnapshot(long primaryTerm, long generation, Path path) throws IOException { - super(path, primaryTerm); + public TranslogFileSnapshot(long primaryTerm, long generation, Path path, Long checksum) throws IOException { + super(path, primaryTerm, checksum); this.generation = generation; } @@ -185,8 +191,9 @@ public static final class CheckpointFileSnapshot extends TransferFileSnapshot { private final long minTranslogGeneration; - public CheckpointFileSnapshot(long primaryTerm, long generation, long minTranslogGeneration, Path path) throws IOException { - super(path, primaryTerm); + public CheckpointFileSnapshot(long primaryTerm, long generation, long minTranslogGeneration, Path path, Long checksum) + throws IOException { + super(path, primaryTerm, checksum); this.minTranslogGeneration = minTranslogGeneration; this.generation = generation; } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java index b34c2282e874f..10dec13c81e1a 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogCheckpointTransferSnapshot.java @@ -145,8 +145,14 @@ public TranslogCheckpointTransferSnapshot build() throws IOException { Path checkpointPath = location.resolve(checkpointGenFileNameMapper.apply(readerGeneration)); generations.add(readerGeneration); translogTransferSnapshot.add( - new TranslogFileSnapshot(readerPrimaryTerm, readerGeneration, translogPath), - new CheckpointFileSnapshot(readerPrimaryTerm, checkpointGeneration, minTranslogGeneration, checkpointPath) + new TranslogFileSnapshot(readerPrimaryTerm, readerGeneration, translogPath, reader.getTranslogChecksum()), + new CheckpointFileSnapshot( + readerPrimaryTerm, + checkpointGeneration, + minTranslogGeneration, + checkpointPath, + reader.getCheckpointChecksum() + ) ); if (readerGeneration > highestGeneration) { highestGeneration = readerGeneration; diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java index 3020703cd398f..65a4df541409c 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -53,7 +53,11 @@ public void setUp() throws Exception { public void testUploadBlobs() throws Exception { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); - FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong(), + 0L + ); BlobContainer blobContainer = mock(BlobContainer.class); when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); @@ -95,7 +99,11 @@ public void onFailure(Exception e) { public void testUploadBlobsIOException() throws Exception { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); - FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong(), + 0L + ); BlobContainer blobContainer = mock(BlobContainer.class); when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); @@ -131,7 +139,11 @@ public void onFailure(Exception e) { public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); - FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong(), + 0L + ); BlobContainer blobContainer = mock(BlobContainer.class); when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index cc00c49892853..684a5a2152925 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -50,7 +50,11 @@ public void setUp() throws Exception { public void testUploadBlob() throws IOException { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); - FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong(), + null + ); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); transferService.uploadBlob(transferFileSnapshot, repository.basePath(), WritePriority.HIGH); } @@ -69,7 +73,11 @@ public void testUploadBlobAsync() throws IOException, InterruptedException { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); AtomicBoolean succeeded = new AtomicBoolean(false); - FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot(testFile, randomNonNegativeLong()); + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong(), + null + ); CountDownLatch latch = new CountDownLatch(1); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); transferService.uploadBlobByThreadPool( diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/FileSnapshotTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/FileSnapshotTests.java index 6d2fb3794b107..8d07af5927135 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/FileSnapshotTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/FileSnapshotTests.java @@ -28,15 +28,15 @@ public void tearDown() throws Exception { public void testFileSnapshotPath() throws IOException { Path file = createTempFile(); Files.writeString(file, "hello"); - fileSnapshot = new FileSnapshot.TransferFileSnapshot(file, 12); + fileSnapshot = new FileSnapshot.TransferFileSnapshot(file, 12, null); assertFileSnapshotProperties(file); - try (FileSnapshot sameFileSnapshot = new FileSnapshot.TransferFileSnapshot(file, 12)) { + try (FileSnapshot sameFileSnapshot = new FileSnapshot.TransferFileSnapshot(file, 12, null)) { assertEquals(sameFileSnapshot, fileSnapshot); } - try (FileSnapshot sameFileDiffPTSnapshot = new FileSnapshot.TransferFileSnapshot(file, 34)) { + try (FileSnapshot sameFileDiffPTSnapshot = new FileSnapshot.TransferFileSnapshot(file, 34, null)) { assertNotEquals(sameFileDiffPTSnapshot, fileSnapshot); } } diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java index be14e4a7bd380..fd0d44564ef6b 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java @@ -34,7 +34,8 @@ public void testOnSuccess() throws IOException { try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( testFile, - randomNonNegativeLong() + randomNonNegativeLong(), + null ) ) { fileTransferTracker.onSuccess(transferFileSnapshot); @@ -58,11 +59,13 @@ public void testOnFailure() throws IOException { try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( testFile, - randomNonNegativeLong() + randomNonNegativeLong(), + null ); FileSnapshot.TransferFileSnapshot transferFileSnapshot2 = new FileSnapshot.TransferFileSnapshot( testFile2, - randomNonNegativeLong() + randomNonNegativeLong(), + null ) ) { @@ -82,7 +85,8 @@ public void testUploaded() throws IOException { try ( FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( testFile, - randomNonNegativeLong() + randomNonNegativeLong(), + null ); ) { 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 0928c9bf15344..66cd257299e25 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 @@ -152,13 +152,15 @@ public Set getCheckpointFileSnapshots() { primaryTerm, generation, minTranslogGeneration, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.CHECKPOINT_SUFFIX) + createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.CHECKPOINT_SUFFIX), + null ), new CheckpointFileSnapshot( primaryTerm, generation, minTranslogGeneration, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.CHECKPOINT_SUFFIX) + createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.CHECKPOINT_SUFFIX), + null ) ); } catch (IOException e) { @@ -173,12 +175,14 @@ public Set getTranslogFileSnapshots() { new TranslogFileSnapshot( primaryTerm, generation, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.TRANSLOG_FILE_SUFFIX) + createTempFile(Translog.TRANSLOG_FILE_PREFIX + generation, Translog.TRANSLOG_FILE_SUFFIX), + null ), new TranslogFileSnapshot( primaryTerm, generation - 1, - createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.TRANSLOG_FILE_SUFFIX) + createTempFile(Translog.TRANSLOG_FILE_PREFIX + (generation - 1), Translog.TRANSLOG_FILE_SUFFIX), + null ) ); } catch (IOException e) { From 5afd572bd82037917ce9a7761edf0a5dd6157d01 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Thu, 15 Jun 2023 17:21:54 +0530 Subject: [PATCH 06/18] Addressing PR comments Signed-off-by: Raghuvansh Raj --- .../java/org/opensearch/threadpool/ThreadPool.java | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index 21659fea90c2f..d9f73a9b41658 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -62,10 +62,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.Callable; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ScheduledExecutorService; @@ -453,16 +451,6 @@ public ScheduledCancellable schedule(Runnable command, TimeValue delay, String e return new ScheduledCancellableAdapter(scheduler.schedule(command, delay.millis(), TimeUnit.MILLISECONDS)); } - public Future executeCallable(Callable command, String executorName) { - ExecutorService executorService = executor(executorName); - return executorService.submit(command); - } - - public boolean isExecutorShutDown(String executorName) { - ExecutorService executorService = executor(executorName); - return executorService.isShutdown(); - } - public void scheduleUnlessShuttingDown(TimeValue delay, String executor, Runnable command) { try { schedule(command, delay, executor); From 53ff2b676e88985afcf1c0a95b4f103caea19f3e Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Fri, 16 Jun 2023 13:50:54 +0530 Subject: [PATCH 07/18] Adding MultiStreamBlobContainer for multipart upload check Signed-off-by: Raghuvansh Raj --- .../multipart/mocks/MockFsBlobContainer.java | 8 ++--- .../common/blobstore/BlobContainer.java | 23 ++----------- .../blobstore/MultiStreamBlobContainer.java | 34 +++++++++++++++++++ .../store/RemoteSegmentStoreDirectory.java | 1 - .../transfer/BlobStoreTransferService.java | 7 ++-- .../RemoteSegmentStoreDirectoryTests.java | 11 +++--- ...oreTransferServiceMockRepositoryTests.java | 11 +++--- 7 files changed, 51 insertions(+), 44 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java index a56f0a338d296..f664a190dc777 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java @@ -9,6 +9,7 @@ package org.opensearch.remotestore.multipart.mocks; import org.apache.lucene.index.CorruptIndexException; +import org.opensearch.common.blobstore.MultiStreamBlobContainer; import org.opensearch.common.io.InputStreamContainer; import org.opensearch.common.StreamContext; import org.opensearch.common.blobstore.BlobPath; @@ -27,7 +28,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -public class MockFsBlobContainer extends FsBlobContainer { +public class MockFsBlobContainer extends FsBlobContainer implements MultiStreamBlobContainer { private static final int TRANSFER_TIMEOUT_MILLIS = 30000; @@ -38,11 +39,6 @@ public MockFsBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; } - @Override - public boolean isMultiStreamUploadSupported() { - return true; - } - @Override public CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException { CompletableFuture completableFuture = new CompletableFuture<>(); diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index cfc5238231d8e..a3350aace32a3 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -44,7 +44,6 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.concurrent.CompletableFuture; /** * An interface for managing a repository of blob entries, where each blob entry is just a named group of bytes. @@ -132,32 +131,14 @@ default long readBlobPreferredLength() { void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException; /** - * Used to check whether vendor plugin support for parallel upload of multiple streams is enabled or not. - * Returns false by default + * Needs to be set to true by the implementation if remote data integrity check is supported * - * @return If multi-stream parallel uploads are supported + * @return true if remote data integrity check is supported */ - default boolean isMultiStreamUploadSupported() { - return false; - } - default boolean isRemoteDataIntegritySupported() { return false; } - /** - * Reads blob content from multiple streams, each from a specific part of the file, which is provided by the - * StreamContextSupplier in the WriteContext passed to this method. An {@link IOException} is thrown if reading - * any of the input streams fails, or writing to the target blob fails - * - * @param writeContext A WriteContext object encapsulating all information needed to perform the upload - * @return A {@link CompletableFuture} representing the upload - * @throws IOException if any of the input streams could not be read, or the target blob could not be written to - */ - default CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException { - throw new UnsupportedOperationException(); - } - /** * Reads blob content from the input stream and writes it to the container in a new blob with the given name, * using an atomic write operation if the implementation supports it. diff --git a/server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java new file mode 100644 index 0000000000000..a5a6d256bcf77 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java @@ -0,0 +1,34 @@ +/* + * 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.common.blobstore; + +import org.opensearch.common.blobstore.stream.write.WriteContext; + +import java.io.IOException; +import java.util.concurrent.CompletableFuture; + +/** + * An extension of {@link BlobContainer} that adds {@link MultiStreamBlobContainer#writeBlobByStreams} to allow + * multipart uploads + * + * @opensearch.internal + */ +public interface MultiStreamBlobContainer extends BlobContainer { + + /** + * Reads blob content from multiple streams, each from a specific part of the file, which is provided by the + * StreamContextSupplier in the WriteContext passed to this method. An {@link IOException} is thrown if reading + * any of the input streams fails, or writing to the target blob fails + * + * @param writeContext A WriteContext object encapsulating all information needed to perform the upload + * @return A {@link CompletableFuture} representing the upload + * @throws IOException if any of the input streams could not be read, or the target blob could not be written to + */ + CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException; +} 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 891ed58f1ac3b..f69b09e630c1d 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -25,7 +25,6 @@ import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.util.ByteUtils; import org.opensearch.index.remote.RemoteStoreUtils; -import org.opensearch.common.util.UploadListener; import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 15b110f39eff0..60969480861fa 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -16,6 +16,7 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.MultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; @@ -94,7 +95,7 @@ public void uploadBlobs( List> resultFutures = new ArrayList<>(); fileSnapshots.forEach(fileSnapshot -> { BlobPath blobPath = blobPaths.get(fileSnapshot.getPrimaryTerm()); - if (!blobStore.blobContainer(blobPath).isMultiStreamUploadSupported()) { + if (!(blobStore.blobContainer(blobPath) instanceof MultiStreamBlobContainer)) { uploadBlobByThreadPool(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); } else { CompletableFuture resultFuture = createUploadFuture(fileSnapshot, listener, blobPath, writePriority); @@ -139,7 +140,9 @@ private CompletableFuture createUploadFuture( blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported() ); WriteContext writeContext = remoteTransferContainer.createWriteContext(); - CompletableFuture uploadFuture = blobStore.blobContainer(blobPath).writeBlobByStreams(writeContext); + CompletableFuture uploadFuture = ((MultiStreamBlobContainer) blobStore.blobContainer(blobPath)).writeBlobByStreams( + writeContext + ); resultFuture = uploadFuture.whenComplete((resp, throwable) -> { try { remoteTransferContainer.close(); 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 0f700314514bd..ba20a675891e3 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -25,7 +25,7 @@ import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.UUIDs; -import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.MultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.VersionedCodecStreamWrapper; @@ -525,9 +525,8 @@ public void testCopyFilesFromMultipart() throws Exception { assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - BlobContainer blobContainer = mock(BlobContainer.class); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); @@ -554,9 +553,8 @@ public void testCopyFilesFromMultipartIOException() throws Exception { assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - BlobContainer blobContainer = mock(BlobContainer.class); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenThrow(new IOException()); @@ -586,9 +584,8 @@ public void testCopyFilesFromMultipartUploadFutureCompletedExceptionally() throw assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - BlobContainer blobContainer = mock(BlobContainer.class); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.completeExceptionally(new IOException()); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java index 65a4df541409c..a144691e4bba8 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -10,9 +10,9 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.LatchedActionListener; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.MultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.test.OpenSearchTestCase; @@ -59,8 +59,7 @@ public void testUploadBlobs() throws Exception { 0L ); - BlobContainer blobContainer = mock(BlobContainer.class); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); @@ -105,8 +104,7 @@ public void testUploadBlobsIOException() throws Exception { 0L ); - BlobContainer blobContainer = mock(BlobContainer.class); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); doThrow(new IOException()).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); @@ -145,8 +143,7 @@ public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception 0L ); - BlobContainer blobContainer = mock(BlobContainer.class); - when(blobContainer.isMultiStreamUploadSupported()).thenReturn(true); + MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.completeExceptionally(new IOException()); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); From 3004c7e257d12ea610e7d6d287476c2741a1e884 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Thu, 22 Jun 2023 11:42:02 +0530 Subject: [PATCH 08/18] Renaming MultiStreamBlobContainer to VerifyingMultiStreamBlobContainer Signed-off-by: Raghuvansh Raj --- ...bContainer.java => MockFsBlobContainerVerifying.java} | 6 +++--- .../remotestore/multipart/mocks/MockFsBlobStore.java | 2 +- ...ainer.java => VerifyingMultiStreamBlobContainer.java} | 4 ++-- .../index/store/RemoteSegmentStoreDirectory.java | 6 ++++++ .../translog/transfer/BlobStoreTransferService.java | 9 ++++----- .../index/store/RemoteSegmentStoreDirectoryTests.java | 8 ++++---- .../BlobStoreTransferServiceMockRepositoryTests.java | 8 ++++---- 7 files changed, 24 insertions(+), 19 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/{MockFsBlobContainer.java => MockFsBlobContainerVerifying.java} (94%) rename server/src/main/java/org/opensearch/common/blobstore/{MultiStreamBlobContainer.java => VerifyingMultiStreamBlobContainer.java} (85%) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java similarity index 94% rename from server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java rename to server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java index f664a190dc777..189c313792425 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainer.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java @@ -9,7 +9,7 @@ package org.opensearch.remotestore.multipart.mocks; import org.apache.lucene.index.CorruptIndexException; -import org.opensearch.common.blobstore.MultiStreamBlobContainer; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.io.InputStreamContainer; import org.opensearch.common.StreamContext; import org.opensearch.common.blobstore.BlobPath; @@ -28,13 +28,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -public class MockFsBlobContainer extends FsBlobContainer implements MultiStreamBlobContainer { +public class MockFsBlobContainerVerifying extends FsBlobContainer implements VerifyingMultiStreamBlobContainer { private static final int TRANSFER_TIMEOUT_MILLIS = 30000; private final boolean triggerDataIntegrityFailure; - public MockFsBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { + public MockFsBlobContainerVerifying(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { super(blobStore, blobPath, path); this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java index 435be3f8e66f2..ac1c486663531 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java @@ -28,7 +28,7 @@ public MockFsBlobStore(int bufferSizeInBytes, Path path, boolean readonly, boole @Override public BlobContainer blobContainer(BlobPath path) { try { - return new MockFsBlobContainer(this, path, buildAndCreate(path), triggerDataIntegrityFailure); + return new MockFsBlobContainerVerifying(this, path, buildAndCreate(path), triggerDataIntegrityFailure); } catch (IOException ex) { throw new OpenSearchException("failed to create blob container", ex); } diff --git a/server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java similarity index 85% rename from server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java rename to server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index a5a6d256bcf77..a244ea5841e2a 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/MultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -14,12 +14,12 @@ import java.util.concurrent.CompletableFuture; /** - * An extension of {@link BlobContainer} that adds {@link MultiStreamBlobContainer#writeBlobByStreams} to allow + * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#writeBlobByStreams} to allow * multipart uploads * * @opensearch.internal */ -public interface MultiStreamBlobContainer extends BlobContainer { +public interface VerifyingMultiStreamBlobContainer extends BlobContainer { /** * Reads blob content from multiple streams, each from a specific part of the file, which is provided by the 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 f69b09e630c1d..bcaf1c9743cf0 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -21,6 +21,12 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.opensearch.common.UUIDs; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; +import org.opensearch.common.blobstore.exception.CorruptFileException; +import org.opensearch.common.blobstore.stream.write.WriteContext; +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.io.VersionedCodecStreamWrapper; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.util.ByteUtils; diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 60969480861fa..a09fd282a1835 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -16,7 +16,7 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.blobstore.MultiStreamBlobContainer; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; @@ -95,7 +95,7 @@ public void uploadBlobs( List> resultFutures = new ArrayList<>(); fileSnapshots.forEach(fileSnapshot -> { BlobPath blobPath = blobPaths.get(fileSnapshot.getPrimaryTerm()); - if (!(blobStore.blobContainer(blobPath) instanceof MultiStreamBlobContainer)) { + if (!(blobStore.blobContainer(blobPath) instanceof VerifyingMultiStreamBlobContainer)) { uploadBlobByThreadPool(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); } else { CompletableFuture resultFuture = createUploadFuture(fileSnapshot, listener, blobPath, writePriority); @@ -140,9 +140,8 @@ private CompletableFuture createUploadFuture( blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported() ); WriteContext writeContext = remoteTransferContainer.createWriteContext(); - CompletableFuture uploadFuture = ((MultiStreamBlobContainer) blobStore.blobContainer(blobPath)).writeBlobByStreams( - writeContext - ); + CompletableFuture uploadFuture = ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)) + .writeBlobByStreams(writeContext); resultFuture = uploadFuture.whenComplete((resp, throwable) -> { try { remoteTransferContainer.close(); 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 ba20a675891e3..4d9a6bbe7f0d1 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -25,7 +25,7 @@ import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.UUIDs; -import org.opensearch.common.blobstore.MultiStreamBlobContainer; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.VersionedCodecStreamWrapper; @@ -525,7 +525,7 @@ public void testCopyFilesFromMultipart() throws Exception { assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); @@ -553,7 +553,7 @@ public void testCopyFilesFromMultipartIOException() throws Exception { assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); @@ -584,7 +584,7 @@ public void testCopyFilesFromMultipartUploadFutureCompletedExceptionally() throw assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.completeExceptionally(new IOException()); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java index a144691e4bba8..60a2b16048c44 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -12,7 +12,7 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; -import org.opensearch.common.blobstore.MultiStreamBlobContainer; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.test.OpenSearchTestCase; @@ -59,7 +59,7 @@ public void testUploadBlobs() throws Exception { 0L ); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.complete(null); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); @@ -104,7 +104,7 @@ public void testUploadBlobsIOException() throws Exception { 0L ); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); doThrow(new IOException()).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); @@ -143,7 +143,7 @@ public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception 0L ); - MultiStreamBlobContainer blobContainer = mock(MultiStreamBlobContainer.class); + VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); uploadResponseCompletableFuture.completeExceptionally(new IOException()); when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); From 303ab59ffd24d5f3c930160e3e0ec84060a1c97c Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Tue, 27 Jun 2023 14:55:23 +0530 Subject: [PATCH 09/18] Removing isRemoteIntegrityEnabled method from BlobContainer Signed-off-by: Raghuvansh Raj --- .../org/opensearch/common/blobstore/BlobContainer.java | 9 --------- .../translog/transfer/BlobStoreTransferService.java | 2 +- 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index a3350aace32a3..a48ac2813bbaf 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -130,15 +130,6 @@ default long readBlobPreferredLength() { */ void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException; - /** - * Needs to be set to true by the implementation if remote data integrity check is supported - * - * @return true if remote data integrity check is supported - */ - default boolean isRemoteDataIntegritySupported() { - return false; - } - /** * Reads blob content from the input stream and writes it to the container in a new blob with the given name, * using an atomic write operation if the implementation supports it. diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index a09fd282a1835..29cdc5dcc782b 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -137,7 +137,7 @@ private CompletableFuture createUploadFuture( writePriority, (size, position) -> new OffsetRangeFileInputStream(fileSnapshot.getPath(), size, position), Objects.requireNonNull(fileSnapshot.getChecksum()), - blobStore.blobContainer(blobPath).isRemoteDataIntegritySupported() + blobStore.blobContainer(blobPath) instanceof VerifyingMultiStreamBlobContainer ); WriteContext writeContext = remoteTransferContainer.createWriteContext(); CompletableFuture uploadFuture = ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)) From 5e500e524b93177ed7c5612cb1fe7646b3721871 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Tue, 27 Jun 2023 14:58:33 +0530 Subject: [PATCH 10/18] Updating docstring for VerifyingMultiStreamBlobContainer Signed-off-by: Raghuvansh Raj --- .../common/blobstore/VerifyingMultiStreamBlobContainer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index a244ea5841e2a..fc2083be166e4 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -15,7 +15,7 @@ /** * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#writeBlobByStreams} to allow - * multipart uploads + * multipart uploads and performs integrity checks on transferred files * * @opensearch.internal */ From c2634d088b4c76b901d54562ad4873f12e20c311 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Mon, 3 Jul 2023 18:01:24 +0530 Subject: [PATCH 11/18] Minor refactor Signed-off-by: Raghuvansh Raj --- .../main/java/org/opensearch/index/translog/TranslogWriter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java index a4b5d486e5913..e7b08b1dda3d2 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -187,7 +187,7 @@ public static TranslogWriter create( final TranslogHeader header = new TranslogHeader(translogUUID, primaryTerm); header.write(channel, !Boolean.TRUE.equals(remoteTranslogEnabled)); TranslogCheckedContainer translogCheckedContainer = null; - if (remoteTranslogEnabled == Boolean.TRUE) { + if (Boolean.TRUE.equals(remoteTranslogEnabled)) { ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); header.write(byteArrayOutputStream); translogCheckedContainer = new TranslogCheckedContainer(byteArrayOutputStream.toByteArray()); From b6dc9258496852c46c1901531c0359b4e2e2d35c Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Mon, 3 Jul 2023 19:42:37 +0530 Subject: [PATCH 12/18] Removed blocking get in translog/segment upload flows Signed-off-by: Raghuvansh Raj --- .../mocks/MockFsBlobContainerVerifying.java | 26 ++-- .../VerifyingMultiStreamBlobContainer.java | 4 +- .../blobstore/stream/write/WriteContext.java | 14 +- .../transfer/RemoteTransferContainer.java | 6 +- .../opensearch/index/shard/FileUploader.java | 89 ------------- .../shard/RemoteStoreRefreshListener.java | 124 ++++++++++++++---- .../store/RemoteSegmentStoreDirectory.java | 91 +++++++++++++ .../transfer/BlobStoreTransferService.java | 40 ++---- .../RemoteTransferContainerTests.java | 2 +- .../RemoteSegmentStoreDirectoryTests.java | 90 ++++++------- ...oreTransferServiceMockRepositoryTests.java | 32 +++-- 11 files changed, 289 insertions(+), 229 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/index/shard/FileUploader.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java index 189c313792425..a76b4d247562d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java @@ -23,7 +23,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -40,8 +39,7 @@ public MockFsBlobContainerVerifying(FsBlobStore blobStore, BlobPath blobPath, Pa } @Override - public CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException { - CompletableFuture completableFuture = new CompletableFuture<>(); + public void writeBlobByStreams(WriteContext writeContext) throws IOException { int nParts = 10; long partSize = writeContext.getFileSize() / nParts; @@ -66,7 +64,7 @@ public CompletableFuture writeBlobByStreams(WriteContext writeContext) thr } inputStream.close(); } catch (IOException e) { - completableFuture.completeExceptionally(e); + writeContext.getCompletionListener().onFailure(e); } finally { latch.countDown(); } @@ -97,23 +95,23 @@ public CompletableFuture writeBlobByStreams(WriteContext writeContext) thr try { // bulks need to succeed for segment files to be generated if (isSegmentFile(writeContext.getFileName()) && triggerDataIntegrityFailure) { - completableFuture.completeExceptionally( - new RuntimeException( - new CorruptIndexException( - "Data integrity check failure for file: " + writeContext.getFileName(), - writeContext.getFileName() + writeContext.getCompletionListener() + .onFailure( + new RuntimeException( + new CorruptIndexException( + "Data integrity check failure for file: " + writeContext.getFileName(), + writeContext.getFileName() + ) ) - ) - ); + ); } else { writeContext.getUploadFinalizer().accept(true); - completableFuture.complete(null); + writeContext.getCompletionListener().onResponse(null); } } catch (Exception e) { - completableFuture.completeExceptionally(e); + writeContext.getCompletionListener().onFailure(e); } - return completableFuture; } private boolean isSegmentFile(String filename) { diff --git a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index fc2083be166e4..10a9c5c26ff01 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -11,7 +11,6 @@ import org.opensearch.common.blobstore.stream.write.WriteContext; import java.io.IOException; -import java.util.concurrent.CompletableFuture; /** * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#writeBlobByStreams} to allow @@ -27,8 +26,7 @@ public interface VerifyingMultiStreamBlobContainer extends BlobContainer { * any of the input streams fails, or writing to the target blob fails * * @param writeContext A WriteContext object encapsulating all information needed to perform the upload - * @return A {@link CompletableFuture} representing the upload * @throws IOException if any of the input streams could not be read, or the target blob could not be written to */ - CompletableFuture writeBlobByStreams(WriteContext writeContext) throws IOException; + void writeBlobByStreams(WriteContext writeContext) throws IOException; } diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java index ef5e3d1e8c26c..0e0f5aadf0a1c 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java @@ -8,6 +8,7 @@ package org.opensearch.common.blobstore.stream.write; +import org.opensearch.action.ActionListener; import org.opensearch.common.CheckedConsumer; import org.opensearch.common.Nullable; import org.opensearch.common.StreamContext; @@ -29,6 +30,7 @@ public class WriteContext { private final CheckedConsumer uploadFinalizer; private final boolean doRemoteDataIntegrityCheck; private final Long expectedChecksum; + private final ActionListener completionListener; /** * Construct a new WriteContext object @@ -40,6 +42,7 @@ public class WriteContext { * @param writePriority The WritePriority of this upload * @param doRemoteDataIntegrityCheck A boolean to inform vendor plugins whether remote data integrity checks need to be done * @param expectedChecksum This parameter expected only when the vendor plugin is expected to do server side data integrity verification + * @param completionListener Callback events should be published on this listener. */ public WriteContext( String fileName, @@ -49,7 +52,8 @@ public WriteContext( WritePriority writePriority, CheckedConsumer uploadFinalizer, boolean doRemoteDataIntegrityCheck, - @Nullable Long expectedChecksum + @Nullable Long expectedChecksum, + ActionListener completionListener ) { this.fileName = fileName; this.streamContextSupplier = streamContextSupplier; @@ -59,6 +63,7 @@ public WriteContext( this.uploadFinalizer = uploadFinalizer; this.doRemoteDataIntegrityCheck = doRemoteDataIntegrityCheck; this.expectedChecksum = expectedChecksum; + this.completionListener = completionListener; } /** @@ -117,4 +122,11 @@ public boolean doRemoteDataIntegrityCheck() { public Long getExpectedChecksum() { return expectedChecksum; } + + /** + * @return Callback events should be published on this listener. + */ + public ActionListener getCompletionListener() { + return completionListener; + } } 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 7864c3ab5c794..a13e6938494c3 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 @@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; +import org.opensearch.action.ActionListener; import org.opensearch.common.CheckedTriFunction; import org.opensearch.common.SetOnce; import org.opensearch.common.StreamContext; @@ -84,7 +85,7 @@ public RemoteTransferContainer( /** * @return The {@link WriteContext} for the current upload */ - public WriteContext createWriteContext() { + public WriteContext createWriteContext(ActionListener respListener) { return new WriteContext( remoteFileName, this::supplyStreamContext, @@ -93,7 +94,8 @@ public WriteContext createWriteContext() { writePriority, this::finalizeUpload, isRemoteDataIntegrityCheckPossible(), - isRemoteDataIntegrityCheckPossible() ? expectedChecksum : null + isRemoteDataIntegrityCheckPossible() ? expectedChecksum : null, + respListener ); } diff --git a/server/src/main/java/org/opensearch/index/shard/FileUploader.java b/server/src/main/java/org/opensearch/index/shard/FileUploader.java deleted file mode 100644 index 04612dace533e..0000000000000 --- a/server/src/main/java/org/opensearch/index/shard/FileUploader.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.shard; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IOContext; -import org.opensearch.common.CheckedFunction; -import org.opensearch.common.util.UploadListener; -import org.opensearch.index.store.RemoteSegmentStoreDirectory; - -import java.io.IOException; -import java.util.Collection; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * This class is a wrapper over the copying of file from local to remote store allowing to decorate the actual copy - * method along with adding hooks of code that can be run before, on success and on failure. - * - * @opensearch.internal - */ -public class FileUploader { - - private static final Logger logger = LogManager.getLogger(FileUploader.class); - - private final UploadListener uploadListener; - - private final RemoteSegmentStoreDirectory remoteDirectory; - - private final Directory storeDirectory; - - private final Set excludeFiles; - - private final CheckedFunction checksumProvider; - - public FileUploader( - UploadListener uploadListener, - RemoteSegmentStoreDirectory remoteDirectory, - Directory storeDirectory, - Set excludeFiles, - CheckedFunction checksumProvider - ) { - this.uploadListener = uploadListener; - this.remoteDirectory = remoteDirectory; - this.storeDirectory = storeDirectory; - this.excludeFiles = excludeFiles; - this.checksumProvider = checksumProvider; - } - - /** - * Calling this method will filter out files that need to be skipped and call - * {@link RemoteSegmentStoreDirectory#copyFilesFrom} - * - * @param files The files that need to be uploaded - * @return A boolean for whether all files were successful or not - * @throws Exception when the underlying upload fails - */ - public boolean uploadFiles(Collection files) throws Exception { - Collection filteredFiles = files.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); - return remoteDirectory.copyFilesFrom(storeDirectory, filteredFiles, IOContext.DEFAULT, uploadTracker); - } - - /** - * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. - * - * @param file that needs to be uploaded. - * @return true if the upload has to be skipped for the file. - */ - private boolean skipUpload(String file) { - try { - // Exclude files that are already uploaded and the exclude files to come up with the list of files to be uploaded. - return excludeFiles.contains(file) || remoteDirectory.containsFile(file, checksumProvider.apply(file)); - } catch (IOException e) { - logger.error( - "Exception while reading checksum of local segment file: {}, ignoring the exception and re-uploading the file", - file - ); - } - return false; - } -} diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 29b6a73502984..2d9b392a0f199 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -19,7 +19,9 @@ import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.opensearch.action.ActionListener; import org.opensearch.action.bulk.BackoffPolicy; +import org.opensearch.action.support.GroupedActionListener; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.logging.Loggers; import org.opensearch.common.unit.TimeValue; @@ -47,6 +49,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -111,7 +114,7 @@ public final class RemoteStoreRefreshListener implements ReferenceManager.Refres private final SegmentReplicationCheckpointPublisher checkpointPublisher; - private final FileUploader fileUploader; + private final UploadListener statsListener; public RemoteStoreRefreshListener( IndexShard indexShard, @@ -138,7 +141,7 @@ public RemoteStoreRefreshListener( this.segmentTracker = segmentTracker; resetBackOffDelayIterator(); this.checkpointPublisher = checkpointPublisher; - this.fileUploader = new FileUploader(new UploadListener() { + this.statsListener = new UploadListener() { @Override public void beforeUpload(String file) { // Start tracking the upload bytes started @@ -157,7 +160,7 @@ public void onFailure(String file) { // Track upload failure segmentTracker.addUploadBytesFailed(latestFileNameSizeOnLocalMap.get(file)); } - }, remoteDirectory, storeDirectory, EXCLUDE_FILES, this::getChecksumOfLocalFile); + }; } @Override @@ -191,7 +194,7 @@ private synchronized void syncSegments(boolean isRetry) { long refreshTimeMs = segmentTracker.getLocalRefreshTimeMs(), refreshClockTimeMs = segmentTracker.getLocalRefreshClockTimeMs(); long refreshSeqNo = segmentTracker.getLocalRefreshSeqNo(); long bytesBeforeUpload = segmentTracker.getUploadBytesSucceeded(), startTimeInNS = System.nanoTime(); - boolean shouldRetry = true; + try { if (this.primaryTerm != indexShard.getOperationPrimaryTerm()) { @@ -243,18 +246,55 @@ private synchronized void syncSegments(boolean isRetry) { // Create a map of file name to size and update the refresh segment tracker updateLocalSizeMapAndTracker(localSegmentsPostRefresh); + CountDownLatch latch = new CountDownLatch(1); + ActionListener segmentUploadsCompletedListener = new ActionListener() { + @Override + public void onResponse(Void unused) { + boolean shouldRetry = true; + try { + // Start metadata file upload + uploadMetadata(localSegmentsPostRefresh, segmentInfos); + clearStaleFilesFromLocalSegmentChecksumMap(localSegmentsPostRefresh); + onSuccessfulSegmentsSync( + refreshTimeMs, + refreshClockTimeMs, + refreshSeqNo, + lastRefreshedCheckpoint, + checkpoint + ); + // At this point since we have uploaded new segments, segment infos and segment metadata file, + // along with marking minSeqNoToKeep, upload has succeeded completely. + shouldRetry = false; + } catch (Exception e) { + // We don't want to fail refresh if upload of new segments fails. The missed segments will be re-tried + // in the next refresh. This should not affect durability of the indexed data after remote trans-log + // integration. + logger.warn("Exception in post new segment upload actions", e); + } finally { + doComplete(shouldRetry); + } + } + + @Override + public void onFailure(Exception e) { + logger.warn("Exception while uploading new segments to the remote segment store", e); + doComplete(true); + } + + private void doComplete(boolean shouldRetry) { + try { + // Update the segment tracker with the final upload status as seen at the end + updateFinalUploadStatusInSegmentTracker(shouldRetry == false, bytesBeforeUpload, startTimeInNS); + afterSegmentsSync(isRetry, shouldRetry); + } finally { + latch.countDown(); + } + } + }; // Start the segments files upload - boolean newSegmentsUploadStatus = uploadNewSegments(localSegmentsPostRefresh); - if (newSegmentsUploadStatus) { - // Start metadata file upload - uploadMetadata(localSegmentsPostRefresh, segmentInfos); - clearStaleFilesFromLocalSegmentChecksumMap(localSegmentsPostRefresh); - onSuccessfulSegmentsSync(refreshTimeMs, refreshClockTimeMs, refreshSeqNo, lastRefreshedCheckpoint, checkpoint); - // At this point since we have uploaded new segments, segment infos and segment metadata file, - // along with marking minSeqNoToKeep, upload has succeeded completely. - shouldRetry = false; - } + uploadNewSegments(localSegmentsPostRefresh, segmentUploadsCompletedListener); + latch.await(); } } catch (EngineException e) { logger.warn("Exception while reading SegmentInfosSnapshot", e); @@ -266,11 +306,7 @@ private synchronized void syncSegments(boolean isRetry) { } } catch (Throwable t) { logger.error("Exception in RemoteStoreRefreshListener.afterRefresh()", t); - } finally { - // Update the segment tracker with the final upload status as seen at the end - updateFinalUploadStatusInSegmentTracker(shouldRetry == false, bytesBeforeUpload, startTimeInNS); } - afterSegmentsSync(isRetry, shouldRetry); } /** @@ -379,18 +415,50 @@ void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos se } } - private boolean uploadNewSegments(Collection localSegmentsPostRefresh) throws IOException { - AtomicBoolean uploadSuccess = new AtomicBoolean(true); + private void uploadNewSegments(Collection localSegmentsPostRefresh, ActionListener listener) { + Collection filteredFiles = localSegmentsPostRefresh.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); + if (filteredFiles.size() == 0) { + listener.onResponse(null); + return; + } + + ActionListener> mappedListener = ActionListener.map(listener, resp -> null); + GroupedActionListener batchUploadListener = new GroupedActionListener<>(mappedListener, filteredFiles.size()); + + for (String src : filteredFiles) { + ActionListener aggregatedListener = ActionListener.wrap(resp -> { + statsListener.onSuccess(src); + batchUploadListener.onResponse(resp); + }, ex -> { + logger.warn(() -> new ParameterizedMessage("Exception: [{}] while uploading segment files", ex), ex); + if (ex instanceof CorruptIndexException) { + indexShard.failShard(ex.getMessage(), ex); + } + statsListener.onFailure(src); + batchUploadListener.onFailure(ex); + }); + statsListener.beforeUpload(src); + remoteDirectory.copyFrom(storeDirectory, src, IOContext.DEFAULT, aggregatedListener); + } + } + + /** + * Whether to upload a file or not depending on whether file is in excluded list or has been already uploaded. + * + * @param file that needs to be uploaded. + * @return true if the upload has to be skipped for the file. + */ + private boolean skipUpload(String file) { try { - uploadSuccess.set(fileUploader.uploadFiles(localSegmentsPostRefresh)); - } catch (Exception e) { - uploadSuccess.set(false); - if (e instanceof CorruptIndexException) { - indexShard.failShard(e.getMessage(), e); - } - logger.warn(() -> new ParameterizedMessage("Exception: [{}] while uploading segment files", e), e); + // Exclude files that are already uploaded and the exclude files to come up with the list of files to be uploaded. + return EXCLUDE_FILES.contains(file) || remoteDirectory.containsFile(file, getChecksumOfLocalFile(file)); + } catch (IOException e) { + logger.error( + "Exception while reading checksum of local segment file: {}, ignoring the exception and re-uploading the file", + file + ); } - return uploadSuccess.get(); + return false; } private String getChecksumOfLocalFile(String file) throws IOException { 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 bcaf1c9743cf0..1b4dd993f9bab 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -11,7 +11,9 @@ import com.jcraft.jzlib.JZlib; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; @@ -20,6 +22,8 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.ActionListener; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.blobstore.exception.CorruptFileException; @@ -364,6 +368,89 @@ public IndexInput openInput(String name, IOContext context) throws IOException { } } + /** + * Copies a file from the source directory to a remote based on multi-stream upload support. + * If vendor plugin supports uploading multiple parts in parallel, BlobContainer#writeBlobByStreams + * will be used, else, the legacy {@link RemoteSegmentStoreDirectory#copyFrom(Directory, String, String, IOContext)} + * will be called. + * + * @param from The directory for the file to be uploaded + * @param src File to be uploaded + * @param context IOContext to be used to open IndexInput of file during remote upload + * @param listener Listener to handle upload callback events + */ + public void copyFrom(Directory from, String src, IOContext context, ActionListener listener) { + if (remoteDataDirectory.getBlobContainer() instanceof VerifyingMultiStreamBlobContainer) { + try { + String remoteFilename = getNewRemoteSegmentFilename(src); + uploadBlob(from, src, remoteFilename, context, listener); + } catch (Exception e) { + listener.onFailure(e); + } + } else { + try { + copyFrom(from, src, src, context); + listener.onResponse(null); + } catch (Exception e) { + logger.warn(() -> new ParameterizedMessage("Exception while uploading file {} to the remote segment store", src), e); + listener.onFailure(e); + } + } + } + + private void uploadBlob(Directory from, String src, String remoteFileName, IOContext ioContext, ActionListener listener) + throws Exception { + long expectedChecksum = calculateChecksumOfChecksum(from, src); + long contentLength; + try (IndexInput indexInput = from.openInput(src, ioContext)) { + contentLength = indexInput.length(); + } + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + src, + remoteFileName, + contentLength, + true, + WritePriority.NORMAL, + (size, position) -> new OffsetRangeIndexInputStream(from.openInput(src, ioContext), size, position), + expectedChecksum, + remoteDataDirectory.getBlobContainer() instanceof VerifyingMultiStreamBlobContainer + ); + ActionListener completionListener = ActionListener.wrap(resp -> { + try { + postUpload(from, src, remoteFileName, getChecksumOfLocalFile(from, src)); + listener.onResponse(null); + } catch (Exception e) { + logger.error(() -> new ParameterizedMessage("Exception in segment postUpload for file [{}]", src), e); + listener.onFailure(e); + } + }, ex -> { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", src), ex); + IOException corruptIndexException = ExceptionsHelper.unwrapCorruption(ex); + if (corruptIndexException != null) { + listener.onFailure(corruptIndexException); + return; + } + Throwable throwable = ExceptionsHelper.unwrap(ex, CorruptFileException.class); + if (throwable != null) { + CorruptFileException corruptFileException = (CorruptFileException) throwable; + listener.onFailure(new CorruptIndexException(corruptFileException.getMessage(), corruptFileException.getFileName())); + return; + } + listener.onFailure(ex); + }); + + completionListener = ActionListener.runBefore(completionListener, () -> { + try { + remoteTransferContainer.close(); + } catch (Exception e) { + logger.warn("Error occurred while closing streams", e); + } + }); + + WriteContext writeContext = remoteTransferContainer.createWriteContext(completionListener); + ((VerifyingMultiStreamBlobContainer) remoteDataDirectory.getBlobContainer()).writeBlobByStreams(writeContext); + } + /** * This acquires a lock on a given commit by creating a lock file in lock directory using {@code FileLockInfo} * @param primaryTerm Primary Term of index at the time of commit. @@ -440,6 +527,10 @@ public void copyFrom(Directory from, String src, String dest, IOContext context, String remoteFilename; remoteFilename = getNewRemoteSegmentFilename(dest); remoteDataDirectory.copyFrom(from, src, remoteFilename, context); + postUpload(from, src, remoteFilename, checksum); + } + + private void postUpload(Directory from, String src, String remoteFilename, String checksum) throws IOException { UploadedSegmentMetadata segmentMetadata = new UploadedSegmentMetadata(src, remoteFilename, checksum, from.fileLength(src)); segmentsUploadedToRemoteStore.put(src, segmentMetadata); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 29cdc5dcc782b..216a06fa66adc 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -29,12 +29,10 @@ import java.io.InputStream; import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; -import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.concurrent.CompletableFuture; import static org.opensearch.common.blobstore.BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC; @@ -92,37 +90,24 @@ public void uploadBlobs( ActionListener listener, WritePriority writePriority ) { - List> resultFutures = new ArrayList<>(); fileSnapshots.forEach(fileSnapshot -> { BlobPath blobPath = blobPaths.get(fileSnapshot.getPrimaryTerm()); if (!(blobStore.blobContainer(blobPath) instanceof VerifyingMultiStreamBlobContainer)) { uploadBlobByThreadPool(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); } else { - CompletableFuture resultFuture = createUploadFuture(fileSnapshot, listener, blobPath, writePriority); - if (resultFuture != null) { - resultFutures.add(resultFuture); - } + uploadBlob(fileSnapshot, listener, blobPath, writePriority); } }); - if (resultFutures.isEmpty() == false) { - CompletableFuture resultFuture = CompletableFuture.allOf(resultFutures.toArray(new CompletableFuture[0])); - try { - resultFuture.get(); - } catch (Exception e) { - logger.warn("Failed to upload blobs", e); - } - } } - private CompletableFuture createUploadFuture( + private void uploadBlob( TransferFileSnapshot fileSnapshot, ActionListener listener, BlobPath blobPath, WritePriority writePriority ) { - CompletableFuture resultFuture = null; try { ChannelFactory channelFactory = FileChannel::open; long contentLength; @@ -139,22 +124,22 @@ private CompletableFuture createUploadFuture( Objects.requireNonNull(fileSnapshot.getChecksum()), blobStore.blobContainer(blobPath) instanceof VerifyingMultiStreamBlobContainer ); - WriteContext writeContext = remoteTransferContainer.createWriteContext(); - CompletableFuture uploadFuture = ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)) - .writeBlobByStreams(writeContext); - resultFuture = uploadFuture.whenComplete((resp, throwable) -> { + ActionListener completionListener = ActionListener.wrap(resp -> listener.onResponse(fileSnapshot), ex -> { + logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), ex); + listener.onFailure(new FileTransferException(fileSnapshot, ex)); + }); + + completionListener = ActionListener.runBefore(completionListener, () -> { try { remoteTransferContainer.close(); } catch (Exception e) { logger.warn("Error occurred while closing streams", e); } - if (throwable != null) { - logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), throwable); - listener.onFailure(new FileTransferException(fileSnapshot, throwable)); - } else { - listener.onResponse(fileSnapshot); - } }); + + WriteContext writeContext = remoteTransferContainer.createWriteContext(completionListener); + ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)).writeBlobByStreams(writeContext); + } catch (Exception e) { logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), e); listener.onFailure(new FileTransferException(fileSnapshot, e)); @@ -166,7 +151,6 @@ private CompletableFuture createUploadFuture( } } - return resultFuture; } @Override 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 48940a0d401fd..ddc5f6b05705a 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 @@ -167,7 +167,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { doRemoteDataIntegrityCheck ) ) { - WriteContext writeContext = remoteTransferContainer.createWriteContext(); + WriteContext writeContext = remoteTransferContainer.createWriteContext(null); assertEquals(remoteFileName, writeContext.getFileName()); assertTrue(writeContext.isFailIfAlreadyExists()); assertEquals(TEST_FILE_SIZE_BYTES, writeContext.getFileSize()); 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 4d9a6bbe7f0d1..20d949026747a 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -23,6 +23,8 @@ import org.apache.lucene.tests.util.LuceneTestCase; import org.junit.After; import org.junit.Before; +import org.mockito.Mockito; +import org.opensearch.action.ActionListener; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; @@ -50,6 +52,8 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.HashMap; @@ -74,7 +78,7 @@ public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { private RemoteStoreMetadataLockManager mdLockManager; private RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; - private TestUploadListener testUploadListener; + private TestUploadListener testUploadTracker; private IndexShard indexShard; private SegmentInfos segmentInfos; private ThreadPool threadPool; @@ -527,15 +531,25 @@ public void testCopyFilesFromMultipart() throws Exception { VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); - uploadResponseCompletableFuture.complete(null); - when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); - - remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener); - + Mockito.doAnswer(invocation -> { + WriteContext writeContext = invocation.getArgument(0); + writeContext.getCompletionListener().onResponse(null); + return null; + }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + + CountDownLatch latch = new CountDownLatch(1); + ActionListener completionListener = new ActionListener() { + @Override + public void onResponse(Void unused) { + latch.countDown(); + } + + @Override + public void onFailure(Exception e) {} + }; + remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener); + assertTrue(latch.await(5000, TimeUnit.SECONDS)); assertTrue(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadListener.UploadStatus.UPLOAD_SUCCESS, testUploadListener.getUploadStatus(filename)); - storeDirectory.close(); } @@ -555,48 +569,24 @@ public void testCopyFilesFromMultipartIOException() throws Exception { VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); - uploadResponseCompletableFuture.complete(null); - when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenThrow(new IOException()); - - assertThrows( - IOException.class, - () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener) - ); - - assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadListener.UploadStatus.UPLOAD_FAILURE, testUploadListener.getUploadStatus(filename)); - - storeDirectory.close(); - } - - public void testCopyFilesFromMultipartUploadFutureCompletedExceptionally() throws Exception { - String filename = "_100.si"; - populateMetadata(); - remoteSegmentStoreDirectory.init(); - - Directory storeDirectory = LuceneTestCase.newDirectory(); - IndexOutput indexOutput = storeDirectory.createOutput(filename, IOContext.DEFAULT); - indexOutput.writeString("Hello World!"); - CodecUtil.writeFooter(indexOutput); - indexOutput.close(); - storeDirectory.sync(List.of(filename)); - - assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - - VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); - when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); - CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); - uploadResponseCompletableFuture.completeExceptionally(new IOException()); - when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); - - assertThrows( - ExecutionException.class, - () -> remoteSegmentStoreDirectory.copyFilesFrom(storeDirectory, List.of(filename), IOContext.DEFAULT, testUploadListener) - ); - + Mockito.doAnswer(invocation -> { + WriteContext writeContext = invocation.getArgument(0); + writeContext.getCompletionListener().onFailure(new Exception("Test exception")); + return null; + }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + CountDownLatch latch = new CountDownLatch(1); + ActionListener completionListener = new ActionListener<>() { + @Override + public void onResponse(Void unused) {} + + @Override + public void onFailure(Exception e) { + latch.countDown(); + } + }; + remoteSegmentStoreDirectory.copyFrom(storeDirectory, filename, IOContext.DEFAULT, completionListener); + assertTrue(latch.await(5000, TimeUnit.SECONDS)); assertFalse(remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore().containsKey(filename)); - assertEquals(TestUploadListener.UploadStatus.UPLOAD_FAILURE, testUploadListener.getUploadStatus(filename)); storeDirectory.close(); } diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java index 60a2b16048c44..824608c71df2e 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -8,6 +8,7 @@ package org.opensearch.index.translog.transfer; +import org.mockito.Mockito; import org.opensearch.action.ActionListener; import org.opensearch.action.LatchedActionListener; import org.opensearch.common.blobstore.BlobPath; @@ -25,7 +26,6 @@ import java.nio.file.StandardOpenOption; import java.util.Collections; import java.util.HashMap; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -60,9 +60,11 @@ public void testUploadBlobs() throws Exception { ); VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); - CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); - uploadResponseCompletableFuture.complete(null); - when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + Mockito.doAnswer(invocation -> { + WriteContext writeContext = invocation.getArgument(0); + writeContext.getCompletionListener().onResponse(null); + return null; + }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); @@ -144,20 +146,19 @@ public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception ); VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); - CompletableFuture uploadResponseCompletableFuture = new CompletableFuture<>(); - uploadResponseCompletableFuture.completeExceptionally(new IOException()); - when(blobContainer.writeBlobByStreams(any(WriteContext.class))).thenReturn(uploadResponseCompletableFuture); + Mockito.doAnswer(invocation -> { + WriteContext writeContext = invocation.getArgument(0); + writeContext.getCompletionListener().onFailure(new Exception("Test exception")); + return null; + }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); CountDownLatch latch = new CountDownLatch(1); AtomicBoolean onResponseCalled = new AtomicBoolean(false); AtomicReference exceptionRef = new AtomicReference<>(); - transferService.uploadBlobs(Collections.singleton(transferFileSnapshot), new HashMap<>() { - { - put(transferFileSnapshot.getPrimaryTerm(), new BlobPath().add("sample_path")); - } - }, new LatchedActionListener<>(new ActionListener<>() { + LatchedActionListener listener = new LatchedActionListener<>(new ActionListener<>() { @Override public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { onResponseCalled.set(true); @@ -167,7 +168,12 @@ public void onResponse(FileSnapshot.TransferFileSnapshot fileSnapshot) { public void onFailure(Exception e) { exceptionRef.set(e); } - }, latch), WritePriority.HIGH); + }, latch); + transferService.uploadBlobs(Collections.singleton(transferFileSnapshot), new HashMap<>() { + { + put(transferFileSnapshot.getPrimaryTerm(), new BlobPath().add("sample_path")); + } + }, listener, WritePriority.HIGH); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); From b3ff7bde865f7e092268ed16f6ac3f5813bfdd8e Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Tue, 4 Jul 2023 17:36:55 +0530 Subject: [PATCH 13/18] Split completion listener and rename method Signed-off-by: Raghuvansh Raj --- .../mocks/MockFsBlobContainerVerifying.java | 11 +++++----- .../VerifyingMultiStreamBlobContainer.java | 6 ++++-- .../blobstore/stream/write/WriteContext.java | 13 +----------- .../transfer/RemoteTransferContainer.java | 5 ++--- .../store/RemoteSegmentStoreDirectory.java | 4 ++-- .../transfer/BlobStoreTransferService.java | 4 ++-- .../RemoteSegmentStoreDirectoryTests.java | 12 +++++------ ...oreTransferServiceMockRepositoryTests.java | 20 +++++++++---------- 8 files changed, 33 insertions(+), 42 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java index a76b4d247562d..f8f658a802461 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java @@ -9,6 +9,7 @@ package org.opensearch.remotestore.multipart.mocks; import org.apache.lucene.index.CorruptIndexException; +import org.opensearch.action.ActionListener; import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.io.InputStreamContainer; import org.opensearch.common.StreamContext; @@ -39,7 +40,7 @@ public MockFsBlobContainerVerifying(FsBlobStore blobStore, BlobPath blobPath, Pa } @Override - public void writeBlobByStreams(WriteContext writeContext) throws IOException { + public void asyncBlobUpload(WriteContext writeContext, ActionListener completionListener) throws IOException { int nParts = 10; long partSize = writeContext.getFileSize() / nParts; @@ -64,7 +65,7 @@ public void writeBlobByStreams(WriteContext writeContext) throws IOException { } inputStream.close(); } catch (IOException e) { - writeContext.getCompletionListener().onFailure(e); + completionListener.onFailure(e); } finally { latch.countDown(); } @@ -95,7 +96,7 @@ public void writeBlobByStreams(WriteContext writeContext) throws IOException { try { // bulks need to succeed for segment files to be generated if (isSegmentFile(writeContext.getFileName()) && triggerDataIntegrityFailure) { - writeContext.getCompletionListener() + completionListener .onFailure( new RuntimeException( new CorruptIndexException( @@ -106,10 +107,10 @@ public void writeBlobByStreams(WriteContext writeContext) throws IOException { ); } else { writeContext.getUploadFinalizer().accept(true); - writeContext.getCompletionListener().onResponse(null); + completionListener.onResponse(null); } } catch (Exception e) { - writeContext.getCompletionListener().onFailure(e); + completionListener.onFailure(e); } } diff --git a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index 10a9c5c26ff01..5e598888864d4 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -8,6 +8,7 @@ package org.opensearch.common.blobstore; +import org.opensearch.action.ActionListener; import org.opensearch.common.blobstore.stream.write.WriteContext; import java.io.IOException; @@ -25,8 +26,9 @@ public interface VerifyingMultiStreamBlobContainer extends BlobContainer { * StreamContextSupplier in the WriteContext passed to this method. An {@link IOException} is thrown if reading * any of the input streams fails, or writing to the target blob fails * - * @param writeContext A WriteContext object encapsulating all information needed to perform the upload + * @param writeContext A WriteContext object encapsulating all information needed to perform the upload + * @param completionListener Listener on which upload events should be published. * @throws IOException if any of the input streams could not be read, or the target blob could not be written to */ - void writeBlobByStreams(WriteContext writeContext) throws IOException; + void asyncBlobUpload(WriteContext writeContext, ActionListener completionListener) throws IOException; } diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java index 0e0f5aadf0a1c..e06d16dc1705c 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java @@ -30,7 +30,6 @@ public class WriteContext { private final CheckedConsumer uploadFinalizer; private final boolean doRemoteDataIntegrityCheck; private final Long expectedChecksum; - private final ActionListener completionListener; /** * Construct a new WriteContext object @@ -42,7 +41,6 @@ public class WriteContext { * @param writePriority The WritePriority of this upload * @param doRemoteDataIntegrityCheck A boolean to inform vendor plugins whether remote data integrity checks need to be done * @param expectedChecksum This parameter expected only when the vendor plugin is expected to do server side data integrity verification - * @param completionListener Callback events should be published on this listener. */ public WriteContext( String fileName, @@ -52,8 +50,7 @@ public WriteContext( WritePriority writePriority, CheckedConsumer uploadFinalizer, boolean doRemoteDataIntegrityCheck, - @Nullable Long expectedChecksum, - ActionListener completionListener + @Nullable Long expectedChecksum ) { this.fileName = fileName; this.streamContextSupplier = streamContextSupplier; @@ -63,7 +60,6 @@ public WriteContext( this.uploadFinalizer = uploadFinalizer; this.doRemoteDataIntegrityCheck = doRemoteDataIntegrityCheck; this.expectedChecksum = expectedChecksum; - this.completionListener = completionListener; } /** @@ -122,11 +118,4 @@ public boolean doRemoteDataIntegrityCheck() { public Long getExpectedChecksum() { return expectedChecksum; } - - /** - * @return Callback events should be published on this listener. - */ - public ActionListener getCompletionListener() { - return completionListener; - } } 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 a13e6938494c3..caf09c7ed9d4a 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 @@ -85,7 +85,7 @@ public RemoteTransferContainer( /** * @return The {@link WriteContext} for the current upload */ - public WriteContext createWriteContext(ActionListener respListener) { + public WriteContext createWriteContext() { return new WriteContext( remoteFileName, this::supplyStreamContext, @@ -94,8 +94,7 @@ public WriteContext createWriteContext(ActionListener respListener) { writePriority, this::finalizeUpload, isRemoteDataIntegrityCheckPossible(), - isRemoteDataIntegrityCheckPossible() ? expectedChecksum : null, - respListener + isRemoteDataIntegrityCheckPossible() ? expectedChecksum : null ); } 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 1b4dd993f9bab..395ecba442e86 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -447,8 +447,8 @@ private void uploadBlob(Directory from, String src, String remoteFileName, IOCon } }); - WriteContext writeContext = remoteTransferContainer.createWriteContext(completionListener); - ((VerifyingMultiStreamBlobContainer) remoteDataDirectory.getBlobContainer()).writeBlobByStreams(writeContext); + WriteContext writeContext = remoteTransferContainer.createWriteContext(); + ((VerifyingMultiStreamBlobContainer) remoteDataDirectory.getBlobContainer()).asyncBlobUpload(writeContext, completionListener); } /** diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 216a06fa66adc..873e96a76ef2d 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -137,8 +137,8 @@ private void uploadBlob( } }); - WriteContext writeContext = remoteTransferContainer.createWriteContext(completionListener); - ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)).writeBlobByStreams(writeContext); + WriteContext writeContext = remoteTransferContainer.createWriteContext(); + ((VerifyingMultiStreamBlobContainer) blobStore.blobContainer(blobPath)).asyncBlobUpload(writeContext, completionListener); } catch (Exception e) { logger.error(() -> new ParameterizedMessage("Failed to upload blob {}", fileSnapshot.getName()), e); 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 20d949026747a..7361bc4480c67 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -532,10 +532,10 @@ public void testCopyFilesFromMultipart() throws Exception { VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); Mockito.doAnswer(invocation -> { - WriteContext writeContext = invocation.getArgument(0); - writeContext.getCompletionListener().onResponse(null); + ActionListener completionListener = invocation.getArgument(1); + completionListener.onResponse(null); return null; - }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + }).when(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); CountDownLatch latch = new CountDownLatch(1); ActionListener completionListener = new ActionListener() { @@ -570,10 +570,10 @@ public void testCopyFilesFromMultipartIOException() throws Exception { VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); when(remoteDataDirectory.getBlobContainer()).thenReturn(blobContainer); Mockito.doAnswer(invocation -> { - WriteContext writeContext = invocation.getArgument(0); - writeContext.getCompletionListener().onFailure(new Exception("Test exception")); + ActionListener completionListener = invocation.getArgument(1); + completionListener.onFailure(new Exception("Test exception")); return null; - }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + }).when(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); CountDownLatch latch = new CountDownLatch(1); ActionListener completionListener = new ActionListener<>() { @Override diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java index 824608c71df2e..1175716679d0f 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceMockRepositoryTests.java @@ -61,10 +61,10 @@ public void testUploadBlobs() throws Exception { VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); Mockito.doAnswer(invocation -> { - WriteContext writeContext = invocation.getArgument(0); - writeContext.getCompletionListener().onResponse(null); + ActionListener completionListener = invocation.getArgument(1); + completionListener.onResponse(null); return null; - }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + }).when(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); @@ -90,7 +90,7 @@ public void onFailure(Exception e) { }, latch), WritePriority.HIGH); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); - verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + verify(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); assertTrue(onResponseCalled.get()); assertEquals(transferFileSnapshot.getPrimaryTerm(), fileSnapshotRef.get().getPrimaryTerm()); assertEquals(transferFileSnapshot.getName(), fileSnapshotRef.get().getName()); @@ -107,7 +107,7 @@ public void testUploadBlobsIOException() throws Exception { ); VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); - doThrow(new IOException()).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + doThrow(new IOException()).when(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); TransferService transferService = new BlobStoreTransferService(blobStore, threadPool); @@ -131,7 +131,7 @@ public void onFailure(Exception e) { }, latch), WritePriority.HIGH); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); - verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + verify(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); assertFalse(onResponseCalled.get()); assertTrue(exceptionRef.get() instanceof FileTransferException); } @@ -147,10 +147,10 @@ public void testUploadBlobsUploadFutureCompletedExceptionally() throws Exception VerifyingMultiStreamBlobContainer blobContainer = mock(VerifyingMultiStreamBlobContainer.class); Mockito.doAnswer(invocation -> { - WriteContext writeContext = invocation.getArgument(0); - writeContext.getCompletionListener().onFailure(new Exception("Test exception")); + ActionListener completionListener = invocation.getArgument(1); + completionListener.onFailure(new Exception("Test exception")); return null; - }).when(blobContainer).writeBlobByStreams(any(WriteContext.class)); + }).when(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); when(blobStore.blobContainer(any(BlobPath.class))).thenReturn(blobContainer); @@ -176,7 +176,7 @@ public void onFailure(Exception e) { }, listener, WritePriority.HIGH); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); - verify(blobContainer).writeBlobByStreams(any(WriteContext.class)); + verify(blobContainer).asyncBlobUpload(any(WriteContext.class), any()); assertFalse(onResponseCalled.get()); assertTrue(exceptionRef.get() instanceof FileTransferException); } From 8d32aa9c3f859941db943aedcfdf34815a681ec0 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Wed, 5 Jul 2023 00:39:56 +0530 Subject: [PATCH 14/18] Running spotless checks Signed-off-by: Raghuvansh Raj --- .../mocks/MockFsBlobContainerVerifying.java | 15 +++++++-------- .../common/blobstore/BlobContainer.java | 2 -- .../VerifyingMultiStreamBlobContainer.java | 2 +- .../blobstore/stream/write/WriteContext.java | 1 - .../transfer/RemoteTransferContainer.java | 1 - .../transfer/RemoteTransferContainerTests.java | 2 +- .../store/RemoteSegmentStoreDirectoryTests.java | 4 +--- 7 files changed, 10 insertions(+), 17 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java index f8f658a802461..4749db3cc55ae 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java @@ -96,15 +96,14 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp try { // bulks need to succeed for segment files to be generated if (isSegmentFile(writeContext.getFileName()) && triggerDataIntegrityFailure) { - completionListener - .onFailure( - new RuntimeException( - new CorruptIndexException( - "Data integrity check failure for file: " + writeContext.getFileName(), - writeContext.getFileName() - ) + completionListener.onFailure( + new RuntimeException( + new CorruptIndexException( + "Data integrity check failure for file: " + writeContext.getFileName(), + writeContext.getFileName() ) - ); + ) + ); } else { writeContext.getUploadFinalizer().accept(true); completionListener.onResponse(null); diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index a48ac2813bbaf..e626824e7e271 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -32,8 +32,6 @@ package org.opensearch.common.blobstore; -import org.opensearch.common.blobstore.stream.write.WriteContext; - import org.opensearch.action.ActionListener; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index 5e598888864d4..0dfcc5c50e4b1 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -14,7 +14,7 @@ import java.io.IOException; /** - * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#writeBlobByStreams} to allow + * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#asyncBlobUpload} to allow * multipart uploads and performs integrity checks on transferred files * * @opensearch.internal diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java index e06d16dc1705c..ef5e3d1e8c26c 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java @@ -8,7 +8,6 @@ package org.opensearch.common.blobstore.stream.write; -import org.opensearch.action.ActionListener; import org.opensearch.common.CheckedConsumer; import org.opensearch.common.Nullable; import org.opensearch.common.StreamContext; 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 caf09c7ed9d4a..7864c3ab5c794 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 @@ -12,7 +12,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; -import org.opensearch.action.ActionListener; import org.opensearch.common.CheckedTriFunction; import org.opensearch.common.SetOnce; import org.opensearch.common.StreamContext; 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 ddc5f6b05705a..48940a0d401fd 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 @@ -167,7 +167,7 @@ public OffsetRangeInputStream get(long size, long position) throws IOException { doRemoteDataIntegrityCheck ) ) { - WriteContext writeContext = remoteTransferContainer.createWriteContext(null); + WriteContext writeContext = remoteTransferContainer.createWriteContext(); assertEquals(remoteFileName, writeContext.getFileName()); assertTrue(writeContext.isFailIfAlreadyExists()); assertEquals(TEST_FILE_SIZE_BYTES, writeContext.getFileSize()); 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 7361bc4480c67..ea092fffa3a9a 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -54,8 +54,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import java.util.HashMap; import java.util.Collection; import java.util.concurrent.ExecutorService; @@ -100,7 +98,7 @@ public void setup() throws IOException { mdLockManager, threadPool ); - testUploadListener = new TestUploadListener(); + testUploadTracker = new TestUploadListener(); Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); From a979103d72e9a7cc74364013b81733331d4180dc Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Wed, 5 Jul 2023 10:32:09 +0530 Subject: [PATCH 15/18] Added latchedListener Signed-off-by: Raghuvansh Raj --- .../index/shard/RemoteStoreRefreshListener.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 2d9b392a0f199..e087bbb265727 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -20,6 +20,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.opensearch.action.ActionListener; +import org.opensearch.action.LatchedActionListener; import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.action.support.GroupedActionListener; import org.opensearch.common.concurrent.GatedCloseable; @@ -247,7 +248,7 @@ private synchronized void syncSegments(boolean isRetry) { // Create a map of file name to size and update the refresh segment tracker updateLocalSizeMapAndTracker(localSegmentsPostRefresh); CountDownLatch latch = new CountDownLatch(1); - ActionListener segmentUploadsCompletedListener = new ActionListener() { + ActionListener segmentUploadsCompletedListener = new LatchedActionListener<>(new ActionListener<>() { @Override public void onResponse(Void unused) { boolean shouldRetry = true; @@ -282,15 +283,11 @@ public void onFailure(Exception e) { } private void doComplete(boolean shouldRetry) { - try { - // Update the segment tracker with the final upload status as seen at the end - updateFinalUploadStatusInSegmentTracker(shouldRetry == false, bytesBeforeUpload, startTimeInNS); - afterSegmentsSync(isRetry, shouldRetry); - } finally { - latch.countDown(); - } + // Update the segment tracker with the final upload status as seen at the end + updateFinalUploadStatusInSegmentTracker(shouldRetry == false, bytesBeforeUpload, startTimeInNS); + afterSegmentsSync(isRetry, shouldRetry); } - }; + }, latch); // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, segmentUploadsCompletedListener); From 9b8d95c2270894ee3b0d5582fd68a7db40734587 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Wed, 5 Jul 2023 12:22:21 +0530 Subject: [PATCH 16/18] Added offset in stream container and moved stream container to common lib Signed-off-by: Raghuvansh Raj --- .../opensearch/common/io/InputStreamContainer.java | 11 ++++++++++- .../blobstore/transfer/RemoteTransferContainer.java | 2 +- 2 files changed, 11 insertions(+), 2 deletions(-) rename {server => libs/common}/src/main/java/org/opensearch/common/io/InputStreamContainer.java (85%) diff --git a/server/src/main/java/org/opensearch/common/io/InputStreamContainer.java b/libs/common/src/main/java/org/opensearch/common/io/InputStreamContainer.java similarity index 85% rename from server/src/main/java/org/opensearch/common/io/InputStreamContainer.java rename to libs/common/src/main/java/org/opensearch/common/io/InputStreamContainer.java index ce5dcff9f5349..eb8a4e1382497 100644 --- a/server/src/main/java/org/opensearch/common/io/InputStreamContainer.java +++ b/libs/common/src/main/java/org/opensearch/common/io/InputStreamContainer.java @@ -19,6 +19,7 @@ public class InputStreamContainer { private final InputStream inputStream; private final long contentLength; + private final long offset; /** * Construct a new stream object @@ -26,9 +27,10 @@ public class InputStreamContainer { * @param inputStream The input stream that is to be encapsulated * @param contentLength The total content length that is to be read from the stream */ - public InputStreamContainer(InputStream inputStream, long contentLength) { + public InputStreamContainer(InputStream inputStream, long contentLength, long offset) { this.inputStream = inputStream; this.contentLength = contentLength; + this.offset = offset; } /** @@ -44,4 +46,11 @@ public InputStream getInputStream() { public long getContentLength() { return contentLength; } + + /** + * @return offset of the source content. + */ + public long getOffset() { + return offset; + } } 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 7864c3ab5c794..ca744efae902d 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 @@ -160,7 +160,7 @@ private LocalStreamSupplier getMultipartStreamSupplier( : offsetRangeInputStream; Objects.requireNonNull(inputStreams.get())[streamIdx] = inputStream; - return new InputStreamContainer(inputStream, size); + return new InputStreamContainer(inputStream, size, position); } catch (IOException e) { log.error("Failed to create input stream", e); throw e; From 74181c58762bb45f40ee88db84578b90d25c9224 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Wed, 5 Jul 2023 13:22:57 +0530 Subject: [PATCH 17/18] Fixed naming for MockFsBlobContainerVerifying Signed-off-by: Raghuvansh Raj --- .../remotestore/multipart/mocks/MockFsBlobStore.java | 2 +- ...tainerVerifying.java => MockFsVerifyingBlobContainer.java} | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) rename server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/{MockFsBlobContainerVerifying.java => MockFsVerifyingBlobContainer.java} (97%) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java index ac1c486663531..f1d9fbba84528 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobStore.java @@ -28,7 +28,7 @@ public MockFsBlobStore(int bufferSizeInBytes, Path path, boolean readonly, boole @Override public BlobContainer blobContainer(BlobPath path) { try { - return new MockFsBlobContainerVerifying(this, path, buildAndCreate(path), triggerDataIntegrityFailure); + return new MockFsVerifyingBlobContainer(this, path, buildAndCreate(path), triggerDataIntegrityFailure); } catch (IOException ex) { throw new OpenSearchException("failed to create blob container", ex); } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java similarity index 97% rename from server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java rename to server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java index 4749db3cc55ae..8f2814eb7c4c4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsBlobContainerVerifying.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java @@ -28,13 +28,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -public class MockFsBlobContainerVerifying extends FsBlobContainer implements VerifyingMultiStreamBlobContainer { +public class MockFsVerifyingBlobContainer extends FsBlobContainer implements VerifyingMultiStreamBlobContainer { private static final int TRANSFER_TIMEOUT_MILLIS = 30000; private final boolean triggerDataIntegrityFailure; - public MockFsBlobContainerVerifying(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { + public MockFsVerifyingBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, boolean triggerDataIntegrityFailure) { super(blobStore, blobPath, path); this.triggerDataIntegrityFailure = triggerDataIntegrityFailure; } From aad413ed62e5c95eaa7f9c548589d46ea277c51c Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Tue, 11 Jul 2023 17:23:22 +0530 Subject: [PATCH 18/18] Added overloaded uploadBlob methods in BlobStoreTransferService Signed-off-by: Raghuvansh Raj --- .../index/translog/transfer/BlobStoreTransferService.java | 4 ++-- .../opensearch/index/translog/transfer/TransferService.java | 2 +- .../translog/transfer/BlobStoreTransferServiceTests.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index 873e96a76ef2d..974e8af42b939 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -54,7 +54,7 @@ public BlobStoreTransferService(BlobStore blobStore, ThreadPool threadPool) { } @Override - public void uploadBlobByThreadPool( + public void uploadBlob( String threadPoolName, final TransferFileSnapshot fileSnapshot, Iterable remoteTransferPath, @@ -93,7 +93,7 @@ public void uploadBlobs( fileSnapshots.forEach(fileSnapshot -> { BlobPath blobPath = blobPaths.get(fileSnapshot.getPrimaryTerm()); if (!(blobStore.blobContainer(blobPath) instanceof VerifyingMultiStreamBlobContainer)) { - uploadBlobByThreadPool(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); + uploadBlob(ThreadPool.Names.TRANSLOG_TRANSFER, fileSnapshot, blobPath, listener, writePriority); } else { uploadBlob(fileSnapshot, listener, blobPath, writePriority); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java index f74517e992782..a240fd38cda11 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java @@ -34,7 +34,7 @@ public interface TransferService { * @param remotePath the remote path where upload should be made * @param listener the callback to be invoked once upload completes successfully/fails */ - void uploadBlobByThreadPool( + void uploadBlob( String threadPoolName, final TransferFileSnapshot fileSnapshot, Iterable remotePath, diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index 684a5a2152925..5502dc3089c62 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -80,7 +80,7 @@ public void testUploadBlobAsync() throws IOException, InterruptedException { ); CountDownLatch latch = new CountDownLatch(1); TransferService transferService = new BlobStoreTransferService(repository.blobStore(), threadPool); - transferService.uploadBlobByThreadPool( + transferService.uploadBlob( ThreadPool.Names.TRANSLOG_TRANSFER, transferFileSnapshot, repository.basePath(),