From d694685c109b189443fde2340599720116c24c4c Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Mon, 9 Jan 2023 19:05:17 +0530 Subject: [PATCH] [Backport 2.x] Adding Remote translog upload, download and failover changes (#5757) * Introduce TranslogFactory for Local/Remote Translog support (#4172) * Introduce TranslogFactory for Local/Remote Translog support Signed-off-by: Bukhtawar Khan * [Remote Translog] Introduce remote translog with upload functionality (#5392) * Introduce remote translog with upload functionality Signed-off-by: Gaurav Bafna Co-authored-by: Bukhtawar Khan * Enable creation of indices using Remote Translog (#5638) * Enable creation of indices using Remote Translog behind a setting and feature flag Signed-off-by: Gaurav Bafna * [Remote Translog] Add support for downloading files from remote translog (#5649) * Add support to download translog from remote store during recovery Signed-off-by: Sachin Kale * Integrate remote translog download on failover (#5699) * Integrate remote translog download on failover Signed-off-by: Ashish Singh Signed-off-by: Bukhtawar Khan Signed-off-by: Gaurav Bafna Signed-off-by: Sachin Kale Signed-off-by: Ashish Singh --- CHANGELOG.md | 5 +- .../opensearch/index/shard/IndexShardIT.java | 2 + .../cluster/metadata/IndexMetadata.java | 40 + .../common/blobstore/fs/FsBlobStore.java | 2 +- .../common/io/stream/BytesStreamInput.java | 15 +- .../common/settings/IndexScopedSettings.java | 3 +- .../org/opensearch/index/IndexModule.java | 8 +- .../org/opensearch/index/IndexService.java | 8 +- .../org/opensearch/index/IndexSettings.java | 10 +- .../opensearch/index/engine/EngineConfig.java | 19 +- .../index/engine/EngineConfigFactory.java | 7 +- .../index/engine/InternalEngine.java | 3 +- .../index/engine/NRTReplicationEngine.java | 3 +- .../opensearch/index/engine/NoOpEngine.java | 28 +- .../index/engine/ReadOnlyEngine.java | 17 +- .../opensearch/index/shard/IndexShard.java | 21 +- .../RemoveCorruptedShardDataCommand.java | 5 + .../opensearch/index/shard/StoreRecovery.java | 33 +- .../translog/InternalTranslogFactory.java | 41 + .../translog/InternalTranslogManager.java | 10 +- .../index/translog/LocalTranslog.java | 160 ++ ...emoteBlobStoreInternalTranslogFactory.java | 72 + .../index/translog/RemoteFsTranslog.java | 285 ++++ .../opensearch/index/translog/Translog.java | 108 +- .../index/translog/TranslogFactory.java | 32 + .../index/translog/TranslogWriter.java | 5 +- .../translog/TruncateTranslogAction.java | 2 +- .../translog/WriteOnlyTranslogManager.java | 6 +- .../transfer/BlobStoreTransferService.java | 11 + .../transfer/FileTransferTracker.java | 91 ++ .../translog/transfer/TransferService.java | 19 + .../TranslogCheckpointTransferSnapshot.java | 30 +- .../transfer/TranslogTransferManager.java | 92 +- .../transfer/TranslogTransferMetadata.java | 46 +- .../opensearch/indices/IndicesService.java | 33 +- .../main/java/org/opensearch/node/Node.java | 7 +- .../org/opensearch/threadpool/ThreadPool.java | 6 + .../opensearch/index/IndexModuleTests.java | 20 +- .../opensearch/index/IndexSettingsTests.java | 52 + .../engine/EngineConfigFactoryTests.java | 7 +- .../index/engine/EngineConfigTests.java | 4 +- .../index/engine/InternalEngineTests.java | 5 +- .../index/shard/IndexShardTests.java | 65 +- ...overyWithRemoteTranslogOnPrimaryTests.java | 1 + .../InternalTranslogManagerTests.java | 21 +- ...slogTests.java => LocalTranslogTests.java} | 222 +-- .../index/translog/RemoteFSTranslogTests.java | 1306 +++++++++++++++++ .../index/translog/TestTranslog.java | 85 ++ .../translog/TranslogManagerTestCase.java | 2 +- .../BlobStoreTransferServiceTests.java | 10 + .../transfer/FileTransferTrackerTests.java | 77 + .../TranslogTransferManagerTests.java | 156 +- ...teStorePeerRecoverySourceHandlerTests.java | 1 + .../snapshots/SnapshotResiliencyTests.java | 9 +- .../threadpool/ScalingThreadPoolTests.java | 1 + .../index/engine/EngineTestCase.java | 5 +- .../index/shard/IndexShardTestCase.java | 31 + 57 files changed, 3020 insertions(+), 345 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/translog/InternalTranslogFactory.java create mode 100644 server/src/main/java/org/opensearch/index/translog/LocalTranslog.java create mode 100644 server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java create mode 100644 server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java create mode 100644 server/src/main/java/org/opensearch/index/translog/TranslogFactory.java create mode 100644 server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java rename server/src/test/java/org/opensearch/index/translog/{TranslogTests.java => LocalTranslogTests.java} (96%) create mode 100644 server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java create mode 100644 server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index a4a777191a35d..e844168e931ed 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,4 @@ + # CHANGELOG All notable changes to this project are documented in this file. @@ -20,7 +21,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add CI bundle pattern to distribution download ([#5348](https://github.com/opensearch-project/OpenSearch/pull/5348)) - Added @gbbafna as an OpenSearch maintainer ([#5668](https://github.com/opensearch-project/OpenSearch/pull/5668)) - Experimental support for extended backward compatiblity in searchable snapshots ([#5429](https://github.com/opensearch-project/OpenSearch/pull/5429)) -- Add support for refresh level durability ([#5253](https://github.com/opensearch-project/OpenSearch/pull/5253)) +- Added Request level Durability using Remote Translog functionality ([#5757](https://github.com/opensearch-project/OpenSearch/pull/5757)) ### Dependencies - Bump bcpg-fips from 1.0.5.1 to 1.0.7.1 ([#5148](https://github.com/opensearch-project/OpenSearch/pull/5148)) @@ -42,7 +43,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Changed - Change http code for DecommissioningFailedException from 500 to 400 ([#5283](https://github.com/opensearch-project/OpenSearch/pull/5283)) - Pre conditions check before updating weighted routing metadata ([#4955](https://github.com/opensearch-project/OpenSearch/pull/4955)) -- Support remote translog transfer for request level durability([#4480](https://github.com/opensearch-project/OpenSearch/pull/4480)) +- Integrate remote segment store in the failover flow ([#5579](https://github.com/opensearch-project/OpenSearch/pull/5579)) ### Deprecated - Refactor fuzziness interface on query builders ([#5433](https://github.com/opensearch-project/OpenSearch/pull/5433)) diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index 5d8855809e1aa..cc2536fcf00e4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -78,6 +78,7 @@ import org.opensearch.index.mapper.SourceToParse; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.TestTranslog; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogStats; @@ -675,6 +676,7 @@ public static final IndexShard newIndexShard( () -> {}, RetentionLeaseSyncer.EMPTY, cbs, + (indexSettings, shardRouting) -> new InternalTranslogFactory(), SegmentReplicationCheckpointPublisher.EMPTY, null ); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 4a3cad9e5a7b2..428b4ef15a043 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -298,6 +298,9 @@ public Iterator> settings() { public static final String SETTING_REMOTE_STORE_REPOSITORY = "index.remote_store.repository"; public static final String SETTING_REMOTE_TRANSLOG_STORE_ENABLED = "index.remote_store.translog.enabled"; + + public static final String SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY = "index.remote_store.translog.repository"; + /** * Used to specify if the index data should be persisted in the remote store. */ @@ -406,6 +409,43 @@ public Iterator> settings() { Property.Final ); + public static final Setting INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING = Setting.simpleString( + SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, + new Setting.Validator<>() { + + @Override + public void validate(final String value) {} + + @Override + public void validate(final String value, final Map, Object> settings) { + if (value == null || value.isEmpty()) { + throw new IllegalArgumentException( + "Setting " + INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey() + " should be provided with non-empty repository ID" + ); + } else { + final Boolean isRemoteTranslogStoreEnabled = (Boolean) settings.get(INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING); + if (isRemoteTranslogStoreEnabled == null || isRemoteTranslogStoreEnabled == false) { + throw new IllegalArgumentException( + "Settings " + + INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey() + + " can only be set/enabled when " + + INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING.getKey() + + " is set to true" + ); + } + } + } + + @Override + public Iterator> settings() { + final List> settings = Collections.singletonList(INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING); + return settings.iterator(); + } + }, + Property.IndexScope, + Property.Final + ); + public static final String SETTING_AUTO_EXPAND_REPLICAS = "index.auto_expand_replicas"; public static final Setting INDEX_AUTO_EXPAND_REPLICAS_SETTING = AutoExpandReplicas.SETTING; diff --git a/server/src/main/java/org/opensearch/common/blobstore/fs/FsBlobStore.java b/server/src/main/java/org/opensearch/common/blobstore/fs/FsBlobStore.java index f25a741b93c8d..6944e01d8ee24 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/fs/FsBlobStore.java +++ b/server/src/main/java/org/opensearch/common/blobstore/fs/FsBlobStore.java @@ -90,7 +90,7 @@ public void close() { // nothing to do here... } - private synchronized Path buildAndCreate(BlobPath path) throws IOException { + protected synchronized Path buildAndCreate(BlobPath path) throws IOException { Path f = buildPath(path); if (readOnly == false) { Files.createDirectories(f); diff --git a/server/src/main/java/org/opensearch/common/io/stream/BytesStreamInput.java b/server/src/main/java/org/opensearch/common/io/stream/BytesStreamInput.java index 1572cd1f500f4..8bf1fe846cd8b 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/BytesStreamInput.java +++ b/server/src/main/java/org/opensearch/common/io/stream/BytesStreamInput.java @@ -80,15 +80,19 @@ public void skipBytes(long count) { pos += count; } - // NOTE: AIOOBE not EOF if you read too much @Override - public byte readByte() { + public byte readByte() throws EOFException { + if (eof()) { + throw new EOFException(); + } return bytes[pos++]; } - // NOTE: AIOOBE not EOF if you read too much @Override - public void readBytes(byte[] b, int offset, int len) { + public void readBytes(byte[] b, int offset, int len) throws EOFException { + if (available() < len) { + throw new EOFException(); + } System.arraycopy(bytes, pos, b, offset, len); pos += len; } @@ -111,6 +115,9 @@ protected void ensureCanReadBytes(int length) throws EOFException { @Override public int read() throws IOException { + if (eof()) { + throw new EOFException(); + } return bytes[pos++] & 0xFF; } diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 079fc38415328..1efce2eba8867 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -225,8 +225,9 @@ public final class IndexScopedSettings extends AbstractScopedSettings { FeatureFlags.REMOTE_STORE, List.of( IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, + IndexMetadata.INDEX_REMOTE_STORE_REPOSITORY_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING, - IndexMetadata.INDEX_REMOTE_STORE_REPOSITORY_SETTING + IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING ), FeatureFlags.SEARCHABLE_SNAPSHOT, List.of( diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 076d0b65a6113..5b9b051af8e8b 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -44,6 +44,7 @@ import org.opensearch.Version; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedFunction; import org.opensearch.common.TriFunction; @@ -71,6 +72,7 @@ import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.FsDirectoryFactory; import org.opensearch.index.store.remote.directory.RemoteSnapshotDirectoryFactory; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.IndicesQueryCache; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -500,7 +502,8 @@ public IndexService newIndexService( NamedWriteableRegistry namedWriteableRegistry, BooleanSupplier idFieldDataEnabled, ValuesSourceRegistry valuesSourceRegistry, - IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory + IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory, + BiFunction translogFactorySupplier ) throws IOException { final IndexEventListener eventListener = freeze(); Function> readerWrapperFactory = indexReaderWrapper @@ -555,7 +558,8 @@ public IndexService newIndexService( allowExpensiveQueries, expressionResolver, valuesSourceRegistry, - recoveryStateFactory + recoveryStateFactory, + translogFactorySupplier ); success = true; return indexService; diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 475129b62177b..78211f12f71ad 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -89,6 +89,7 @@ import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; @@ -113,6 +114,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -172,6 +174,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final IndexNameExpressionResolver expressionResolver; private final Supplier indexSortSupplier; private final ValuesSourceRegistry valuesSourceRegistry; + private final BiFunction translogFactorySupplier; public IndexService( IndexSettings indexSettings, @@ -203,7 +206,8 @@ public IndexService( BooleanSupplier allowExpensiveQueries, IndexNameExpressionResolver expressionResolver, ValuesSourceRegistry valuesSourceRegistry, - IndexStorePlugin.RecoveryStateFactory recoveryStateFactory + IndexStorePlugin.RecoveryStateFactory recoveryStateFactory, + BiFunction translogFactorySupplier ) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; @@ -275,6 +279,7 @@ public IndexService( this.trimTranslogTask = new AsyncTrimTranslogTask(this); this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); + this.translogFactorySupplier = translogFactorySupplier; updateFsyncTaskIfNecessary(); } @@ -547,6 +552,7 @@ public synchronized IndexShard createShard( () -> globalCheckpointSyncer.accept(shardId), retentionLeaseSyncer, circuitBreakerService, + translogFactorySupplier, this.indexSettings.isSegRepEnabled() ? checkpointPublisher : null, remoteStore ); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index b735a9e4e3ea0..878523226f23e 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -586,8 +586,9 @@ public final class IndexSettings { private final int numberOfShards; private final ReplicationType replicationType; private final boolean isRemoteStoreEnabled; - private final String remoteStoreRepository; private final boolean isRemoteTranslogStoreEnabled; + private final String remoteStoreTranslogRepository; + private final String remoteStoreRepository; private final boolean isRemoteSnapshot; private Version extendedCompatibilitySnapshotVersion; // volatile fields are updated via #updateIndexMetadata(IndexMetadata) under lock @@ -750,8 +751,9 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null); replicationType = ReplicationType.parseString(settings.get(IndexMetadata.SETTING_REPLICATION_TYPE)); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); - remoteStoreRepository = settings.get(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY); isRemoteTranslogStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false); + remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); + remoteStoreRepository = settings.get(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY); isRemoteSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match(this.settings); if (isRemoteSnapshot && FeatureFlags.isEnabled(SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY)) { @@ -1040,6 +1042,10 @@ public Version getExtendedCompatibilitySnapshotVersion() { return extendedCompatibilitySnapshotVersion; } + public String getRemoteStoreTranslogRepository() { + return remoteStoreTranslogRepository; + } + /** * Returns the node settings. The settings returned from {@link #getSettings()} are a merged version of the * index settings and the node settings where node settings are overwritten by index settings. diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index 4ae6646ed14f0..ba30103f70269 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -51,8 +51,10 @@ import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.index.shard.ShardId; import org.opensearch.index.store.Store; +import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.TranslogConfig; import org.opensearch.index.translog.TranslogDeletionPolicyFactory; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.IndexingMemoryController; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.threadpool.ThreadPool; @@ -150,6 +152,8 @@ public Supplier retentionLeasesSupplier() { private final TranslogConfig translogConfig; + private final TranslogFactory translogFactory; + public EngineConfig( ShardId shardId, ThreadPool threadPool, @@ -253,7 +257,8 @@ public EngineConfig( retentionLeasesSupplier, primaryTermSupplier, tombstoneDocSupplier, - false + false, + new InternalTranslogFactory() ); } @@ -284,7 +289,8 @@ public EngineConfig( Supplier retentionLeasesSupplier, LongSupplier primaryTermSupplier, TombstoneDocSupplier tombstoneDocSupplier, - boolean isReadOnlyReplica + boolean isReadOnlyReplica, + TranslogFactory translogFactory ) { if (isReadOnlyReplica && indexSettings.isSegRepEnabled() == false) { throw new IllegalArgumentException("Shard can only be wired as a read only replica with Segment Replication enabled"); @@ -328,6 +334,7 @@ public EngineConfig( this.primaryTermSupplier = primaryTermSupplier; this.tombstoneDocSupplier = tombstoneDocSupplier; this.isReadOnlyReplica = isReadOnlyReplica; + this.translogFactory = translogFactory; } /** @@ -532,6 +539,14 @@ public boolean isReadOnlyReplica() { return indexSettings.isSegRepEnabled() && isReadOnlyReplica; } + /** + * Returns the underlying translog factory + * @return the translog factory + */ + public TranslogFactory getTranslogFactory() { + return translogFactory; + } + /** * A supplier supplies tombstone documents which will be used in soft-update methods. * The returned document consists only _uid, _seqno, _term and _version fields; other metadata fields are excluded. diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java index c8aec3570f8b5..f0db086e47816 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java @@ -28,6 +28,7 @@ import org.opensearch.index.store.Store; import org.opensearch.index.translog.TranslogConfig; import org.opensearch.index.translog.TranslogDeletionPolicyFactory; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.plugins.EnginePlugin; import org.opensearch.plugins.PluginsService; @@ -147,7 +148,8 @@ public EngineConfig newEngineConfig( Supplier retentionLeasesSupplier, LongSupplier primaryTermSupplier, EngineConfig.TombstoneDocSupplier tombstoneDocSupplier, - boolean isReadOnlyReplica + boolean isReadOnlyReplica, + TranslogFactory translogFactory ) { CodecService codecServiceToUse = codecService; if (codecService == null && this.codecServiceFactory != null) { @@ -178,7 +180,8 @@ public EngineConfig newEngineConfig( retentionLeasesSupplier, primaryTermSupplier, tombstoneDocSupplier, - isReadOnlyReplica + isReadOnlyReplica, + translogFactory ); } diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index e9c61b3ee3766..3b4c8077c86ef 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -289,7 +289,8 @@ public void onFailure(String reason, Exception ex) { () -> getLocalCheckpointTracker(), translogUUID, new CompositeTranslogEventListener(Arrays.asList(internalTranslogEventListener, translogEventListener), shardId), - this::ensureOpen + this::ensureOpen, + engineConfig.getTranslogFactory() ); this.translogManager = translogManagerRef; this.softDeletesPolicy = newSoftDeletesPolicy(); diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java index f083706df39a0..540f29fb93655 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java @@ -108,7 +108,8 @@ public void onAfterTranslogSync() { } } }, - this + this, + engineConfig.getTranslogFactory() ); this.translogManager = translogManagerRef; } catch (IOException e) { diff --git a/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java b/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java index af2cc10d6a5b1..48727b5c84b39 100644 --- a/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java @@ -44,13 +44,14 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.DocsStats; import org.opensearch.index.store.Store; +import org.opensearch.index.translog.DefaultTranslogDeletionPolicy; +import org.opensearch.index.translog.LocalTranslog; +import org.opensearch.index.translog.NoOpTranslogManager; import org.opensearch.index.translog.Translog; -import org.opensearch.index.translog.TranslogManager; import org.opensearch.index.translog.TranslogConfig; -import org.opensearch.index.translog.TranslogException; -import org.opensearch.index.translog.NoOpTranslogManager; -import org.opensearch.index.translog.DefaultTranslogDeletionPolicy; import org.opensearch.index.translog.TranslogDeletionPolicy; +import org.opensearch.index.translog.TranslogException; +import org.opensearch.index.translog.TranslogManager; import java.io.IOException; import java.io.UncheckedIOException; @@ -195,14 +196,15 @@ public void trimUnreferencedTranslogFiles() throws TranslogException { final TranslogDeletionPolicy translogDeletionPolicy = new DefaultTranslogDeletionPolicy(-1, -1, 0); translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); try ( - Translog translog = new Translog( - translogConfig, - translogUuid, - translogDeletionPolicy, - engineConfig.getGlobalCheckpointSupplier(), - engineConfig.getPrimaryTermSupplier(), - seqNo -> {} - ) + Translog translog = engineConfig.getTranslogFactory() + .newTranslog( + translogConfig, + translogUuid, + translogDeletionPolicy, + engineConfig.getGlobalCheckpointSupplier(), + engineConfig.getPrimaryTermSupplier(), + seqNo -> {} + ) ) { translog.trimUnreferencedReaders(); // refresh the translog stats @@ -253,7 +255,7 @@ public void trimUnreferencedTranslogFiles() { final TranslogDeletionPolicy translogDeletionPolicy = new DefaultTranslogDeletionPolicy(-1, -1, 0); translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( translogConfig, translogUuid, translogDeletionPolicy, diff --git a/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java index c20ddcee6da62..bd4b3136fdde7 100644 --- a/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java @@ -277,14 +277,15 @@ private static TranslogStats translogStats(final EngineConfig config, final Segm final long localCheckpoint = Long.parseLong(infos.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); translogDeletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); try ( - Translog translog = new Translog( - translogConfig, - translogUuid, - translogDeletionPolicy, - config.getGlobalCheckpointSupplier(), - config.getPrimaryTermSupplier(), - seqNo -> {} - ) + Translog translog = config.getTranslogFactory() + .newTranslog( + translogConfig, + translogUuid, + translogDeletionPolicy, + config.getGlobalCheckpointSupplier(), + config.getPrimaryTermSupplier(), + seqNo -> {} + ) ) { return translog.stats(); } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 596cf28c5e94e..a1a755cd34e9b 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -159,6 +159,7 @@ import org.opensearch.index.store.StoreStats; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogConfig; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.index.translog.TranslogStats; import org.opensearch.index.warmer.ShardIndexWarmerService; import org.opensearch.index.warmer.WarmerStats; @@ -205,6 +206,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiConsumer; +import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongSupplier; @@ -320,6 +322,7 @@ Runnable getGlobalCheckpointSyncer() { private volatile boolean useRetentionLeasesInPeerRecovery; private final Store remoteStore; + private final BiFunction translogFactorySupplier; public IndexShard( final ShardRouting shardRouting, @@ -342,6 +345,7 @@ public IndexShard( final Runnable globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final CircuitBreakerService circuitBreakerService, + final BiFunction translogFactorySupplier, @Nullable final SegmentReplicationCheckpointPublisher checkpointPublisher, @Nullable final Store remoteStore ) throws IOException { @@ -428,6 +432,7 @@ public boolean shouldCache(Query query) { this.refreshPendingLocationListener = new RefreshPendingLocationListener(); this.checkpointPublisher = checkpointPublisher; this.remoteStore = remoteStore; + this.translogFactorySupplier = translogFactorySupplier; } public ThreadPool getThreadPool() { @@ -2320,10 +2325,10 @@ public void recoverFromStore(ActionListener listener) { storeRecovery.recoverFromStore(this, listener); } - public void restoreFromRemoteStore(ActionListener listener) { + public void restoreFromRemoteStore(Repository repository, ActionListener listener) { assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard"; StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); - storeRecovery.recoverFromRemoteStore(this, listener); + storeRecovery.recoverFromRemoteStore(this, repository, listener); } public void restoreFromRepository(Repository repository, ActionListener listener) { @@ -3113,7 +3118,14 @@ public void startRecovery( executeRecovery("from store", recoveryState, recoveryListener, this::recoverFromStore); break; case REMOTE_STORE: - executeRecovery("from remote store", recoveryState, recoveryListener, this::restoreFromRemoteStore); + final Repository remoteTranslogRepo; + final String remoteTranslogRepoName = indexSettings.getRemoteStoreTranslogRepository(); + if (remoteTranslogRepoName != null) { + remoteTranslogRepo = repositoriesService.repository(remoteTranslogRepoName); + } else { + remoteTranslogRepo = null; + } + executeRecovery("from remote store", recoveryState, recoveryListener, l -> restoreFromRemoteStore(remoteTranslogRepo, l)); break; case PEER: try { @@ -3359,7 +3371,8 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro replicationTracker::getRetentionLeases, () -> getOperationPrimaryTerm(), tombstoneDocSupplier(), - indexSettings.isSegRepEnabled() && shardRouting.primary() == false + indexSettings.isSegRepEnabled() && shardRouting.primary() == false, + translogFactorySupplier.apply(indexSettings, shardRouting) ); } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommand.java b/server/src/main/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommand.java index c7e380f842fa0..ca679d457c0dc 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommand.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommand.java @@ -191,6 +191,11 @@ protected void findAndProcessShardPath( } final IndexSettings indexSettings = new IndexSettings(indexMetadata, settings); + if (indexSettings.isRemoteTranslogStoreEnabled()) { + // ToDo : Need to revisit corrupt shard recovery strategy for remote store enabled indices + throw new OpenSearchException("tool doesn't work for remote translog enabled indices"); + } + final Index index = indexMetadata.getIndex(); final ShardId shId = new ShardId(index, shardId); diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index ea71520ff561e..47d10513f4aaa 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -60,11 +60,16 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.store.Store; +import org.opensearch.index.translog.RemoteFsTranslog; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.transfer.FileTransferTracker; +import org.opensearch.index.translog.transfer.TranslogTransferManager; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Arrays; @@ -114,13 +119,13 @@ void recoverFromStore(final IndexShard indexShard, ActionListener liste } } - void recoverFromRemoteStore(final IndexShard indexShard, ActionListener listener) { + void recoverFromRemoteStore(final IndexShard indexShard, Repository repository, ActionListener listener) { if (canRecover(indexShard)) { RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); assert recoveryType == RecoverySource.Type.REMOTE_STORE : "expected remote store recovery type but was: " + recoveryType; ActionListener.completeWith(recoveryListener(indexShard, listener), () -> { logger.debug("starting recovery from remote store ..."); - recoverFromRemoteStore(indexShard); + recoverFromRemoteStore(indexShard, repository); return true; }); } else { @@ -435,7 +440,7 @@ private ActionListener recoveryListener(IndexShard indexShard, ActionLi }); } - private void recoverFromRemoteStore(IndexShard indexShard) throws IndexShardRecoveryException { + private void recoverFromRemoteStore(IndexShard indexShard, Repository repository) throws IndexShardRecoveryException { final Store remoteStore = indexShard.remoteStore(); if (remoteStore == null) { throw new IndexShardRecoveryException( @@ -453,9 +458,12 @@ private void recoverFromRemoteStore(IndexShard indexShard) throws IndexShardReco // Download segments from remote segment store indexShard.syncSegmentsFromRemoteSegmentStore(true); - // This creates empty trans-log for now - // ToDo: Add code to restore from remote trans-log - bootstrap(indexShard, store); + if (repository != null) { + syncTranslogFilesFromRemoteTranslog(indexShard, repository); + } else { + bootstrap(indexShard, store); + } + assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; indexShard.recoveryState().getIndex().setFileDetailsComplete(); indexShard.openEngineAndRecoverFromTranslog(); @@ -470,6 +478,19 @@ private void recoverFromRemoteStore(IndexShard indexShard) throws IndexShardReco } } + private void syncTranslogFilesFromRemoteTranslog(IndexShard indexShard, Repository repository) throws IOException { + assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; + FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId); + TranslogTransferManager translogTransferManager = RemoteFsTranslog.buildTranslogTransferManager( + blobStoreRepository, + indexShard.getThreadPool().executor(ThreadPool.Names.TRANSLOG_TRANSFER), + shardId, + fileTransferTracker + ); + RemoteFsTranslog.download(translogTransferManager, indexShard.shardPath().resolveTranslog()); + } + /** * Recovers the state of the shard from the store. */ diff --git a/server/src/main/java/org/opensearch/index/translog/InternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/InternalTranslogFactory.java new file mode 100644 index 0000000000000..a363992203721 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/InternalTranslogFactory.java @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import java.io.IOException; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; + +/** + * Translog Factory for the local on-disk {@link Translog} + * + * @opensearch.internal + */ +public class InternalTranslogFactory implements TranslogFactory { + + @Override + public Translog newTranslog( + TranslogConfig translogConfig, + String translogUUID, + TranslogDeletionPolicy translogDeletionPolicy, + LongSupplier globalCheckpointSupplier, + LongSupplier primaryTermSupplier, + LongConsumer persistedSequenceNumberConsumer + ) throws IOException { + + return new LocalTranslog( + translogConfig, + translogUUID, + translogDeletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer + ); + } +} diff --git a/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java b/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java index cdd4d08579312..34a8aa996bfb2 100644 --- a/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java +++ b/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java @@ -58,7 +58,8 @@ public InternalTranslogManager( Supplier localCheckpointTrackerSupplier, String translogUUID, TranslogEventListener translogEventListener, - LifecycleAware engineLifeCycleAware + LifecycleAware engineLifeCycleAware, + TranslogFactory translogFactory ) throws IOException { this.shardId = shardId; this.readLock = readLock; @@ -71,7 +72,7 @@ public InternalTranslogManager( if (tracker != null) { tracker.markSeqNoAsPersisted(seqNo); } - }, translogUUID); + }, translogUUID, translogFactory); assert translog.getGeneration() != null; this.translog = translog; assert pendingTranslogRecovery.get() == false : "translog recovery can't be pending before we set it"; @@ -337,10 +338,11 @@ protected Translog openTranslog( TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier, LongConsumer persistedSequenceNumberConsumer, - String translogUUID + String translogUUID, + TranslogFactory translogFactory ) throws IOException { - return new Translog( + return translogFactory.newTranslog( translogConfig, translogUUID, translogDeletionPolicy, diff --git a/server/src/main/java/org/opensearch/index/translog/LocalTranslog.java b/server/src/main/java/org/opensearch/index/translog/LocalTranslog.java new file mode 100644 index 0000000000000..404132f45f7cb --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/LocalTranslog.java @@ -0,0 +1,160 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.opensearch.common.util.concurrent.ReleasableLock; +import org.opensearch.core.internal.io.IOUtils; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; + +/** + * A {@link Translog} implementation that creates translog files in local filesystem. + * @opensearch.internal + */ +public class LocalTranslog extends Translog { + + /** + * Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogGeneration} is + * {@code null}. If the generation is {@code null} this method is destructive and will delete all files in the translog path given. If + * the generation is not {@code null}, this method tries to open the given translog generation. The generation is treated as the last + * generation referenced from already committed data. This means all operations that have not yet been committed should be in the + * translog file referenced by this generation. The translog creation will fail if this generation can't be opened. + * + * @param config the configuration of this translog + * @param translogUUID the translog uuid to open, null for a new translog + * @param deletionPolicy an instance of {@link TranslogDeletionPolicy} that controls when a translog file can be safely + * deleted + * @param globalCheckpointSupplier a supplier for the global checkpoint + * @param primaryTermSupplier a supplier for the latest value of primary term of the owning index shard. The latest term value is + * examined and stored in the header whenever a new generation is rolled. It's guaranteed from outside + * that a new generation is rolled when the term is increased. This guarantee allows to us to validate + * and reject operation whose term is higher than the primary term stored in the translog header. + * @param persistedSequenceNumberConsumer a callback that's called whenever an operation with a given sequence number is successfully + * persisted. + */ + public LocalTranslog( + final TranslogConfig config, + final String translogUUID, + TranslogDeletionPolicy deletionPolicy, + final LongSupplier globalCheckpointSupplier, + final LongSupplier primaryTermSupplier, + final LongConsumer persistedSequenceNumberConsumer + ) throws IOException { + super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); + try { + final Checkpoint checkpoint = readCheckpoint(location); + final Path nextTranslogFile = location.resolve(getFilename(checkpoint.generation + 1)); + final Path currentCheckpointFile = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); + // this is special handling for error condition when we create a new writer but we fail to bake + // the newly written file (generation+1) into the checkpoint. This is still a valid state + // we just need to cleanup before we continue + // we hit this before and then blindly deleted the new generation even though we managed to bake it in and then hit this: + // https://discuss.elastic.co/t/cannot-recover-index-because-of-missing-tanslog-files/38336 as an example + // + // For this to happen we must have already copied the translog.ckp file into translog-gen.ckp so we first check if that + // file exists. If not we don't even try to clean it up and wait until we fail creating it + assert Files.exists(nextTranslogFile) == false || Files.size(nextTranslogFile) <= TranslogHeader.headerSizeInBytes(translogUUID) + : "unexpected translog file: [" + nextTranslogFile + "]"; + if (Files.exists(currentCheckpointFile) // current checkpoint is already copied + && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning + logger.warn( + "deleted previously created, but not yet committed, next generation [{}]. This can happen due to a" + + " tragic exception when creating a new generation", + nextTranslogFile.getFileName() + ); + } + this.readers.addAll(recoverFromFiles(checkpoint)); + if (readers.isEmpty()) { + throw new IllegalStateException("at least one reader must be recovered"); + } + boolean success = false; + current = null; + try { + current = createWriter( + checkpoint.generation + 1, + getMinFileGeneration(), + checkpoint.globalCheckpoint, + persistedSequenceNumberConsumer + ); + success = true; + } finally { + // we have to close all the recovered ones otherwise we leak file handles here + // for instance if we have a lot of tlog and we can't create the writer we keep on holding + // on to all the uncommitted tlog files if we don't close + if (success == false) { + IOUtils.closeWhileHandlingException(readers); + } + } + } catch (Exception e) { + // close the opened translog files if we fail to create a new translog... + IOUtils.closeWhileHandlingException(current); + IOUtils.closeWhileHandlingException(readers); + throw e; + } + } + + /** + * Ensures that the given location has be synced / written to the underlying storage. + * + * @return Returns true iff this call caused an actual sync operation otherwise false + */ + @Override + public boolean ensureSynced(Location location) throws IOException { + try (ReleasableLock ignored = readLock.acquire()) { + if (location.generation == current.getGeneration()) { // if we have a new one it's already synced + ensureOpen(); + return current.syncUpTo(location.translogLocation + location.size); + } + } catch (final Exception ex) { + closeOnTragicEvent(ex); + throw ex; + } + return false; + } + + /** + * return stats + */ + @Override + public TranslogStats stats() { + // acquire lock to make the two numbers roughly consistent (no file change half way) + try (ReleasableLock lock = readLock.acquire()) { + long uncommittedGen = getMinGenerationForSeqNo(deletionPolicy.getLocalCheckpointOfSafeCommit() + 1).translogFileGeneration; + return new TranslogStats( + totalOperations(), + sizeInBytes(), + totalOperationsByMinGen(uncommittedGen), + sizeInBytesByMinGen(uncommittedGen), + earliestLastModifiedAge() + ); + } + } + + @Override + public void close() throws IOException { + assert Translog.calledFromOutsideOrViaTragedyClose() + : "Translog.close method is called from inside Translog, but not via closeOnTragicEvent method"; + if (closed.compareAndSet(false, true)) { + try (ReleasableLock lock = writeLock.acquire()) { + try { + current.sync(); + } finally { + closeFilesIfNoPendingRetentionLocks(); + } + } finally { + logger.debug("translog closed"); + } + } + } + +} diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java new file mode 100644 index 0000000000000..0d9e01aef4891 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/RemoteBlobStoreInternalTranslogFactory.java @@ -0,0 +1,72 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryMissingException; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; +import java.util.function.Supplier; + +/** + * Translog Factory for the remotefs translog {@link RemoteFsTranslog} + * + * @opensearch.internal + */ +public class RemoteBlobStoreInternalTranslogFactory implements TranslogFactory { + + private final Repository repository; + + private final ExecutorService executorService; + + public RemoteBlobStoreInternalTranslogFactory( + Supplier repositoriesServiceSupplier, + ThreadPool threadPool, + String repositoryName + ) { + Repository repository; + try { + repository = repositoriesServiceSupplier.get().repository(repositoryName); + } catch (RepositoryMissingException ex) { + throw new IllegalArgumentException("Repository should be created before creating index with remote_store enabled setting", ex); + } + this.repository = repository; + this.executorService = threadPool.executor(ThreadPool.Names.TRANSLOG_TRANSFER); + } + + @Override + public Translog newTranslog( + TranslogConfig config, + String translogUUID, + TranslogDeletionPolicy deletionPolicy, + LongSupplier globalCheckpointSupplier, + LongSupplier primaryTermSupplier, + LongConsumer persistedSequenceNumberConsumer + ) throws IOException { + + assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; + BlobStoreRepository blobStoreRepository = ((BlobStoreRepository) repository); + return new RemoteFsTranslog( + config, + translogUUID, + deletionPolicy, + globalCheckpointSupplier, + primaryTermSupplier, + persistedSequenceNumberConsumer, + blobStoreRepository, + executorService + ); + } +} diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java new file mode 100644 index 0000000000000..629b1bebccac8 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -0,0 +1,285 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.opensearch.common.io.FileSystemUtils; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; +import org.opensearch.common.util.concurrent.ReleasableLock; +import org.opensearch.core.internal.io.IOUtils; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.index.translog.transfer.FileTransferTracker; +import org.opensearch.index.translog.transfer.TransferSnapshot; +import org.opensearch.index.translog.transfer.TranslogCheckpointTransferSnapshot; +import org.opensearch.index.translog.transfer.TranslogTransferManager; +import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; + +/** + * A Translog implementation which syncs local FS with a remote store + * The current impl uploads translog , ckp and metadata to remote store + * for every sync, post syncing to disk. Post that, a new generation is + * created. + * + * @opensearch.internal + */ +public class RemoteFsTranslog extends Translog { + + private final BlobStoreRepository blobStoreRepository; + private final TranslogTransferManager translogTransferManager; + private final FileTransferTracker fileTransferTracker; + private volatile long maxRemoteTranslogGenerationUploaded; + + public RemoteFsTranslog( + TranslogConfig config, + String translogUUID, + TranslogDeletionPolicy deletionPolicy, + LongSupplier globalCheckpointSupplier, + LongSupplier primaryTermSupplier, + LongConsumer persistedSequenceNumberConsumer, + BlobStoreRepository blobStoreRepository, + ExecutorService executorService + ) throws IOException { + super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, persistedSequenceNumberConsumer); + this.blobStoreRepository = blobStoreRepository; + fileTransferTracker = new FileTransferTracker(shardId); + this.translogTransferManager = buildTranslogTransferManager(blobStoreRepository, executorService, shardId, fileTransferTracker); + + try { + download(translogTransferManager, location); + Checkpoint checkpoint = readCheckpoint(location); + this.readers.addAll(recoverFromFiles(checkpoint)); + if (readers.isEmpty()) { + throw new IllegalStateException("at least one reader must be recovered"); + } + boolean success = false; + current = null; + try { + current = createWriter( + checkpoint.generation + 1, + getMinFileGeneration(), + checkpoint.globalCheckpoint, + persistedSequenceNumberConsumer + ); + success = true; + } finally { + // we have to close all the recovered ones otherwise we leak file handles here + // for instance if we have a lot of tlog and we can't create the writer we keep + // on holding + // on to all the uncommitted tlog files if we don't close + if (success == false) { + IOUtils.closeWhileHandlingException(readers); + } + } + } catch (Exception e) { + // close the opened translog files if we fail to create a new translog... + IOUtils.closeWhileHandlingException(current); + IOUtils.closeWhileHandlingException(readers); + throw e; + } + } + + public static void download(TranslogTransferManager translogTransferManager, Path location) throws IOException { + + TranslogTransferMetadata translogMetadata = translogTransferManager.readMetadata(); + if (translogMetadata != null) { + if (Files.notExists(location)) { + Files.createDirectories(location); + } + // Delete translog files on local before downloading from remote + for (Path file : FileSystemUtils.files(location)) { + Files.delete(file); + } + Map generationToPrimaryTermMapper = translogMetadata.getGenerationToPrimaryTermMapper(); + for (long i = translogMetadata.getGeneration(); i >= translogMetadata.getMinTranslogGeneration(); i--) { + String generation = Long.toString(i); + translogTransferManager.downloadTranslog(generationToPrimaryTermMapper.get(generation), generation, location); + } + // We copy the latest generation .ckp file to translog.ckp so that flows that depend on + // existence of translog.ckp file work in the same way + Files.copy( + location.resolve(Translog.getCommitCheckpointFileName(translogMetadata.getGeneration())), + location.resolve(Translog.CHECKPOINT_FILE_NAME) + ); + } + } + + public static TranslogTransferManager buildTranslogTransferManager( + BlobStoreRepository blobStoreRepository, + ExecutorService executorService, + ShardId shardId, + FileTransferTracker fileTransferTracker + ) { + return new TranslogTransferManager( + new BlobStoreTransferService(blobStoreRepository.blobStore(), executorService), + blobStoreRepository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())), + fileTransferTracker + ); + } + + @Override + public boolean ensureSynced(Location location) throws IOException { + try (ReleasableLock ignored = writeLock.acquire()) { + assert location.generation <= current.getGeneration(); + if (location.generation == current.getGeneration()) { + ensureOpen(); + return prepareAndUpload(primaryTermSupplier.getAsLong(), location.generation); + } + } catch (final Exception ex) { + closeOnTragicEvent(ex); + throw ex; + } + return false; + } + + @Override + public void rollGeneration() throws IOException { + syncBeforeRollGeneration(); + if (current.totalOperations() == 0 && primaryTermSupplier.getAsLong() == current.getPrimaryTerm()) { + return; + } + prepareAndUpload(primaryTermSupplier.getAsLong(), null); + } + + private boolean prepareAndUpload(Long primaryTerm, Long generation) throws IOException { + try (Releasable ignored = writeLock.acquire()) { + if (generation == null || generation == current.getGeneration()) { + try { + final TranslogReader reader = current.closeIntoReader(); + readers.add(reader); + copyCheckpointTo(location.resolve(getCommitCheckpointFileName(current.getGeneration()))); + if (closed.get() == false) { + logger.trace("Creating new writer for gen: [{}]", current.getGeneration() + 1); + current = createWriter(current.getGeneration() + 1); + } + } catch (final Exception e) { + tragedy.setTragicException(e); + closeOnTragicEvent(e); + throw e; + } + } else if (generation < current.getGeneration()) { + return false; + } + + // Do we need remote writes in sync fashion ? + // If we don't , we should swallow FileAlreadyExistsException while writing to remote store + // and also verify for same during primary-primary relocation + // Writing remote in sync fashion doesn't hurt as global ckp update + // is not updated in remote translog except in primary to primary recovery. + if (generation == null) { + if (closed.get() == false) { + return upload(primaryTerm, current.getGeneration() - 1); + } else { + return upload(primaryTerm, current.getGeneration()); + } + } else { + return upload(primaryTerm, generation); + } + } + } + + private boolean upload(Long primaryTerm, Long generation) throws IOException { + logger.trace("uploading translog for {} {} ", primaryTerm, generation); + try ( + TranslogCheckpointTransferSnapshot transferSnapshotProvider = new TranslogCheckpointTransferSnapshot.Builder( + primaryTerm, + generation, + location, + readers, + Translog::getCommitCheckpointFileName + ).build() + ) { + Releasable transferReleasable = Releasables.wrap(deletionPolicy.acquireTranslogGen(getMinFileGeneration())); + return translogTransferManager.transferSnapshot(transferSnapshotProvider, new TranslogTransferListener() { + @Override + + public void onUploadComplete(TransferSnapshot transferSnapshot) throws IOException { + transferReleasable.close(); + closeFilesIfNoPendingRetentionLocks(); + maxRemoteTranslogGenerationUploaded = generation; + logger.trace("uploaded translog for {} {} ", primaryTerm, generation); + } + + @Override + public void onUploadFailed(TransferSnapshot transferSnapshot, Exception ex) throws IOException { + transferReleasable.close(); + closeFilesIfNoPendingRetentionLocks(); + if (ex instanceof IOException) { + throw (IOException) ex; + } else { + throw (RuntimeException) ex; + } + } + }); + } + + } + + // Visible for testing + public Set allUploaded() { + return fileTransferTracker.allUploaded(); + } + + private boolean syncToDisk() throws IOException { + try (ReleasableLock lock = readLock.acquire()) { + return current.sync(); + } catch (final Exception ex) { + closeOnTragicEvent(ex); + throw ex; + } + } + + @Override + public void sync() throws IOException { + try { + if (syncToDisk() || syncNeeded()) { + prepareAndUpload(primaryTermSupplier.getAsLong(), null); + } + } catch (final Exception e) { + tragedy.setTragicException(e); + closeOnTragicEvent(e); + throw e; + } + } + + /** + * Returns true if an fsync and/or remote transfer is required to ensure durability of the translogs operations or it's metadata. + */ + public boolean syncNeeded() { + try (ReleasableLock lock = readLock.acquire()) { + return current.syncNeeded() + || (maxRemoteTranslogGenerationUploaded + 1 < this.currentFileGeneration() && current.totalOperations() == 0); + } + } + + @Override + public void close() throws IOException { + assert Translog.calledFromOutsideOrViaTragedyClose() + : "Translog.close method is called from inside Translog, but not via closeOnTragicEvent method"; + if (closed.compareAndSet(false, true)) { + try (ReleasableLock lock = writeLock.acquire()) { + sync(); + } finally { + logger.debug("translog closed"); + closeFilesIfNoPendingRetentionLocks(); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index 7f22ad1bf320d..f5a9faff8bfff 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -112,7 +112,7 @@ * * @opensearch.internal */ -public class Translog extends AbstractIndexShardComponent implements IndexShardComponent, Closeable { +public abstract class Translog extends AbstractIndexShardComponent implements IndexShardComponent, Closeable { /* * TODO @@ -134,21 +134,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC public static final int DEFAULT_HEADER_SIZE_IN_BYTES = TranslogHeader.headerSizeInBytes(UUIDs.randomBase64UUID()); // the list of translog readers is guaranteed to be in order of translog generation - private final List readers = new ArrayList<>(); - private final BigArrays bigArrays; + protected final List readers = new ArrayList<>(); + protected final BigArrays bigArrays; protected final ReleasableLock readLock; protected final ReleasableLock writeLock; - private final Path location; - private TranslogWriter current; + protected final Path location; + protected TranslogWriter current; protected final TragicExceptionHolder tragedy = new TragicExceptionHolder(); - private final AtomicBoolean closed = new AtomicBoolean(); - private final TranslogConfig config; - private final LongSupplier globalCheckpointSupplier; - private final LongSupplier primaryTermSupplier; - private final String translogUUID; - private final TranslogDeletionPolicy deletionPolicy; - private final LongConsumer persistedSequenceNumberConsumer; + protected final AtomicBoolean closed = new AtomicBoolean(); + protected final TranslogConfig config; + protected final LongSupplier globalCheckpointSupplier; + protected final LongSupplier primaryTermSupplier; + protected final String translogUUID; + protected final TranslogDeletionPolicy deletionPolicy; + protected final LongConsumer persistedSequenceNumberConsumer; /** * Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogGeneration} is @@ -190,61 +190,10 @@ public Translog( writeLock = new ReleasableLock(rwl.writeLock()); this.location = config.getTranslogPath(); Files.createDirectories(this.location); - - try { - final Checkpoint checkpoint = readCheckpoint(location); - final Path nextTranslogFile = location.resolve(getFilename(checkpoint.generation + 1)); - final Path currentCheckpointFile = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); - // this is special handling for error condition when we create a new writer but we fail to bake - // the newly written file (generation+1) into the checkpoint. This is still a valid state - // we just need to cleanup before we continue - // we hit this before and then blindly deleted the new generation even though we managed to bake it in and then hit this: - // https://discuss.elastic.co/t/cannot-recover-index-because-of-missing-tanslog-files/38336 as an example - // - // For this to happen we must have already copied the translog.ckp file into translog-gen.ckp so we first check if that - // file exists. If not we don't even try to clean it up and wait until we fail creating it - assert Files.exists(nextTranslogFile) == false || Files.size(nextTranslogFile) <= TranslogHeader.headerSizeInBytes(translogUUID) - : "unexpected translog file: [" + nextTranslogFile + "]"; - if (Files.exists(currentCheckpointFile) // current checkpoint is already copied - && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning - logger.warn( - "deleted previously created, but not yet committed, next generation [{}]. This can happen due to a" - + " tragic exception when creating a new generation", - nextTranslogFile.getFileName() - ); - } - this.readers.addAll(recoverFromFiles(checkpoint)); - if (readers.isEmpty()) { - throw new IllegalStateException("at least one reader must be recovered"); - } - boolean success = false; - current = null; - try { - current = createWriter( - checkpoint.generation + 1, - getMinFileGeneration(), - checkpoint.globalCheckpoint, - persistedSequenceNumberConsumer - ); - success = true; - } finally { - // we have to close all the recovered ones otherwise we leak file handles here - // for instance if we have a lot of tlog and we can't create the writer we keep on holding - // on to all the uncommitted tlog files if we don't close - if (success == false) { - IOUtils.closeWhileHandlingException(readers); - } - } - } catch (Exception e) { - // close the opened translog files if we fail to create a new translog... - IOUtils.closeWhileHandlingException(current); - IOUtils.closeWhileHandlingException(readers); - throw e; - } } /** recover all translog files found on disk */ - private ArrayList recoverFromFiles(Checkpoint checkpoint) throws IOException { + protected ArrayList recoverFromFiles(Checkpoint checkpoint) throws IOException { boolean success = false; ArrayList foundTranslogs = new ArrayList<>(); try (ReleasableLock ignored = writeLock.acquire()) { @@ -255,7 +204,7 @@ private ArrayList recoverFromFiles(Checkpoint checkpoint) throws // the generation id we found in the lucene commit. This gives for better error messages if the wrong // translog was found. for (long i = checkpoint.generation; i >= minGenerationToRecoverFrom; i--) { - Path committedTranslogFile = location.resolve(getFilename(i)); + Path committedTranslogFile = location.resolve(Translog.getFilename(i)); if (Files.exists(committedTranslogFile) == false) { throw new TranslogCorruptedException( committedTranslogFile.toString(), @@ -270,7 +219,7 @@ private ArrayList recoverFromFiles(Checkpoint checkpoint) throws } final Checkpoint readerCheckpoint = i == checkpoint.generation ? checkpoint - : Checkpoint.read(location.resolve(getCommitCheckpointFileName(i))); + : Checkpoint.read(location.resolve(Translog.getCommitCheckpointFileName(i))); final TranslogReader reader = openReader(committedTranslogFile, readerCheckpoint); assert reader.getPrimaryTerm() <= primaryTermSupplier.getAsLong() : "Primary terms go backwards; current term [" + primaryTermSupplier.getAsLong() @@ -287,11 +236,11 @@ private ArrayList recoverFromFiles(Checkpoint checkpoint) throws // when we clean up files, we first update the checkpoint with a new minReferencedTranslog and then delete them; // if we crash just at the wrong moment, it may be that we leave one unreferenced file behind so we delete it if there IOUtils.deleteFilesIgnoringExceptions( - location.resolve(getFilename(minGenerationToRecoverFrom - 1)), - location.resolve(getCommitCheckpointFileName(minGenerationToRecoverFrom - 1)) + location.resolve(Translog.getFilename(minGenerationToRecoverFrom - 1)), + location.resolve(Translog.getCommitCheckpointFileName(minGenerationToRecoverFrom - 1)) ); - Path commitCheckpoint = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); + Path commitCheckpoint = location.resolve(Translog.getCommitCheckpointFileName(checkpoint.generation)); if (Files.exists(commitCheckpoint)) { Checkpoint checkpointFromDisk = Checkpoint.read(commitCheckpoint); if (checkpoint.equals(checkpointFromDisk) == false) { @@ -317,7 +266,7 @@ private ArrayList recoverFromFiles(Checkpoint checkpoint) throws return foundTranslogs; } - private void copyCheckpointTo(Path targetPath) throws IOException { + protected void copyCheckpointTo(Path targetPath) throws IOException { // a temp file to copy checkpoint to - note it must be in on the same FS otherwise atomic move won't work final Path tempFile = Files.createTempFile(location, TRANSLOG_FILE_PREFIX, CHECKPOINT_SUFFIX); boolean tempFileRenamed = false; @@ -383,7 +332,7 @@ public boolean isOpen() { return closed.get() == false; } - private static boolean calledFromOutsideOrViaTragedyClose() { + protected static boolean calledFromOutsideOrViaTragedyClose() { List frames = Stream.of(Thread.currentThread().getStackTrace()).skip(3). // skip getStackTrace, current method // and close method frames limit(10). // limit depth of analysis to 10 frames, it should be enough to catch closing with, e.g. IOUtils @@ -817,7 +766,7 @@ public static String getFilename(long generation) { return TRANSLOG_FILE_PREFIX + generation + TRANSLOG_FILE_SUFFIX; } - static String getCommitCheckpointFileName(long generation) { + public static String getCommitCheckpointFileName(long generation) { return TRANSLOG_FILE_PREFIX + generation + CHECKPOINT_SUFFIX; } @@ -868,18 +817,7 @@ public void trimOperations(long belowTerm, long aboveSeqNo) throws IOException { * * @return Returns true iff this call caused an actual sync operation otherwise false */ - public boolean ensureSynced(Location location) throws IOException { - try (ReleasableLock lock = readLock.acquire()) { - if (location.generation == current.getGeneration()) { // if we have a new one it's already synced - ensureOpen(); - return current.syncUpTo(location.translogLocation + location.size); - } - } catch (final Exception ex) { - closeOnTragicEvent(ex); - throw ex; - } - return false; - } + public abstract boolean ensureSynced(Location location) throws IOException; /** * Ensures that all locations in the given stream have been synced / written to the underlying storage. @@ -1910,7 +1848,7 @@ long getFirstOperationPosition() { // for testing return current.getFirstOperationOffset(); } - private void ensureOpen() { + protected void ensureOpen() { if (closed.get()) { throw new AlreadyClosedException("translog is already closed", tragedy.get()); } diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogFactory.java b/server/src/main/java/org/opensearch/index/translog/TranslogFactory.java new file mode 100644 index 0000000000000..5500bda99808d --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/TranslogFactory.java @@ -0,0 +1,32 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import java.io.IOException; +import java.util.function.LongConsumer; +import java.util.function.LongSupplier; + +/** + * Translog Factory to enable creation of various local on-disk + * and remote store flavors of {@link Translog} + * + * @opensearch.internal + */ +@FunctionalInterface +public interface TranslogFactory { + + Translog newTranslog( + final TranslogConfig config, + final String translogUUID, + final TranslogDeletionPolicy deletionPolicy, + final LongSupplier globalCheckpointSupplier, + final LongSupplier primaryTermSupplier, + final LongConsumer persistedSequenceNumberConsumer + ) throws IOException; +} 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 413975f82678b..178cdc110ec3b 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -359,9 +359,10 @@ synchronized boolean assertNoSeqAbove(long belowTerm, long aboveSeqNo) { * * Note: any exception during the sync process will be interpreted as a tragic exception and the writer will be closed before * raising the exception. + * @return */ - public void sync() throws IOException { - syncUpTo(Long.MAX_VALUE); + public boolean sync() throws IOException { + return syncUpTo(Long.MAX_VALUE); } /** diff --git a/server/src/main/java/org/opensearch/index/translog/TruncateTranslogAction.java b/server/src/main/java/org/opensearch/index/translog/TruncateTranslogAction.java index 33294eb9e7d24..9180f110cc020 100644 --- a/server/src/main/java/org/opensearch/index/translog/TruncateTranslogAction.java +++ b/server/src/main/java/org/opensearch/index/translog/TruncateTranslogAction.java @@ -213,7 +213,7 @@ public long minTranslogGenRequired(List readers, TranslogWriter } }; try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( translogConfig, translogUUID, retainAllTranslogPolicy, diff --git a/server/src/main/java/org/opensearch/index/translog/WriteOnlyTranslogManager.java b/server/src/main/java/org/opensearch/index/translog/WriteOnlyTranslogManager.java index 09f5f38a9f6a9..96a2dd05851c0 100644 --- a/server/src/main/java/org/opensearch/index/translog/WriteOnlyTranslogManager.java +++ b/server/src/main/java/org/opensearch/index/translog/WriteOnlyTranslogManager.java @@ -35,7 +35,8 @@ public WriteOnlyTranslogManager( Supplier localCheckpointTrackerSupplier, String translogUUID, TranslogEventListener translogEventListener, - LifecycleAware engineLifecycleAware + LifecycleAware engineLifecycleAware, + TranslogFactory translogFactory ) throws IOException { super( translogConfig, @@ -47,7 +48,8 @@ public WriteOnlyTranslogManager( localCheckpointTrackerSupplier, translogUUID, translogEventListener, - engineLifecycleAware + engineLifecycleAware, + translogFactory ); } 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 36d9d71217837..3a8e77d4cc1fc 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 @@ -19,6 +19,7 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Set; import java.util.concurrent.ExecutorService; /** @@ -68,4 +69,14 @@ public void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable throw ex; } } + + @Override + public InputStream downloadBlob(Iterable path, String fileName) throws IOException { + return blobStore.blobContainer((BlobPath) path).readBlob(fileName); + } + + @Override + public Set listAll(Iterable path) throws IOException { + return blobStore.blobContainer((BlobPath) path).listBlobs().keySet(); + } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java b/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java new file mode 100644 index 0000000000000..e950be0993e83 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/transfer/FileTransferTracker.java @@ -0,0 +1,91 @@ +/* + * 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.index.shard.ShardId; +import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; +import org.opensearch.index.translog.transfer.listener.FileTransferListener; + +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * FileTransferTracker keeps track of translog files uploaded to the remote translog + */ +public class FileTransferTracker implements FileTransferListener { + + private final ConcurrentHashMap fileTransferTracker; + private final ShardId shardId; + + public FileTransferTracker(ShardId shardId) { + this.shardId = shardId; + this.fileTransferTracker = new ConcurrentHashMap<>(); + } + + @Override + public void onSuccess(TransferFileSnapshot fileSnapshot) { + add(fileSnapshot.getName(), TransferState.SUCCESS); + } + + void add(String file, boolean success) { + TransferState targetState = success ? TransferState.SUCCESS : TransferState.FAILED; + add(file, targetState); + } + + private void add(String file, TransferState targetState) { + fileTransferTracker.compute(file, (k, v) -> { + if (v == null || v.validateNextState(targetState)) { + return targetState; + } + throw new IllegalStateException("Unexpected transfer state " + v + "while setting target to" + targetState); + }); + } + + @Override + public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { + add(fileSnapshot.getName(), TransferState.FAILED); + } + + public Set exclusionFilter(Set original) { + return original.stream() + .filter(fileSnapshot -> fileTransferTracker.get(fileSnapshot.getName()) != TransferState.SUCCESS) + .collect(Collectors.toSet()); + } + + public Set allUploaded() { + Set successFileTransferTracker = new HashSet<>(); + fileTransferTracker.forEach((k, v) -> { + if (v == TransferState.SUCCESS) { + successFileTransferTracker.add(k); + } + }); + return successFileTransferTracker; + } + + /** + * Represents the state of the upload operation + */ + private enum TransferState { + SUCCESS, + FAILED; + + public boolean validateNextState(TransferState target) { + switch (this) { + case FAILED: + return true; + case SUCCESS: + return Objects.equals(SUCCESS, target); + } + return false; + } + } +} 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 ed6c185352833..6a67de99287fd 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 @@ -12,6 +12,8 @@ import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import java.io.IOException; +import java.io.InputStream; +import java.util.Set; /** * Interface for the translog transfer service responsible for interacting with a remote store @@ -40,4 +42,21 @@ void uploadBlobAsync( */ void uploadBlob(final TransferFileSnapshot fileSnapshot, Iterable remotePath) throws IOException; + /** + * Lists the files + * @param path : the path to list + * @return : the lists of files + * @throws IOException the exception while listing the path + */ + Set listAll(Iterable path) throws IOException; + + /** + * + * @param path + * @param fileName + * @return inputstream of the remote file + * @throws IOException the exception while reading the data + */ + InputStream downloadBlob(Iterable path, String fileName) throws IOException; + } 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 30b81627614b7..b34c2282e874f 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 @@ -11,8 +11,10 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.index.translog.TranslogReader; +import java.io.Closeable; import java.io.IOException; import java.nio.file.Path; +import java.util.ArrayList; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -21,16 +23,16 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; +import static org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; import static org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; import static org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; -import static org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; /** * Implementation for a {@link TransferSnapshot} which builds the snapshot from the translog and checkpoint files present on the local-disk * * @opensearch.internal */ -public class TranslogCheckpointTransferSnapshot implements TransferSnapshot { +public class TranslogCheckpointTransferSnapshot implements TransferSnapshot, Closeable { private final Set> translogCheckpointFileInfoTupleSet; private final int size; @@ -69,6 +71,26 @@ public Set getCheckpointFileSnapshots() { return translogCheckpointFileInfoTupleSet.stream().map(Tuple::v2).collect(Collectors.toSet()); } + public void close() throws IOException { + List exceptionList = new ArrayList<>(); + Set fileSnapshots = getTranslogFileSnapshots(); + fileSnapshots.addAll(getCheckpointFileSnapshots()); + + for (FileSnapshot fileSnapshot : fileSnapshots) { + try { + fileSnapshot.close(); + } catch (IOException e) { + exceptionList.add(e); + } + } + + if (!exceptionList.isEmpty()) { + IOException ex = new IOException("IO Exception while closing file snapshots"); + exceptionList.forEach(ex::addSuppressed); + throw ex; + } + } + @Override public String toString() { return new StringBuilder("TranslogTransferSnapshot [").append(" primary term = ") @@ -136,9 +158,11 @@ public TranslogCheckpointTransferSnapshot build() throws IOException { translogTransferSnapshot.setMinTranslogGeneration(highestGenMinTranslogGeneration); assert this.primaryTerm == highestGenPrimaryTerm : "inconsistent primary term"; - assert this.generation == highestGeneration : "inconsistent generation"; + assert this.generation == highestGeneration : " inconsistent generation "; + final long finalHighestGeneration = highestGeneration; assert LongStream.iterate(lowestGeneration, i -> i + 1) .limit(highestGeneration) + .filter(l -> (l <= finalHighestGeneration)) .boxed() .collect(Collectors.toList()) .equals(generations.stream().sorted().collect(Collectors.toList())) == true : "generation gaps found"; 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 02ebab8ed6826..6750eedd86180 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 @@ -11,13 +11,18 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.IndexInput; import org.opensearch.action.ActionListener; import org.opensearch.action.LatchedActionListener; import org.opensearch.common.blobstore.BlobPath; -import org.opensearch.index.translog.transfer.listener.FileTransferListener; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -27,7 +32,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.function.UnaryOperator; import java.util.stream.Collectors; import static org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; @@ -42,23 +46,24 @@ public class TranslogTransferManager { private final TransferService transferService; private final BlobPath remoteBaseTransferPath; - private final FileTransferListener fileTransferListener; - private final UnaryOperator> exclusionFilter; + private final BlobPath remoteMetadataTransferPath; + private final FileTransferTracker fileTransferTracker; private static final long TRANSFER_TIMEOUT_IN_MILLIS = 30000; private static final Logger logger = LogManager.getLogger(TranslogTransferManager.class); + private final static String METADATA_DIR = "metadata"; + public TranslogTransferManager( TransferService transferService, BlobPath remoteBaseTransferPath, - FileTransferListener fileTransferListener, - UnaryOperator> exclusionFilter + FileTransferTracker fileTransferTracker ) { this.transferService = transferService; this.remoteBaseTransferPath = remoteBaseTransferPath; - this.fileTransferListener = fileTransferListener; - this.exclusionFilter = exclusionFilter; + this.remoteMetadataTransferPath = remoteBaseTransferPath.add(METADATA_DIR); + this.fileTransferTracker = fileTransferTracker; } public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTransferListener translogTransferListener) @@ -66,11 +71,16 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans List exceptionList = new ArrayList<>(transferSnapshot.getTranslogTransferMetadata().getCount()); Set toUpload = new HashSet<>(transferSnapshot.getTranslogTransferMetadata().getCount()); try { - toUpload.addAll(exclusionFilter.apply(transferSnapshot.getTranslogFileSnapshots())); - toUpload.addAll(exclusionFilter.apply(transferSnapshot.getCheckpointFileSnapshots())); + toUpload.addAll(fileTransferTracker.exclusionFilter(transferSnapshot.getTranslogFileSnapshots())); + toUpload.addAll(fileTransferTracker.exclusionFilter((transferSnapshot.getCheckpointFileSnapshots()))); + if (toUpload.isEmpty()) { + logger.trace("Nothing to upload for transfer"); + translogTransferListener.onUploadComplete(transferSnapshot); + return true; + } final CountDownLatch latch = new CountDownLatch(toUpload.size()); LatchedActionListener latchedActionListener = new LatchedActionListener<>( - ActionListener.wrap(fileTransferListener::onSuccess, ex -> { + ActionListener.wrap(fileTransferTracker::onSuccess, ex -> { assert ex instanceof FileTransferException; logger.error( () -> new ParameterizedMessage( @@ -80,7 +90,7 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans ex ); FileTransferException e = (FileTransferException) ex; - fileTransferListener.onFailure(e.getFileSnapshot(), ex); + fileTransferTracker.onFailure(e.getFileSnapshot(), ex); exceptionList.add(ex); }), latch @@ -104,15 +114,11 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans throw ex; } if (exceptionList.isEmpty()) { - final TransferFileSnapshot transferFileSnapshot = prepareMetadata(transferSnapshot); - transferService.uploadBlob( - prepareMetadata(transferSnapshot), - remoteBaseTransferPath.add(String.valueOf(transferFileSnapshot.getPrimaryTerm())) - ); + transferService.uploadBlob(prepareMetadata(transferSnapshot), remoteMetadataTransferPath); translogTransferListener.onUploadComplete(transferSnapshot); return true; } else { - Exception ex = new RuntimeException("Failed to upload some files during transfer"); + Exception ex = new IOException("Failed to upload " + exceptionList.size() + " files during transfer"); exceptionList.forEach(ex::addSuppressed); throw ex; } @@ -123,6 +129,52 @@ public boolean transferSnapshot(TransferSnapshot transferSnapshot, TranslogTrans } } + public boolean downloadTranslog(String primaryTerm, String generation, Path location) throws IOException { + logger.info( + "Downloading translog files with: Primary Term = {}, Generation = {}, Location = {}", + primaryTerm, + generation, + location + ); + // Download Checkpoint file from remote to local FS + String ckpFileName = Translog.getCommitCheckpointFileName(Long.parseLong(generation)); + downloadToFS(ckpFileName, location, primaryTerm); + // Download translog file from remote to local FS + String translogFilename = Translog.getFilename(Long.parseLong(generation)); + downloadToFS(translogFilename, location, primaryTerm); + return true; + } + + private void downloadToFS(String fileName, Path location, String primaryTerm) throws IOException { + Path filePath = location.resolve(fileName); + // Here, we always override the existing file if present. + // We need to change this logic when we introduce incremental download + if (Files.exists(filePath)) { + Files.delete(filePath); + } + try (InputStream inputStream = transferService.downloadBlob(remoteBaseTransferPath.add(primaryTerm), fileName)) { + Files.copy(inputStream, filePath); + } + // Mark in FileTransferTracker so that the same files are not uploaded at the time of translog sync + fileTransferTracker.add(fileName, true); + } + + public TranslogTransferMetadata readMetadata() throws IOException { + return transferService.listAll(remoteMetadataTransferPath) + .stream() + .max(TranslogTransferMetadata.METADATA_FILENAME_COMPARATOR) + .map(filename -> { + try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, filename);) { + IndexInput indexInput = new ByteArrayIndexInput("metadata file", inputStream.readAllBytes()); + return new TranslogTransferMetadata(indexInput); + } catch (IOException e) { + logger.error(() -> new ParameterizedMessage("Exception while reading metadata file: {}", filename), e); + return null; + } + }) + .orElse(null); + } + private TransferFileSnapshot prepareMetadata(TransferSnapshot transferSnapshot) throws IOException { Map generationPrimaryTermMap = transferSnapshot.getTranslogFileSnapshots().stream().map(s -> { assert s instanceof TranslogFileSnapshot; @@ -136,12 +188,10 @@ private TransferFileSnapshot prepareMetadata(TransferSnapshot transferSnapshot) ); TranslogTransferMetadata translogTransferMetadata = transferSnapshot.getTranslogTransferMetadata(); translogTransferMetadata.setGenerationToPrimaryTermMapper(new HashMap<>(generationPrimaryTermMap)); - TransferFileSnapshot fileSnapshot = new TransferFileSnapshot( + return new TransferFileSnapshot( translogTransferMetadata.getFileName(), translogTransferMetadata.createMetadataBytes(), translogTransferMetadata.getPrimaryTerm() ); - - return fileSnapshot; } } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java index 0aae773f593fd..243294d85c97d 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java @@ -10,6 +10,7 @@ import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.store.DataOutput; +import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.OutputStreamIndexOutput; import org.apache.lucene.util.SetOnce; import org.opensearch.common.bytes.BytesReference; @@ -17,6 +18,8 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -37,7 +40,7 @@ public class TranslogTransferMetadata { private final long timeStamp; - private final int count; + private int count; private final SetOnce> generationToPrimaryTermMapper = new SetOnce<>(); @@ -49,6 +52,8 @@ public class TranslogTransferMetadata { private static final String METADATA_CODEC = "md"; + public static final Comparator METADATA_FILENAME_COMPARATOR = new MetadataFilenameComparator(); + public TranslogTransferMetadata(long primaryTerm, long generation, long minTranslogGeneration, int count) { this.primaryTerm = primaryTerm; this.generation = generation; @@ -57,6 +62,16 @@ public TranslogTransferMetadata(long primaryTerm, long generation, long minTrans this.count = count; } + public TranslogTransferMetadata(IndexInput indexInput) throws IOException { + CodecUtil.checksumEntireFile(indexInput); + CodecUtil.checkHeader(indexInput, METADATA_CODEC, CURRENT_VERSION, CURRENT_VERSION); + this.primaryTerm = indexInput.readLong(); + this.generation = indexInput.readLong(); + this.minTranslogGeneration = indexInput.readLong(); + this.timeStamp = indexInput.readLong(); + this.generationToPrimaryTermMapper.set(indexInput.readMapOfStrings()); + } + public long getPrimaryTerm() { return primaryTerm; } @@ -77,6 +92,10 @@ public void setGenerationToPrimaryTermMapper(Map generationToPri generationToPrimaryTermMapper.set(generationToPrimaryTermMap); } + public Map getGenerationToPrimaryTermMapper() { + return generationToPrimaryTermMapper.get(); + } + public String getFileName() { return String.join( METADATA_SEPARATOR, @@ -122,6 +141,29 @@ private void write(DataOutput out) throws IOException { out.writeLong(generation); out.writeLong(minTranslogGeneration); out.writeLong(timeStamp); - out.writeMapOfStrings(generationToPrimaryTermMapper.get()); + if (generationToPrimaryTermMapper.get() != null) { + out.writeMapOfStrings(generationToPrimaryTermMapper.get()); + } else { + out.writeMapOfStrings(new HashMap<>()); + } + } + + private static class MetadataFilenameComparator implements Comparator { + @Override + public int compare(String first, String second) { + // Format of metadata filename is ____ + String[] filenameTokens1 = first.split(METADATA_SEPARATOR); + String[] filenameTokens2 = second.split(METADATA_SEPARATOR); + // Here, we are not comparing only primary term and generation. + // Timestamp is not a good measure of comparison in case primary term and generation are same. + for (int i = 0; i < filenameTokens1.length - 1; i++) { + if (filenameTokens1[i].equals(filenameTokens2[i]) == false) { + return Long.compare(Long.parseLong(filenameTokens1[i]), Long.parseLong(filenameTokens2[i])); + } + } + throw new IllegalArgumentException( + "TranslogTransferMetadata files " + first + " and " + second + " have same primary term and generation" + ); + } } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 9f14ca7d013b2..f30ad2e2244b3 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -134,6 +134,9 @@ import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.IndexingStats; import org.opensearch.index.shard.ShardId; +import org.opensearch.index.translog.InternalTranslogFactory; +import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.index.translog.TranslogStats; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.indices.cluster.IndicesClusterStateService; @@ -179,10 +182,12 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyList; @@ -271,6 +276,7 @@ public class IndicesService extends AbstractLifecycleComponent private final boolean nodeWriteDanglingIndicesInfo; private final ValuesSourceRegistry valuesSourceRegistry; private final IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory; + private final BiFunction translogFactorySupplier; @Override protected void doStart() { @@ -299,7 +305,8 @@ public IndicesService( Map directoryFactories, ValuesSourceRegistry valuesSourceRegistry, Map recoveryStateFactories, - IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory + IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory, + Supplier repositoriesServiceSupplier ) { this.settings = settings; this.threadPool = threadPool; @@ -388,6 +395,7 @@ protected void closeInternal() { this.allowExpensiveQueries = ALLOW_EXPENSIVE_QUERIES.get(clusterService.getSettings()); clusterService.getClusterSettings().addSettingsUpdateConsumer(ALLOW_EXPENSIVE_QUERIES, this::setAllowExpensiveQueries); this.remoteDirectoryFactory = remoteDirectoryFactory; + this.translogFactorySupplier = getTranslogFactorySupplier(repositoriesServiceSupplier, threadPool); } public IndicesService( @@ -412,7 +420,8 @@ public IndicesService( Map directoryFactories, ValuesSourceRegistry valuesSourceRegistry, Map recoveryStateFactories, - IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory + IndexStorePlugin.RemoteDirectoryFactory remoteDirectoryFactory, + Supplier repositoriesServiceSupplier ) { this.settings = settings; this.threadPool = threadPool; @@ -501,6 +510,23 @@ protected void closeInternal() { this.allowExpensiveQueries = ALLOW_EXPENSIVE_QUERIES.get(clusterService.getSettings()); clusterService.getClusterSettings().addSettingsUpdateConsumer(ALLOW_EXPENSIVE_QUERIES, this::setAllowExpensiveQueries); this.remoteDirectoryFactory = remoteDirectoryFactory; + this.translogFactorySupplier = getTranslogFactorySupplier(repositoriesServiceSupplier, threadPool); + } + + private static BiFunction getTranslogFactorySupplier( + Supplier repositoriesServiceSupplier, + ThreadPool threadPool + ) { + return (indexSettings, shardRouting) -> { + if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { + return new RemoteBlobStoreInternalTranslogFactory( + repositoriesServiceSupplier, + threadPool, + indexSettings.getRemoteStoreTranslogRepository() + ); + } + return new InternalTranslogFactory(); + }; } private static final String DANGLING_INDICES_UPDATE_THREAD_NAME = "DanglingIndices#updateTask"; @@ -864,7 +890,8 @@ private synchronized IndexService createIndexService( namedWriteableRegistry, this::isIdFieldDataEnabled, valuesSourceRegistry, - remoteDirectoryFactory + remoteDirectoryFactory, + translogFactorySupplier ); } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 69816db248c66..b05c58e4d23a4 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -665,7 +665,6 @@ protected Node( ); final IndicesService indicesService; - if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { indicesService = new IndicesService( settings, @@ -689,7 +688,8 @@ protected Node( Map.copyOf(directoryFactories), searchModule.getValuesSourceRegistry(), recoveryStateFactories, - remoteDirectoryFactory + remoteDirectoryFactory, + repositoriesServiceReference::get ); } else { indicesService = new IndicesService( @@ -713,7 +713,8 @@ protected Node( Map.copyOf(directoryFactories), searchModule.getValuesSourceRegistry(), recoveryStateFactories, - remoteDirectoryFactory + remoteDirectoryFactory, + repositoriesServiceReference::get ); } diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index bf4ff89fdad3e..3cc73c864a5c9 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -107,6 +107,7 @@ public static class Names { public static final String FETCH_SHARD_STORE = "fetch_shard_store"; public static final String SYSTEM_READ = "system_read"; public static final String SYSTEM_WRITE = "system_write"; + public static final String TRANSLOG_TRANSFER = "translog_transfer"; } /** @@ -171,6 +172,7 @@ public static ThreadPoolType fromType(String type) { map.put(Names.SEARCH_THROTTLED, ThreadPoolType.FIXED_AUTO_QUEUE_SIZE); map.put(Names.SYSTEM_READ, ThreadPoolType.FIXED); map.put(Names.SYSTEM_WRITE, ThreadPoolType.FIXED); + map.put(Names.TRANSLOG_TRANSFER, ThreadPoolType.SCALING); THREAD_POOL_TYPES = Collections.unmodifiableMap(map); } @@ -255,6 +257,10 @@ public ThreadPool( ); builders.put(Names.SYSTEM_READ, new FixedExecutorBuilder(settings, Names.SYSTEM_READ, halfProcMaxAt5, 2000, false)); builders.put(Names.SYSTEM_WRITE, new FixedExecutorBuilder(settings, Names.SYSTEM_WRITE, halfProcMaxAt5, 1000, false)); + builders.put( + Names.TRANSLOG_TRANSFER, + new ScalingExecutorBuilder(Names.TRANSLOG_TRANSFER, 1, halfProcMaxAt10, TimeValue.timeValueMinutes(5)) + ); for (final ExecutorBuilder builder : customBuilders) { if (builders.containsKey(builder.name())) { diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 6bfdd9ae16773..d2374e767639c 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -44,6 +44,7 @@ import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.store.Directory; +import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce.AlreadySetException; import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; @@ -90,6 +91,9 @@ import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.FsDirectoryFactory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; +import org.opensearch.index.translog.InternalTranslogFactory; +import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.IndicesModule; import org.opensearch.indices.IndicesQueryCache; import org.opensearch.indices.analysis.AnalysisModule; @@ -120,6 +124,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiFunction; import static java.util.Collections.emptyMap; import static java.util.Collections.singletonMap; @@ -217,6 +222,18 @@ public void tearDown() throws Exception { } private IndexService newIndexService(IndexModule module) throws IOException { + final SetOnce repositoriesServiceReference = new SetOnce<>(); + repositoriesServiceReference.set(repositoriesService); + BiFunction translogFactorySupplier = (indexSettings, shardRouting) -> { + if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { + return new RemoteBlobStoreInternalTranslogFactory( + repositoriesServiceReference::get, + threadPool, + indexSettings.getRemoteStoreTranslogRepository() + ); + } + return new InternalTranslogFactory(); + }; return module.newIndexService( CREATE_INDEX, nodeEnvironment, @@ -234,7 +251,8 @@ private IndexService newIndexService(IndexModule module) throws IOException { writableRegistry(), () -> false, null, - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService) + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService), + translogFactorySupplier ); } diff --git a/server/src/test/java/org/opensearch/index/IndexSettingsTests.java b/server/src/test/java/org/opensearch/index/IndexSettingsTests.java index e0f1066587e5b..1b5ef7369419f 100644 --- a/server/src/test/java/org/opensearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/opensearch/index/IndexSettingsTests.java @@ -989,4 +989,56 @@ public void testExtendedCompatibilityVersionWithoutFeatureFlag() { assertTrue(settings.isRemoteSnapshot()); assertEquals(Version.CURRENT.minimumIndexCompatibilityVersion(), settings.getExtendedCompatibilitySnapshotVersion()); } + + public void testSetRemoteTranslogRepositoryFailsWhenRemoteTranslogIsNotEnabled() { + Settings indexSettings = Settings.builder() + .put("index.replication.type", ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "repo1") + .build(); + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.get(indexSettings) + ); + assertEquals( + "Settings index.remote_store.translog.repository can only be set/enabled when index.remote_store.translog.enabled is set to true", + iae.getMessage() + ); + } + + public void testRemoteTranslogRepoDefaultSetting() { + IndexMetadata metadata = newIndexMeta( + "index", + Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build() + ); + IndexSettings settings = new IndexSettings(metadata, Settings.EMPTY); + assertNull(settings.getRemoteStoreRepository()); + } + + public void testRemoteTranslogExplicitSetting() { + IndexMetadata metadata = newIndexMeta( + "index", + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "tlog-store") + .build() + ); + IndexSettings settings = new IndexSettings(metadata, Settings.EMPTY); + assertNull(settings.getRemoteStoreRepository()); + assertEquals("tlog-store", settings.getRemoteStoreTranslogRepository()); + } + + public void testSetRemoteTranslogRepositoryFailsWhenEmptyString() { + Settings indexSettings = Settings.builder() + .put("index.replication.type", ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "") + .build(); + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.get(indexSettings) + ); + assertEquals("Setting index.remote_store.translog.repository should be provided with non-empty repository ID", iae.getMessage()); + } } diff --git a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java index 7ddd92ea7b36e..269d89352fb18 100644 --- a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java @@ -16,6 +16,7 @@ import org.opensearch.index.codec.CodecService; import org.opensearch.index.codec.CodecServiceFactory; import org.opensearch.index.seqno.RetentionLeases; +import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.TranslogDeletionPolicy; import org.opensearch.index.translog.TranslogDeletionPolicyFactory; import org.opensearch.index.translog.TranslogReader; @@ -66,7 +67,8 @@ public void testCreateEngineConfigFromFactory() { () -> new RetentionLeases(0, 0, Collections.emptyList()), null, null, - false + false, + new InternalTranslogFactory() ); assertNotNull(config.getCodec()); @@ -143,7 +145,8 @@ public void testCreateCodecServiceFromFactory() { () -> new RetentionLeases(0, 0, Collections.emptyList()), null, null, - false + false, + new InternalTranslogFactory() ); assertNotNull(config.getCodec()); } diff --git a/server/src/test/java/org/opensearch/index/engine/EngineConfigTests.java b/server/src/test/java/org/opensearch/index/engine/EngineConfigTests.java index 1c6d06e9bcc08..1754d6082b86d 100644 --- a/server/src/test/java/org/opensearch/index/engine/EngineConfigTests.java +++ b/server/src/test/java/org/opensearch/index/engine/EngineConfigTests.java @@ -13,6 +13,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.seqno.RetentionLeases; +import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.IndexSettingsModule; import org.opensearch.test.OpenSearchTestCase; @@ -102,7 +103,8 @@ private EngineConfig createReadOnlyEngine(IndexSettings indexSettings) { () -> RetentionLeases.EMPTY, null, null, - true + true, + new InternalTranslogFactory() ); } } diff --git a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java index 593d09cd17da8..454ef73bb520a 100644 --- a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java @@ -141,12 +141,13 @@ import org.opensearch.index.shard.ShardUtils; import org.opensearch.index.store.Store; import org.opensearch.index.translog.DefaultTranslogDeletionPolicy; +import org.opensearch.index.translog.LocalTranslog; import org.opensearch.index.translog.SnapshotMatchers; import org.opensearch.index.translog.TestTranslog; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogConfig; -import org.opensearch.index.translog.TranslogException; import org.opensearch.index.translog.TranslogDeletionPolicyFactory; +import org.opensearch.index.translog.TranslogException; import org.opensearch.index.translog.listener.TranslogEventListener; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.test.IndexSettingsModule; @@ -3675,7 +3676,7 @@ public void testRecoverFromForeignTranslog() throws IOException { final Path badTranslogLog = createTempDir(); final String badUUID = Translog.createEmptyTranslog(badTranslogLog, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); - Translog translog = new Translog( + Translog translog = new LocalTranslog( new TranslogConfig(shardId, badTranslogLog, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE), badUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 163f60e347b5f..45c41a64d98b0 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -130,6 +130,8 @@ import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreStats; import org.opensearch.index.store.StoreUtils; +import org.opensearch.index.translog.InternalTranslogFactory; +import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.TestTranslog; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogStats; @@ -2751,7 +2753,7 @@ public void testRestoreShardFromRemoteStore(boolean performFlush) throws IOExcep DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); target.markAsRecovering("remote_store", new RecoveryState(routing, localNode, null)); final PlainActionFuture future = PlainActionFuture.newFuture(); - target.restoreFromRemoteStore(future); + target.restoreFromRemoteStore(null, future); target.remoteStore().decRef(); assertTrue(future.actionGet()); @@ -4591,6 +4593,67 @@ public void testReadOnlyReplicaEngineConfig() throws IOException { closeShards(primaryShard, replicaShard); } + public void testTranslogFactoryWithoutRemoteStore() throws IOException { + Settings primarySettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + final IndexShard primaryShard = newStartedShard(true, primarySettings, new NRTReplicationEngineFactory()); + assertEquals(primaryShard.getEngine().getClass(), InternalEngine.class); + assertEquals(primaryShard.getEngine().config().getTranslogFactory().getClass(), InternalTranslogFactory.class); + + final IndexShard replicaShard = newStartedShard(true, primarySettings, new NRTReplicationEngineFactory()); + assertEquals(replicaShard.getEngine().getClass(), InternalEngine.class); + assertEquals(replicaShard.getEngine().config().getTranslogFactory().getClass(), InternalTranslogFactory.class); + + closeShards(primaryShard, replicaShard); + } + + public void testTranslogFactoryForReplicaShardWithoutRemoteStore() throws IOException { + Settings primarySettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + final IndexShard primaryShard = newStartedShard(false, primarySettings, new NRTReplicationEngineFactory()); + assertEquals(primaryShard.getEngine().getClass(), InternalEngine.class); + assertEquals(primaryShard.getEngine().config().getTranslogFactory().getClass(), InternalTranslogFactory.class); + closeShards(primaryShard); + } + + public void testTranslogFactoryForRemoteTranslogBackedPrimaryShard() throws IOException { + Settings primarySettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "seg-test") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "txlog-test") + .build(); + final IndexShard primaryShard = newStartedShard(true, primarySettings, new NRTReplicationEngineFactory()); + assertEquals(primaryShard.getEngine().getClass(), InternalEngine.class); + assertEquals(primaryShard.getEngine().config().getTranslogFactory().getClass(), RemoteBlobStoreInternalTranslogFactory.class); + closeShards(primaryShard); + } + + public void testTranslogFactoryForRemoteTranslogBackedReplicaShard() throws IOException { + Settings primarySettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "seg-test") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "txlog-test") + .build(); + final IndexShard replicaShard = newStartedShard(false, primarySettings, new NRTReplicationEngineFactory()); + assertEquals(replicaShard.getEngine().getClass(), InternalEngine.class); + assertEquals(replicaShard.getEngine().config().getTranslogFactory().getClass(), InternalTranslogFactory.class); + closeShards(replicaShard); + } + public void testCloseShardWhileEngineIsWarming() throws Exception { CountDownLatch warmerStarted = new CountDownLatch(1); CountDownLatch warmerBlocking = new CountDownLatch(1); diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 39030efae878a..12c3305e5f051 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -35,6 +35,7 @@ public class ReplicaRecoveryWithRemoteTranslogOnPrimaryTests extends OpenSearchI .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "translog-repo") .build(); public void testStartSequenceForReplicaRecovery() throws Exception { diff --git a/server/src/test/java/org/opensearch/index/translog/InternalTranslogManagerTests.java b/server/src/test/java/org/opensearch/index/translog/InternalTranslogManagerTests.java index 5171f0dfa1d18..234abfba66622 100644 --- a/server/src/test/java/org/opensearch/index/translog/InternalTranslogManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/InternalTranslogManagerTests.java @@ -47,7 +47,8 @@ public void testRecoveryFromTranslog() throws IOException { () -> tracker, translogUUID, TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER, - () -> {} + () -> {}, + new InternalTranslogFactory() ); final int docs = randomIntBetween(1, 100); for (int i = 0; i < docs; i++) { @@ -85,7 +86,8 @@ public void onBeginTranslogRecovery() { beginTranslogRecoveryInvoked.set(true); } }, - () -> {} + () -> {}, + new InternalTranslogFactory() ); AtomicInteger opsRecovered = new AtomicInteger(); int opsRecoveredFromTranslog = translogManager.recoverFromTranslog((snapshot) -> { @@ -122,7 +124,8 @@ public void testTranslogRollsGeneration() throws IOException { () -> tracker, translogUUID, TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER, - () -> {} + () -> {}, + new InternalTranslogFactory() ); final int docs = randomIntBetween(1, 100); for (int i = 0; i < docs; i++) { @@ -150,7 +153,8 @@ public void testTranslogRollsGeneration() throws IOException { () -> new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED), translogUUID, TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER, - () -> {} + () -> {}, + new InternalTranslogFactory() ); AtomicInteger opsRecovered = new AtomicInteger(); int opsRecoveredFromTranslog = translogManager.recoverFromTranslog((snapshot) -> { @@ -183,7 +187,8 @@ public void testTrimOperationsFromTranslog() throws IOException { () -> tracker, translogUUID, TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER, - () -> {} + () -> {}, + new InternalTranslogFactory() ); final int docs = randomIntBetween(1, 100); for (int i = 0; i < docs; i++) { @@ -213,7 +218,8 @@ public void testTrimOperationsFromTranslog() throws IOException { () -> new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED), translogUUID, TranslogEventListener.NOOP_TRANSLOG_EVENT_LISTENER, - () -> {} + () -> {}, + new InternalTranslogFactory() ); AtomicInteger opsRecovered = new AtomicInteger(); int opsRecoveredFromTranslog = translogManager.recoverFromTranslog((snapshot) -> { @@ -260,7 +266,8 @@ public void onAfterTranslogSync() { } } }, - () -> {} + () -> {}, + new InternalTranslogFactory() ); translogManagerAtomicReference.set(translogManager); Engine.Index index = indexForDoc(doc); diff --git a/server/src/test/java/org/opensearch/index/translog/TranslogTests.java b/server/src/test/java/org/opensearch/index/translog/LocalTranslogTests.java similarity index 96% rename from server/src/test/java/org/opensearch/index/translog/TranslogTests.java rename to server/src/test/java/org/opensearch/index/translog/LocalTranslogTests.java index 153677e00c22b..3033c0eeda564 100644 --- a/server/src/test/java/org/opensearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/opensearch/index/translog/LocalTranslogTests.java @@ -113,7 +113,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.Deque; import java.util.HashMap; import java.util.HashSet; @@ -165,7 +164,7 @@ import static org.mockito.Mockito.when; @LuceneTestCase.SuppressFileSystems("ExtrasFS") -public class TranslogTests extends OpenSearchTestCase { +public class LocalTranslogTests extends OpenSearchTestCase { protected final ShardId shardId = new ShardId("index", "_na_", 1); @@ -217,7 +216,7 @@ protected Translog createTranslog(TranslogConfig config) throws IOException { shardId, primaryTerm.get() ); - return new Translog( + return new LocalTranslog( config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()), @@ -228,7 +227,7 @@ protected Translog createTranslog(TranslogConfig config) throws IOException { } protected Translog openTranslog(TranslogConfig config, String translogUUID) throws IOException { - return new Translog( + return new LocalTranslog( config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()), @@ -264,7 +263,7 @@ private Translog create(Path path) throws IOException { final TranslogConfig translogConfig = getTranslogConfig(path); final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings()); final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); - return new Translog( + return new LocalTranslog( translogConfig, translogUUID, deletionPolicy, @@ -852,27 +851,12 @@ private void assertFilePresences(Translog translog) { } - static class LocationOperation implements Comparable { - final Translog.Operation operation; - final Translog.Location location; - - LocationOperation(Translog.Operation operation, Translog.Location location) { - this.operation = operation; - this.location = location; - } - - @Override - public int compareTo(LocationOperation o) { - return location.compareTo(o.location); - } - } - public void testConcurrentWritesWithVaryingSize() throws Throwable { final int opsPerThread = randomIntBetween(10, 200); int threadCount = 2 + randomInt(5); logger.info("testing with [{}] threads, each doing [{}] ops", threadCount, opsPerThread); - final BlockingQueue writtenOperations = new ArrayBlockingQueue<>(threadCount * opsPerThread); + final BlockingQueue writtenOperations = new ArrayBlockingQueue<>(threadCount * opsPerThread); Thread[] threads = new Thread[threadCount]; final Exception[] threadExceptions = new Exception[threadCount]; @@ -902,10 +886,10 @@ public void testConcurrentWritesWithVaryingSize() throws Throwable { threads[i].join(60 * 1000); } - List collect = new ArrayList<>(writtenOperations); + List collect = new ArrayList<>(writtenOperations); Collections.sort(collect); try (Translog.Snapshot snapshot = translog.newSnapshot()) { - for (LocationOperation locationOperation : collect) { + for (TestTranslog.LocationOperation locationOperation : collect) { Translog.Operation op = snapshot.next(); assertNotNull(op); Translog.Operation expectedOp = locationOperation.operation; @@ -1323,7 +1307,7 @@ public void testLocationComparison() throws IOException { } assertEquals(max.generation, translog.currentFileGeneration()); - try (Translog.Snapshot snap = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snap = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { Translog.Operation next; Translog.Operation maxOp = null; while ((next = snap.next()) != null) { @@ -1515,7 +1499,7 @@ public int write(ByteBuffer src) throws IOException { ); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, new DefaultTranslogDeletionPolicy(-1, -1, 0), @@ -1630,7 +1614,7 @@ public void force(boolean metaData) throws IOException { ); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, new DefaultTranslogDeletionPolicy(-1, -1, 0), @@ -1736,7 +1720,7 @@ public void testBasicRecovery() throws IOException { assertNull(snapshot.next()); } } else { - translog = new Translog( + translog = new LocalTranslog( config, translogGeneration.translogUUID, translog.getDeletionPolicy(), @@ -1795,7 +1779,7 @@ public void testRecoveryUncommitted() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -1811,7 +1795,7 @@ public void testRecoveryUncommitted() throws IOException { translog.currentFileGeneration() ); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snapshot = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; for (int i = 0; i < upTo; i++) { Translog.Operation next = snapshot.next(); @@ -1822,7 +1806,7 @@ public void testRecoveryUncommitted() throws IOException { } if (randomBoolean()) { // recover twice try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -1838,7 +1822,7 @@ public void testRecoveryUncommitted() throws IOException { translog.currentFileGeneration() ); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snapshot = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; for (int i = 0; i < upTo; i++) { Translog.Operation next = snapshot.next(); @@ -1884,7 +1868,7 @@ public void testRecoveryUncommittedFileExists() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -1900,7 +1884,7 @@ public void testRecoveryUncommittedFileExists() throws IOException { translog.currentFileGeneration() ); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snapshot = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; for (int i = 0; i < upTo; i++) { Translog.Operation next = snapshot.next(); @@ -1912,7 +1896,7 @@ public void testRecoveryUncommittedFileExists() throws IOException { if (randomBoolean()) { // recover twice try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -1928,7 +1912,7 @@ public void testRecoveryUncommittedFileExists() throws IOException { translog.currentFileGeneration() ); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snapshot = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; for (int i = 0; i < upTo; i++) { Translog.Operation next = snapshot.next(); @@ -1976,7 +1960,14 @@ public void testRecoveryUncommittedCorruptedCheckpoint() throws IOException { final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); final TranslogCorruptedException translogCorruptedException = expectThrows( TranslogCorruptedException.class, - () -> new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {}) + () -> new LocalTranslog( + config, + translogUUID, + deletionPolicy, + () -> SequenceNumbers.NO_OPS_PERFORMED, + primaryTerm::get, + seqNo -> {} + ) ); assertThat( translogCorruptedException.getMessage(), @@ -1995,7 +1986,7 @@ public void testRecoveryUncommittedCorruptedCheckpoint() throws IOException { StandardOpenOption.TRUNCATE_EXISTING ); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -2011,7 +2002,7 @@ public void testRecoveryUncommittedCorruptedCheckpoint() throws IOException { translog.currentFileGeneration() ); assertFalse(translog.syncNeeded()); - try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { + try (Translog.Snapshot snapshot = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; for (int i = 0; i < upTo; i++) { Translog.Operation next = snapshot.next(); @@ -2156,7 +2147,7 @@ Collection operations() { public void testRandomExceptionsOnTrimOperations() throws Exception { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); fail.failNever(); TranslogConfig config = getTranslogConfig(tempDir); List fileChannels = new ArrayList<>(); @@ -2293,7 +2284,7 @@ public void testOpenForeignTranslog() throws IOException { final String foreignTranslog = randomRealisticUnicodeOfCodepointLengthBetween(1, translogGeneration.translogUUID.length()); try { - new Translog( + new LocalTranslog( config, foreignTranslog, createTranslogDeletionPolicy(), @@ -2305,7 +2296,7 @@ public void testOpenForeignTranslog() throws IOException { } catch (TranslogCorruptedException ex) { } - this.translog = new Translog( + this.translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -2339,7 +2330,7 @@ public void testCloseConcurrently() throws Throwable { int threadCount = 2 + randomInt(5); logger.info("testing with [{}] threads, each doing [{}] ops", threadCount, opsPerThread); - final BlockingQueue writtenOperations = new ArrayBlockingQueue<>(threadCount * opsPerThread); + final BlockingQueue writtenOperations = new ArrayBlockingQueue<>(threadCount * opsPerThread); Thread[] threads = new Thread[threadCount]; final Exception[] threadExceptions = new Exception[threadCount]; @@ -2373,11 +2364,11 @@ public void testCloseConcurrently() throws Throwable { } } - private class TranslogThread extends Thread { + class TranslogThread extends Thread { private final CountDownLatch downLatch; private final int opsPerThread; private final int threadId; - private final Collection writtenOperations; + private final Collection writtenOperations; private final Exception[] threadExceptions; private final Translog translog; private final AtomicLong seqNoGenerator; @@ -2387,7 +2378,7 @@ private class TranslogThread extends Thread { CountDownLatch downLatch, int opsPerThread, int threadId, - Collection writtenOperations, + Collection writtenOperations, AtomicLong seqNoGenerator, Exception[] threadExceptions ) { @@ -2433,7 +2424,7 @@ public void run() { } Translog.Location loc = add(op); - writtenOperations.add(new LocationOperation(op, loc)); + writtenOperations.add(new TestTranslog.LocationOperation(op, loc)); if (rarely()) { // lets verify we can concurrently read this assertEquals(op, translog.readOperation(loc)); } @@ -2453,7 +2444,7 @@ protected void afterAdd() throws IOException {} public void testFailFlush() throws IOException { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); TranslogConfig config = getTranslogConfig(tempDir); Translog translog = getFailableTranslog(fail, config); @@ -2535,7 +2526,7 @@ public void testFailFlush() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); try ( - Translog tlog = new Translog( + Translog tlog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -2599,7 +2590,7 @@ public void testTranslogOpsCountIsCorrect() throws IOException { public void testTragicEventCanBeAnyException() throws IOException { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); TranslogConfig config = getTranslogConfig(tempDir); Translog translog = getFailableTranslog(fail, config, false, true, null, createTranslogDeletionPolicy()); LineFileDocs lineFileDocs = new LineFileDocs(random()); // writes pretty big docs so we cross buffer boarders regularly @@ -2627,7 +2618,7 @@ public void testTragicEventCanBeAnyException() throws IOException { public void testFatalIOExceptionsWhileWritingConcurrently() throws IOException, InterruptedException { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); TranslogConfig config = getTranslogConfig(tempDir); Translog translog = getFailableTranslog(fail, config); @@ -2639,7 +2630,7 @@ public void testFatalIOExceptionsWhileWritingConcurrently() throws IOException, final CountDownLatch downLatch = new CountDownLatch(1); final CountDownLatch added = new CountDownLatch(randomIntBetween(10, 100)); final AtomicLong seqNoGenerator = new AtomicLong(); - List writtenOperations = Collections.synchronizedList(new ArrayList<>()); + List writtenOperations = Collections.synchronizedList(new ArrayList<>()); for (int i = 0; i < threadCount; i++) { final int threadId = i; threads[i] = new TranslogThread(translog, downLatch, 200, threadId, writtenOperations, seqNoGenerator, threadExceptions) { @@ -2692,7 +2683,7 @@ protected void afterAdd() throws IOException { // drop all that haven't been synced writtenOperations.removeIf(next -> checkpoint.offset < (next.location.translogLocation + next.location.size)); try ( - Translog tlog = new Translog( + Translog tlog = new LocalTranslog( config, translogUUID, createTranslogDeletionPolicy(), @@ -2755,7 +2746,7 @@ public void testRecoveryFromAFutureGenerationCleansUp() throws IOException { TranslogConfig config = translog.getConfig(); final TranslogDeletionPolicy deletionPolicy = new DefaultTranslogDeletionPolicy(-1, -1, 0); deletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); - translog = new Translog( + translog = new LocalTranslog( config, translog.getTranslogUUID(), deletionPolicy, @@ -2780,7 +2771,7 @@ public void testRecoveryFromAFutureGenerationCleansUp() throws IOException { */ public void testRecoveryFromFailureOnTrimming() throws IOException { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); fail.failNever(); final TranslogConfig config = getTranslogConfig(tempDir); final long localCheckpoint; @@ -2824,7 +2815,7 @@ public void testRecoveryFromFailureOnTrimming() throws IOException { final TranslogDeletionPolicy deletionPolicy = new DefaultTranslogDeletionPolicy(-1, -1, 0); deletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -2844,46 +2835,12 @@ public void testRecoveryFromFailureOnTrimming() throws IOException { } } - private Translog getFailableTranslog(FailSwitch fail, final TranslogConfig config) throws IOException { + private Translog getFailableTranslog(TestTranslog.FailSwitch fail, final TranslogConfig config) throws IOException { return getFailableTranslog(fail, config, randomBoolean(), false, null, createTranslogDeletionPolicy()); } - private static class FailSwitch { - private volatile int failRate; - private volatile boolean onceFailedFailAlways = false; - - public boolean fail() { - final int rnd = randomIntBetween(1, 100); - boolean fail = rnd <= failRate; - if (fail && onceFailedFailAlways) { - failAlways(); - } - return fail; - } - - public void failNever() { - failRate = 0; - } - - public void failAlways() { - failRate = 100; - } - - public void failRandomly() { - failRate = randomIntBetween(1, 100); - } - - public void failRate(int rate) { - failRate = rate; - } - - public void onceFailedFailAlways() { - onceFailedFailAlways = true; - } - } - private Translog getFailableTranslog( - final FailSwitch fail, + final TestTranslog.FailSwitch fail, final TranslogConfig config, final boolean partialWrites, final boolean throwUnknownException, @@ -2894,7 +2851,7 @@ private Translog getFailableTranslog( } private Translog getFailableTranslog( - final FailSwitch fail, + final TestTranslog.FailSwitch fail, final TranslogConfig config, final boolean partialWrites, final boolean throwUnknownException, @@ -2934,7 +2891,14 @@ private Translog getFailableTranslog( primaryTerm.get() ); } - return new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {}) { + return new LocalTranslog( + config, + translogUUID, + deletionPolicy, + () -> SequenceNumbers.NO_OPS_PERFORMED, + primaryTerm::get, + seqNo -> {} + ) { @Override ChannelFactory getChannelFactory() { return channelFactory; @@ -2953,11 +2917,11 @@ void deleteReaderFiles(TranslogReader reader) { } public static class ThrowingFileChannel extends FilterFileChannel { - private final FailSwitch fail; + private final TestTranslog.FailSwitch fail; private final boolean partialWrite; private final boolean throwUnknownException; - public ThrowingFileChannel(FailSwitch fail, boolean partialWrite, boolean throwUnknownException, FileChannel delegate) + public ThrowingFileChannel(TestTranslog.FailSwitch fail, boolean partialWrite, boolean throwUnknownException, FileChannel delegate) throws MockDirectoryWrapper.FakeIOException { super(delegate); this.fail = fail; @@ -3072,7 +3036,7 @@ public void testFailWhileCreateWriteWithRecoveredTLogs() throws IOException { translog.add(new Translog.Index("boom", 0, primaryTerm.get(), "boom".getBytes(Charset.forName("UTF-8")))); translog.close(); try { - new Translog( + new LocalTranslog( config, translog.getTranslogUUID(), createTranslogDeletionPolicy(), @@ -3140,7 +3104,7 @@ public void testRecoverWithUnbackedNextGenInIllegalState() throws IOException { TranslogException ex = expectThrows( TranslogException.class, - () -> new Translog( + () -> new LocalTranslog( config, translog.getTranslogUUID(), translog.getDeletionPolicy(), @@ -3167,7 +3131,7 @@ public void testRecoverWithUnbackedNextGenAndFutureFile() throws IOException { // we add N+1 and N+2 to ensure we only delete the N+1 file and never jump ahead and wipe without the right condition Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 2) + ".tlog")); try ( - Translog tlog = new Translog( + Translog tlog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -3189,7 +3153,14 @@ public void testRecoverWithUnbackedNextGenAndFutureFile() throws IOException { TranslogException ex = expectThrows( TranslogException.class, - () -> new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {}) + () -> new LocalTranslog( + config, + translogUUID, + deletionPolicy, + () -> SequenceNumbers.NO_OPS_PERFORMED, + primaryTerm::get, + seqNo -> {} + ) ); assertEquals(ex.getMessage(), "failed to create new translog file"); assertEquals(ex.getCause().getClass(), FileAlreadyExistsException.class); @@ -3204,7 +3175,7 @@ public void testWithRandomException() throws IOException { final int runs = randomIntBetween(5, 10); for (int run = 0; run < runs; run++) { Path tempDir = createTempDir(); - final FailSwitch fail = new FailSwitch(); + final TestTranslog.FailSwitch fail = new TestTranslog.FailSwitch(); fail.failRandomly(); TranslogConfig config = getTranslogConfig(tempDir); final int numOps = randomIntBetween(100, 200); @@ -3312,7 +3283,7 @@ public void testWithRandomException() throws IOException { ); } try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, generationUUID, deletionPolicy, @@ -3374,7 +3345,7 @@ public void testCheckpointOnDiskFull() throws IOException { throw new MockDirectoryWrapper.FakeIOException(); } FileChannel open = FileChannel.open(p, o); - FailSwitch failSwitch = new FailSwitch(); + TestTranslog.FailSwitch failSwitch = new TestTranslog.FailSwitch(); failSwitch.failNever(); // don't fail in the ctor ThrowingFileChannel channel = new ThrowingFileChannel(failSwitch, false, false, open); failSwitch.failAlways(); @@ -3407,7 +3378,7 @@ public void testPendingDelete() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(config.getIndexSettings()); translog.close(); - translog = new Translog( + translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -3421,7 +3392,7 @@ public void testPendingDelete() throws IOException { translog.add(new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 3 })); translog.close(); IOUtils.close(lock); - translog = new Translog( + translog = new LocalTranslog( config, translogUUID, deletionPolicy, @@ -3780,7 +3751,7 @@ public void testCloseSnapshotTwice() throws Exception { // close method should never be called directly from Translog (the only exception is closeOnTragicEvent) public void testTranslogCloseInvariant() throws IOException { assumeTrue("test only works with assertions enabled", Assertions.ENABLED); - class MisbehavingTranslog extends Translog { + class MisbehavingTranslog extends LocalTranslog { MisbehavingTranslog( TranslogConfig config, String translogUUID, @@ -3860,41 +3831,6 @@ public void testMaxSeqNo() throws Exception { assertThat(translog.getMaxSeqNo(), equalTo(expectedMaxSeqNo)); } - static class SortedSnapshot implements Translog.Snapshot { - private final Translog.Snapshot snapshot; - private List operations = null; - - SortedSnapshot(Translog.Snapshot snapshot) { - this.snapshot = snapshot; - } - - @Override - public int totalOperations() { - return snapshot.totalOperations(); - } - - @Override - public Translog.Operation next() throws IOException { - if (operations == null) { - operations = new ArrayList<>(); - Translog.Operation op; - while ((op = snapshot.next()) != null) { - operations.add(op); - } - operations.sort(Comparator.comparing(Translog.Operation::seqNo)); - } - if (operations.isEmpty()) { - return null; - } - return operations.remove(0); - } - - @Override - public void close() throws IOException { - snapshot.close(); - } - } - public void testCrashDuringCheckpointCopy() throws IOException { final Path path = createTempDir(); final AtomicBoolean failOnCopy = new AtomicBoolean(); @@ -3923,7 +3859,7 @@ public void copy(Path source, Path target, CopyOption... options) throws IOExcep assertFalse(brokenTranslog.isOpen()); try ( - Translog recoveredTranslog = new Translog( + Translog recoveredTranslog = new LocalTranslog( getTranslogConfig(path), brokenTranslog.getTranslogUUID(), brokenTranslog.getDeletionPolicy(), @@ -3957,7 +3893,7 @@ public void testSyncConcurrently() throws Exception { } }; try ( - Translog translog = new Translog( + Translog translog = new LocalTranslog( config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()), @@ -4038,7 +3974,7 @@ public void force(boolean metaData) throws IOException { channelFactory, primaryTerm.get() ); - final Translog translog = new Translog( + final Translog translog = new LocalTranslog( config, translogUUID, createTranslogDeletionPolicy(), diff --git a/server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java b/server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java new file mode 100644 index 0000000000000..bd9a608e6d85e --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/RemoteFSTranslogTests.java @@ -0,0 +1,1306 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.backward_codecs.store.EndiannessReverserUtil; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.ByteArrayDataOutput; +import org.apache.lucene.store.DataOutput; +import org.apache.lucene.tests.mockfile.FilterFileChannel; +import org.apache.lucene.tests.util.LuceneTestCase; +import org.junit.After; +import org.junit.Before; +import org.opensearch.OpenSearchException; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.RepositoryMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.fs.FsBlobContainer; +import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.bytes.ReleasableBytesReference; +import org.opensearch.common.io.FileSystemUtils; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.ByteSizeUnit; +import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.common.util.concurrent.AbstractRunnable; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.xcontent.NamedXContentRegistry; +import org.opensearch.core.internal.io.IOUtils; +import org.opensearch.env.Environment; +import org.opensearch.env.TestEnvironment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.MissingHistoryOperationsException; +import org.opensearch.index.seqno.LocalCheckpointTracker; +import org.opensearch.index.seqno.LocalCheckpointTrackerTests; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.blobstore.BlobStoreTestUtil; +import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.test.IndexSettingsModule; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.Charset; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.BrokenBarrierException; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.LongConsumer; +import java.util.zip.CRC32; +import java.util.zip.CheckedInputStream; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.opensearch.common.util.BigArrays.NON_RECYCLING_INSTANCE; +import static org.opensearch.index.translog.SnapshotMatchers.containsOperationsInAnyOrder; +import static org.opensearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; + +@LuceneTestCase.SuppressFileSystems("ExtrasFS") + +public class RemoteFSTranslogTests extends OpenSearchTestCase { + + protected final ShardId shardId = new ShardId("index", "_na_", 1); + + protected RemoteFsTranslog translog; + private AtomicLong globalCheckpoint; + protected Path translogDir; + // A default primary term is used by translog instances created in this test. + private final AtomicLong primaryTerm = new AtomicLong(); + private final AtomicReference persistedSeqNoConsumer = new AtomicReference<>(); + private ThreadPool threadPool; + + BlobStoreRepository repository; + + BlobStoreTransferService blobStoreTransferService; + + TestTranslog.FailSwitch fail; + + private LongConsumer getPersistedSeqNoConsumer() { + return seqNo -> { + final LongConsumer consumer = persistedSeqNoConsumer.get(); + if (consumer != null) { + consumer.accept(seqNo); + } + }; + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + primaryTerm.set(randomLongBetween(1, Integer.MAX_VALUE)); + // if a previous test failed we clean up things here + translogDir = createTempDir(); + translog = create(translogDir); + } + + @Override + @After + public void tearDown() throws Exception { + try { + translog.getDeletionPolicy().assertNoOpenTranslogRefs(); + translog.close(); + } finally { + super.tearDown(); + terminate(threadPool); + } + } + + private RemoteFsTranslog create(Path path) throws IOException { + final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get()); + return create(path, createRepository(), translogUUID); + } + + private RemoteFsTranslog create(Path path, BlobStoreRepository repository, String translogUUID) throws IOException { + this.repository = repository; + globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final TranslogConfig translogConfig = getTranslogConfig(path); + final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings()); + threadPool = new TestThreadPool(getClass().getName()); + blobStoreTransferService = new BlobStoreTransferService( + repository.blobStore(), + threadPool.executor(ThreadPool.Names.TRANSLOG_TRANSFER) + ); + return new RemoteFsTranslog( + translogConfig, + translogUUID, + deletionPolicy, + () -> globalCheckpoint.get(), + primaryTerm::get, + getPersistedSeqNoConsumer(), + repository, + threadPool.executor(ThreadPool.Names.TRANSLOG_TRANSFER) + ); + + } + + private TranslogConfig getTranslogConfig(final Path path) { + final Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT) + // only randomize between nog age retention and a long one, so failures will have a chance of reproducing + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), randomBoolean() ? "-1ms" : "1h") + .put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), randomIntBetween(-1, 2048) + "b") + .build(); + return getTranslogConfig(path, settings); + } + + private TranslogConfig getTranslogConfig(final Path path, final Settings settings) { + final ByteSizeValue bufferSize = randomFrom( + TranslogConfig.DEFAULT_BUFFER_SIZE, + new ByteSizeValue(8, ByteSizeUnit.KB), + new ByteSizeValue(10 + randomInt(128 * 1024), ByteSizeUnit.BYTES) + ); + + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(shardId.getIndex(), settings); + return new TranslogConfig(shardId, path, indexSettings, NON_RECYCLING_INSTANCE, bufferSize); + } + + private BlobStoreRepository createRepository() { + Settings settings = Settings.builder().put("location", randomAlphaOfLength(10)).build(); + RepositoryMetadata repositoryMetadata = new RepositoryMetadata(randomAlphaOfLength(10), FsRepository.TYPE, settings); + final ClusterService clusterService = BlobStoreTestUtil.mockClusterService(repositoryMetadata); + fail = new TestTranslog.FailSwitch(); + fail.failNever(); + final FsRepository repository = new ThrowingBlobRepository( + repositoryMetadata, + createEnvironment(), + xContentRegistry(), + clusterService, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + fail + ) { + @Override + protected void assertSnapshotOrGenericThread() { + // eliminate thread name check as we create repo manually + } + }; + clusterService.addStateApplier(event -> repository.updateState(event.state())); + // Apply state once to initialize repo properly like RepositoriesService would + repository.updateState(clusterService.state()); + repository.start(); + return repository; + } + + /** Create a {@link Environment} with random path.home and path.repo **/ + private Environment createEnvironment() { + Path home = createTempDir(); + return TestEnvironment.newEnvironment( + Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), home.toAbsolutePath()) + .put(Environment.PATH_REPO_SETTING.getKey(), home.resolve("repo").toAbsolutePath()) + .build() + ); + } + + private Translog.Location addToTranslogAndList(Translog translog, List list, Translog.Operation op) + throws IOException { + Translog.Location loc = translog.add(op); + Random random = random(); + if (random.nextBoolean()) { + translog.ensureSynced(loc); + } + list.add(op); + return loc; + } + + private Translog.Location addToTranslogAndListAndUpload(Translog translog, List list, Translog.Operation op) + throws IOException { + Translog.Location loc = translog.add(op); + translog.ensureSynced(loc); + list.add(op); + return loc; + } + + public void testSimpleOperations() throws IOException { + ArrayList ops = new ArrayList<>(); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot, SnapshotMatchers.size(0)); + } + + addToTranslogAndList(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); + } + + addToTranslogAndList(translog, ops, new Translog.Delete("2", 1, primaryTerm.get())); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot.totalOperations(), equalTo(ops.size())); + assertThat(snapshot, containsOperationsInAnyOrder(ops)); + } + + final long seqNo = randomLongBetween(0, Integer.MAX_VALUE); + final String reason = randomAlphaOfLength(16); + final long noopTerm = randomLongBetween(1, primaryTerm.get()); + addToTranslogAndList(translog, ops, new Translog.NoOp(seqNo, noopTerm, reason)); + + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot, containsOperationsInAnyOrder(ops)); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); + } + + try (Translog.Snapshot snapshot = translog.newSnapshot(seqNo + 1, randomLongBetween(seqNo + 1, Long.MAX_VALUE))) { + assertThat(snapshot, SnapshotMatchers.size(0)); + assertThat(snapshot.totalOperations(), equalTo(0)); + } + + } + + public void testReadLocation() throws IOException { + ArrayList ops = new ArrayList<>(); + ArrayList locs = new ArrayList<>(); + locs.add(addToTranslogAndList(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 }))); + locs.add(addToTranslogAndList(translog, ops, new Translog.Index("2", 1, primaryTerm.get(), new byte[] { 1 }))); + locs.add(addToTranslogAndList(translog, ops, new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 1 }))); + translog.sync(); + int i = 0; + for (Translog.Operation op : ops) { + assertEquals(op, translog.readOperation(locs.get(i++))); + } + assertNull(translog.readOperation(new Translog.Location(100, 0, 0))); + } + + public void testReadLocationDownload() throws IOException { + ArrayList ops = new ArrayList<>(); + ArrayList locs = new ArrayList<>(); + locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 }))); + locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("2", 1, primaryTerm.get(), new byte[] { 1 }))); + locs.add(addToTranslogAndListAndUpload(translog, ops, new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 1 }))); + translog.sync(); + int i = 0; + for (Translog.Operation op : ops) { + assertEquals(op, translog.readOperation(locs.get(i++))); + } + + String translogUUID = translog.translogUUID; + try { + translog.getDeletionPolicy().assertNoOpenTranslogRefs(); + translog.close(); + } finally { + terminate(threadPool); + } + + // Delete translog files to test download flow + for (Path file : FileSystemUtils.files(translogDir)) { + Files.delete(file); + } + + // Creating RemoteFsTranslog with the same location + Translog newTranslog = create(translogDir, repository, translogUUID); + i = 0; + for (Translog.Operation op : ops) { + assertEquals(op, newTranslog.readOperation(locs.get(i++))); + } + try { + newTranslog.close(); + } catch (Exception e) { + // Ignoring this exception for now. Once the download flow populates FileTracker, + // we can remove this try-catch block + } + } + + public void testSnapshotWithNewTranslog() throws IOException { + List toClose = new ArrayList<>(); + try { + ArrayList ops = new ArrayList<>(); + Translog.Snapshot snapshot = translog.newSnapshot(); + toClose.add(snapshot); + assertThat(snapshot, SnapshotMatchers.size(0)); + + addToTranslogAndList(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); + Translog.Snapshot snapshot1 = translog.newSnapshot(); + toClose.add(snapshot1); + + addToTranslogAndList(translog, ops, new Translog.Index("2", 1, primaryTerm.get(), new byte[] { 2 })); + + assertThat(snapshot1, SnapshotMatchers.equalsTo(ops.get(0))); + + translog.rollGeneration(); + addToTranslogAndList(translog, ops, new Translog.Index("3", 2, primaryTerm.get(), new byte[] { 3 })); + + Translog.Snapshot snapshot2 = translog.newSnapshot(); + toClose.add(snapshot2); + translog.getDeletionPolicy().setLocalCheckpointOfSafeCommit(2); + assertThat(snapshot2, containsOperationsInAnyOrder(ops)); + assertThat(snapshot2.totalOperations(), equalTo(ops.size())); + } finally { + IOUtils.closeWhileHandlingException(toClose); + } + } + + public void testSnapshotOnClosedTranslog() throws IOException { + assertTrue(Files.exists(translogDir.resolve(Translog.getFilename(1)))); + translog.add(new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); + translog.close(); + AlreadyClosedException ex = expectThrows(AlreadyClosedException.class, () -> translog.newSnapshot()); + assertEquals(ex.getMessage(), "translog is already closed"); + } + + public void testRangeSnapshot() throws Exception { + long minSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + long maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + final int generations = between(2, 20); + Map> operationsByGen = new HashMap<>(); + for (int gen = 0; gen < generations; gen++) { + Set seqNos = new HashSet<>(); + int numOps = randomIntBetween(1, 100); + for (int i = 0; i < numOps; i++) { + final long seqNo = randomValueOtherThanMany(seqNos::contains, () -> randomLongBetween(0, 1000)); + minSeqNo = SequenceNumbers.min(minSeqNo, seqNo); + maxSeqNo = SequenceNumbers.max(maxSeqNo, seqNo); + seqNos.add(seqNo); + } + List ops = new ArrayList<>(seqNos.size()); + for (long seqNo : seqNos) { + Translog.Index op = new Translog.Index(randomAlphaOfLength(10), seqNo, primaryTerm.get(), new byte[] { randomByte() }); + translog.add(op); + ops.add(op); + } + operationsByGen.put(translog.currentFileGeneration(), ops); + translog.rollGeneration(); + if (rarely()) { + translog.rollGeneration(); // empty generation + } + } + + if (minSeqNo > 0) { + long fromSeqNo = randomLongBetween(0, minSeqNo - 1); + long toSeqNo = randomLongBetween(fromSeqNo, minSeqNo - 1); + try (Translog.Snapshot snapshot = translog.newSnapshot(fromSeqNo, toSeqNo)) { + assertThat(snapshot.totalOperations(), equalTo(0)); + assertNull(snapshot.next()); + } + } + + long fromSeqNo = randomLongBetween(maxSeqNo + 1, Long.MAX_VALUE); + long toSeqNo = randomLongBetween(fromSeqNo, Long.MAX_VALUE); + try (Translog.Snapshot snapshot = translog.newSnapshot(fromSeqNo, toSeqNo)) { + assertThat(snapshot.totalOperations(), equalTo(0)); + assertNull(snapshot.next()); + } + + fromSeqNo = randomLongBetween(0, 2000); + toSeqNo = randomLongBetween(fromSeqNo, 2000); + try (Translog.Snapshot snapshot = translog.newSnapshot(fromSeqNo, toSeqNo)) { + Set seenSeqNos = new HashSet<>(); + List expectedOps = new ArrayList<>(); + for (long gen = translog.currentFileGeneration(); gen > 0; gen--) { + for (Translog.Operation op : operationsByGen.getOrDefault(gen, Collections.emptyList())) { + if (fromSeqNo <= op.seqNo() && op.seqNo() <= toSeqNo && seenSeqNos.add(op.seqNo())) { + expectedOps.add(op); + } + } + } + assertThat(TestTranslog.drainSnapshot(snapshot, false), equalTo(expectedOps)); + } + } + + public void testSimpleOperationsUpload() throws IOException { + ArrayList ops = new ArrayList<>(); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot, SnapshotMatchers.size(0)); + } + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 0, primaryTerm.get(), new byte[] { 1 })); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + assertThat(snapshot, SnapshotMatchers.equalsTo(ops)); + assertThat(snapshot.totalOperations(), equalTo(ops.size())); + } + + assertEquals(translog.allUploaded().size(), 4); + + addToTranslogAndListAndUpload(translog, ops, new Translog.Index("1", 1, primaryTerm.get(), new byte[] { 1 })); + assertEquals(translog.allUploaded().size(), 6); + + translog.rollGeneration(); + assertEquals(translog.allUploaded().size(), 6); + + Set mdFiles = blobStoreTransferService.listAll( + repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add("metadata") + ); + assertEquals(mdFiles.size(), 2); + logger.info("All md files {}", mdFiles); + + Set tlogFiles = blobStoreTransferService.listAll( + repository.basePath().add(shardId.getIndex().getUUID()).add(String.valueOf(shardId.id())).add(String.valueOf(primaryTerm.get())) + ); + logger.info("All data files {}", tlogFiles); + + // assert content of ckp and tlog files + BlobPath path = repository.basePath() + .add(shardId.getIndex().getUUID()) + .add(String.valueOf(shardId.id())) + .add(String.valueOf(primaryTerm.get())); + for (TranslogReader reader : translog.readers) { + final long readerGeneration = reader.getGeneration(); + logger.error("Asserting content of {}", readerGeneration); + Path translogPath = reader.path(); + try ( + InputStream stream = new CheckedInputStream(Files.newInputStream(translogPath), new CRC32()); + InputStream tlogStream = blobStoreTransferService.downloadBlob(path, Translog.getFilename(readerGeneration)); + ) { + byte[] content = stream.readAllBytes(); + byte[] tlog = tlogStream.readAllBytes(); + assertArrayEquals(tlog, content); + } + + Path checkpointPath = translog.location().resolve(Translog.getCommitCheckpointFileName(readerGeneration)); + try ( + CheckedInputStream stream = new CheckedInputStream(Files.newInputStream(checkpointPath), new CRC32()); + InputStream ckpStream = blobStoreTransferService.downloadBlob(path, Translog.getCommitCheckpointFileName(readerGeneration)) + ) { + byte[] content = stream.readAllBytes(); + byte[] ckp = ckpStream.readAllBytes(); + assertArrayEquals(ckp, content); + } + } + } + + private Long populateTranslogOps(boolean withMissingOps) throws IOException { + long minSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + long maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + final int generations = between(2, 20); + long currentSeqNo = 0L; + List firstGenOps = null; + Map> operationsByGen = new HashMap<>(); + for (int gen = 0; gen < generations; gen++) { + List seqNos = new ArrayList<>(); + int numOps = randomIntBetween(4, 10); + for (int i = 0; i < numOps; i++, currentSeqNo++) { + minSeqNo = SequenceNumbers.min(minSeqNo, currentSeqNo); + maxSeqNo = SequenceNumbers.max(maxSeqNo, currentSeqNo); + seqNos.add(currentSeqNo); + } + Collections.shuffle(seqNos, new Random(100)); + List ops = new ArrayList<>(seqNos.size()); + for (long seqNo : seqNos) { + Translog.Index op = new Translog.Index(randomAlphaOfLength(10), seqNo, primaryTerm.get(), new byte[] { randomByte() }); + boolean shouldAdd = !withMissingOps || seqNo % 4 != 0; + if (shouldAdd) { + translog.add(op); + ops.add(op); + } + } + operationsByGen.put(translog.currentFileGeneration(), ops); + if (firstGenOps == null) { + firstGenOps = ops; + } + translog.rollGeneration(); + if (rarely()) { + translog.rollGeneration(); // empty generation + } + } + return currentSeqNo; + } + + public void testFullRangeSnapshot() throws Exception { + // Successful snapshot + long nextSeqNo = populateTranslogOps(false); + long fromSeqNo = 0L; + long toSeqNo = Math.min(nextSeqNo - 1, fromSeqNo + 15); + try (Translog.Snapshot snapshot = translog.newSnapshot(fromSeqNo, toSeqNo, true)) { + int totOps = 0; + for (Translog.Operation op = snapshot.next(); op != null; op = snapshot.next()) { + totOps++; + } + assertEquals(totOps, toSeqNo - fromSeqNo + 1); + } + } + + public void testFullRangeSnapshotWithFailures() throws Exception { + long nextSeqNo = populateTranslogOps(true); + long fromSeqNo = 0L; + long toSeqNo = Math.min(nextSeqNo - 1, fromSeqNo + 15); + try (Translog.Snapshot snapshot = translog.newSnapshot(fromSeqNo, toSeqNo, true)) { + int totOps = 0; + for (Translog.Operation op = snapshot.next(); op != null; op = snapshot.next()) { + totOps++; + } + fail("Should throw exception for missing operations"); + } catch (MissingHistoryOperationsException e) { + assertTrue(e.getMessage().contains("Not all operations between from_seqno")); + } + } + + public void testConcurrentWritesWithVaryingSize() throws Throwable { + final int opsPerThread = randomIntBetween(10, 200); + int threadCount = 2 + randomInt(5); + + logger.info("testing with [{}] threads, each doing [{}] ops", threadCount, opsPerThread); + final BlockingQueue writtenOperations = new ArrayBlockingQueue<>(threadCount * opsPerThread); + + Thread[] threads = new Thread[threadCount]; + final Exception[] threadExceptions = new Exception[threadCount]; + final AtomicLong seqNoGenerator = new AtomicLong(); + final CountDownLatch downLatch = new CountDownLatch(1); + for (int i = 0; i < threadCount; i++) { + final int threadId = i; + threads[i] = new TranslogThread( + translog, + downLatch, + opsPerThread, + threadId, + writtenOperations, + seqNoGenerator, + threadExceptions + ); + threads[i].setDaemon(true); + threads[i].start(); + } + + downLatch.countDown(); + + for (int i = 0; i < threadCount; i++) { + if (threadExceptions[i] != null) { + throw threadExceptions[i]; + } + threads[i].join(60 * 1000); + } + + List collect = new ArrayList<>(writtenOperations); + collect.sort(Comparator.comparing(op -> op.operation.seqNo())); + + List opsList = new ArrayList<>(threadCount * opsPerThread); + try (Translog.Snapshot snapshot = translog.newSnapshot()) { + for (Translog.Operation op = snapshot.next(); op != null; op = snapshot.next()) { + opsList.add(op); + } + } + opsList.sort(Comparator.comparing(op -> op.seqNo())); + + for (int i = 0; i < threadCount * opsPerThread; i++) { + assertEquals(opsList.get(i), collect.get(i).operation); + } + } + + /** + * Tests that concurrent readers and writes maintain view and snapshot semantics + */ + public void testConcurrentWriteViewsAndSnapshot() throws Throwable { + final Thread[] writers = new Thread[randomIntBetween(1, 3)]; + final Thread[] readers = new Thread[randomIntBetween(1, 3)]; + final int flushEveryOps = randomIntBetween(5, 100); + final int maxOps = randomIntBetween(200, 1000); + final Object signalReaderSomeDataWasIndexed = new Object(); + final AtomicLong idGenerator = new AtomicLong(); + final CyclicBarrier barrier = new CyclicBarrier(writers.length + readers.length + 1); + + // a map of all written ops and their returned location. + final Map writtenOps = ConcurrentCollections.newConcurrentMap(); + + // a signal for all threads to stop + final AtomicBoolean run = new AtomicBoolean(true); + + final Object flushMutex = new Object(); + final AtomicLong lastCommittedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final LocalCheckpointTracker tracker = LocalCheckpointTrackerTests.createEmptyTracker(); + final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); + // any errors on threads + final List errors = new CopyOnWriteArrayList<>(); + logger.info("using [{}] readers. [{}] writers. flushing every ~[{}] ops.", readers.length, writers.length, flushEveryOps); + for (int i = 0; i < writers.length; i++) { + final String threadName = "writer_" + i; + final int threadId = i; + writers[i] = new Thread(new AbstractRunnable() { + @Override + public void doRun() throws BrokenBarrierException, InterruptedException, IOException { + barrier.await(); + int counter = 0; + while (run.get() && idGenerator.get() < maxOps) { + long id = idGenerator.getAndIncrement(); + final Translog.Operation op; + final Translog.Operation.Type type = Translog.Operation.Type.values()[((int) (id % Translog.Operation.Type + .values().length))]; + switch (type) { + case CREATE: + case INDEX: + op = new Translog.Index("" + id, id, primaryTerm.get(), new byte[] { (byte) id }); + break; + case DELETE: + op = new Translog.Delete(Long.toString(id), id, primaryTerm.get()); + break; + case NO_OP: + op = new Translog.NoOp(id, 1, Long.toString(id)); + break; + default: + throw new AssertionError("unsupported operation type [" + type + "]"); + } + Translog.Location location = translog.add(op); + tracker.markSeqNoAsProcessed(id); + Translog.Location existing = writtenOps.put(op, location); + if (existing != null) { + fail("duplicate op [" + op + "], old entry at " + location); + } + if (id % writers.length == threadId) { + translog.ensureSynced(location); + } + if (id % flushEveryOps == 0) { + synchronized (flushMutex) { + // we need not do this concurrently as we need to make sure that the generation + // we're committing - is still present when we're committing + long localCheckpoint = tracker.getProcessedCheckpoint(); + translog.rollGeneration(); + // expose the new checkpoint (simulating a commit), before we trim the translog + lastCommittedLocalCheckpoint.set(localCheckpoint); + deletionPolicy.setLocalCheckpointOfSafeCommit(localCheckpoint); + translog.trimUnreferencedReaders(); + } + } + if (id % 7 == 0) { + synchronized (signalReaderSomeDataWasIndexed) { + signalReaderSomeDataWasIndexed.notifyAll(); + } + } + counter++; + } + logger.info("--> [{}] done. wrote [{}] ops.", threadName, counter); + } + + @Override + public void onFailure(Exception e) { + logger.error(() -> new ParameterizedMessage("--> writer [{}] had an error", threadName), e); + errors.add(e); + } + }, threadName); + writers[i].start(); + } + + for (int i = 0; i < readers.length; i++) { + final String threadId = "reader_" + i; + readers[i] = new Thread(new AbstractRunnable() { + Closeable retentionLock = null; + long committedLocalCheckpointAtView; + + @Override + public void onFailure(Exception e) { + logger.error(() -> new ParameterizedMessage("--> reader [{}] had an error", threadId), e); + errors.add(e); + try { + closeRetentionLock(); + } catch (IOException inner) { + inner.addSuppressed(e); + logger.error("unexpected error while closing view, after failure", inner); + } + } + + void closeRetentionLock() throws IOException { + if (retentionLock != null) { + retentionLock.close(); + } + } + + void acquireRetentionLock() throws IOException { + closeRetentionLock(); + retentionLock = translog.acquireRetentionLock(); + // captures the last committed checkpoint, while holding the view, simulating + // recovery logic which captures a view and gets a lucene commit + committedLocalCheckpointAtView = lastCommittedLocalCheckpoint.get(); + logger.info("--> [{}] min gen after acquiring lock [{}]", threadId, translog.getMinFileGeneration()); + } + + @Override + protected void doRun() throws Exception { + barrier.await(); + int iter = 0; + while (idGenerator.get() < maxOps) { + if (iter++ % 10 == 0) { + acquireRetentionLock(); + } + + // captures al views that are written since the view was created (with a small caveat see bellow) + // these are what we expect the snapshot to return (and potentially some more). + Set expectedOps = new HashSet<>(writtenOps.keySet()); + expectedOps.removeIf(op -> op.seqNo() <= committedLocalCheckpointAtView); + try (Translog.Snapshot snapshot = translog.newSnapshot(committedLocalCheckpointAtView + 1L, Long.MAX_VALUE)) { + Translog.Operation op; + while ((op = snapshot.next()) != null) { + expectedOps.remove(op); + } + } + if (expectedOps.isEmpty() == false) { + StringBuilder missed = new StringBuilder("missed ").append(expectedOps.size()) + .append(" operations from [") + .append(committedLocalCheckpointAtView + 1L) + .append("]"); + boolean failed = false; + for (Translog.Operation expectedOp : expectedOps) { + final Translog.Location loc = writtenOps.get(expectedOp); + failed = true; + missed.append("\n --> [").append(expectedOp).append("] written at ").append(loc); + } + if (failed) { + fail(missed.toString()); + } + } + // slow down things a bit and spread out testing.. + synchronized (signalReaderSomeDataWasIndexed) { + if (idGenerator.get() < maxOps) { + signalReaderSomeDataWasIndexed.wait(); + } + } + } + closeRetentionLock(); + logger.info("--> [{}] done. tested [{}] snapshots", threadId, iter); + } + }, threadId); + readers[i].start(); + } + + barrier.await(); + logger.debug("--> waiting for threads to stop"); + for (Thread thread : writers) { + thread.join(); + } + logger.debug("--> waiting for readers to stop"); + // force stopping, if all writers crashed + synchronized (signalReaderSomeDataWasIndexed) { + idGenerator.set(Long.MAX_VALUE); + signalReaderSomeDataWasIndexed.notifyAll(); + } + for (Thread thread : readers) { + thread.join(); + } + if (errors.size() > 0) { + Throwable e = errors.get(0); + for (Throwable suppress : errors.subList(1, errors.size())) { + e.addSuppressed(suppress); + } + throw e; + } + logger.info("--> test done. total ops written [{}]", writtenOps.size()); + } + + public void testSyncUpTo() throws IOException { + int translogOperations = randomIntBetween(10, 100); + int count = 0; + for (int op = 0; op < translogOperations; op++) { + int seqNo = ++count; + final Translog.Location location = translog.add( + new Translog.Index("" + op, seqNo, primaryTerm.get(), Integer.toString(seqNo).getBytes(Charset.forName("UTF-8"))) + ); + if (randomBoolean()) { + assertTrue("at least one operation pending", translog.syncNeeded()); + assertTrue("this operation has not been synced", translog.ensureSynced(location)); + // we are the last location so everything should be synced + assertFalse("the last call to ensureSycned synced all previous ops", translog.syncNeeded()); + seqNo = ++count; + translog.add( + new Translog.Index("" + op, seqNo, primaryTerm.get(), Integer.toString(seqNo).getBytes(Charset.forName("UTF-8"))) + ); + assertTrue("one pending operation", translog.syncNeeded()); + assertFalse("this op has been synced before", translog.ensureSynced(location)); // not syncing now + assertTrue("we only synced a previous operation yet", translog.syncNeeded()); + } + if (rarely()) { + translog.rollGeneration(); + assertFalse("location is from a previous translog - already synced", translog.ensureSynced(location)); // not syncing now + assertFalse("no sync needed since no operations in current translog", translog.syncNeeded()); + } + + if (randomBoolean()) { + translog.sync(); + assertFalse("translog has been synced already", translog.ensureSynced(location)); + } + } + } + + public void testSyncUpFailure() throws IOException { + int translogOperations = randomIntBetween(1, 20); + int count = 0; + fail.failAlways(); + ArrayList locations = new ArrayList<>(); + for (int op = 0; op < translogOperations; op++) { + int seqNo = ++count; + final Translog.Location location = translog.add( + new Translog.Index("" + op, seqNo, primaryTerm.get(), Integer.toString(seqNo).getBytes(Charset.forName("UTF-8"))) + ); + if (randomBoolean()) { + fail.failAlways(); + try { + translog.ensureSynced(location); + fail("io exception expected"); + } catch (IOException e) { + assertTrue("at least one operation pending", translog.syncNeeded()); + } + } else { + fail.failNever(); + translog.ensureSynced(location); + assertFalse("no sync needed since no operations in current translog", translog.syncNeeded()); + } + locations.add(location); + + } + // clean up + fail.failNever(); + + // writes should get synced up now + translog.sync(); + assertFalse(translog.syncNeeded()); + for (Translog.Location location : locations) { + assertFalse("all of the locations should be synced: " + location, translog.ensureSynced(location)); + } + + } + + public void testSyncUpToStream() throws IOException { + int iters = randomIntBetween(5, 10); + for (int i = 0; i < iters; i++) { + int translogOperations = randomIntBetween(10, 100); + int count = 0; + ArrayList locations = new ArrayList<>(); + for (int op = 0; op < translogOperations; op++) { + if (rarely()) { + translog.rollGeneration(); + } + final Translog.Location location = translog.add( + new Translog.Index("" + op, op, primaryTerm.get(), Integer.toString(++count).getBytes(Charset.forName("UTF-8"))) + ); + locations.add(location); + } + Collections.shuffle(locations, random()); + if (randomBoolean()) { + assertTrue("at least one operation pending", translog.syncNeeded()); + assertTrue("this operation has not been synced", translog.ensureSynced(locations.stream())); + // we are the last location so everything should be synced + assertFalse("the last call to ensureSycned synced all previous ops", translog.syncNeeded()); + } else if (rarely()) { + translog.rollGeneration(); + // not syncing now + assertFalse("location is from a previous translog - already synced", translog.ensureSynced(locations.stream())); + assertFalse("no sync needed since no operations in current translog", translog.syncNeeded()); + } else { + translog.sync(); + assertFalse("translog has been synced already", translog.ensureSynced(locations.stream())); + } + for (Translog.Location location : locations) { + assertFalse("all of the locations should be synced: " + location, translog.ensureSynced(location)); + } + } + } + + public void testLocationComparison() throws IOException { + List locations = new ArrayList<>(); + int translogOperations = randomIntBetween(10, 100); + int count = 0; + for (int op = 0; op < translogOperations; op++) { + locations.add( + translog.add( + new Translog.Index("" + op, op, primaryTerm.get(), Integer.toString(++count).getBytes(Charset.forName("UTF-8"))) + ) + ); + if (randomBoolean()) { + translog.ensureSynced(locations.get(op)); + } + if (rarely() && translogOperations > op + 1) { + translog.rollGeneration(); + } + } + Collections.shuffle(locations, random()); + Translog.Location max = locations.get(0); + for (Translog.Location location : locations) { + max = max(max, location); + } + + try (Translog.Snapshot snap = new TestTranslog.SortedSnapshot(translog.newSnapshot())) { + Translog.Operation next; + Translog.Operation maxOp = null; + while ((next = snap.next()) != null) { + maxOp = next; + } + assertNotNull(maxOp); + assertEquals(maxOp.getSource().source.utf8ToString(), Integer.toString(count)); + } + } + + public static Translog.Location max(Translog.Location a, Translog.Location b) { + if (a.compareTo(b) > 0) { + return a; + } + return b; + } + + public void testTranslogWriter() throws IOException { + final TranslogWriter writer = translog.createWriter(translog.currentFileGeneration() + 1); + final Set persistedSeqNos = new HashSet<>(); + persistedSeqNoConsumer.set(persistedSeqNos::add); + final int numOps = scaledRandomIntBetween(8, 250000); + final Set seenSeqNos = new HashSet<>(); + boolean opsHaveValidSequenceNumbers = randomBoolean(); + for (int i = 0; i < numOps; i++) { + byte[] bytes = new byte[4]; + DataOutput out = EndiannessReverserUtil.wrapDataOutput(new ByteArrayDataOutput(bytes)); + out.writeInt(i); + long seqNo; + do { + seqNo = opsHaveValidSequenceNumbers ? randomNonNegativeLong() : SequenceNumbers.UNASSIGNED_SEQ_NO; + opsHaveValidSequenceNumbers = opsHaveValidSequenceNumbers || !rarely(); + } while (seenSeqNos.contains(seqNo)); + if (seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { + seenSeqNos.add(seqNo); + } + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), seqNo); + } + assertThat(persistedSeqNos, empty()); + writer.sync(); + persistedSeqNos.remove(SequenceNumbers.UNASSIGNED_SEQ_NO); + assertEquals(seenSeqNos, persistedSeqNos); + + final BaseTranslogReader reader = randomBoolean() + ? writer + : translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME))); + for (int i = 0; i < numOps; i++) { + ByteBuffer buffer = ByteBuffer.allocate(4); + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * i); + buffer.flip(); + final int value = buffer.getInt(); + assertEquals(i, value); + } + final long minSeqNo = seenSeqNos.stream().min(Long::compareTo).orElse(SequenceNumbers.NO_OPS_PERFORMED); + final long maxSeqNo = seenSeqNos.stream().max(Long::compareTo).orElse(SequenceNumbers.NO_OPS_PERFORMED); + assertThat(reader.getCheckpoint().minSeqNo, equalTo(minSeqNo)); + assertThat(reader.getCheckpoint().maxSeqNo, equalTo(maxSeqNo)); + + byte[] bytes = new byte[4]; + DataOutput out = EndiannessReverserUtil.wrapDataOutput(new ByteArrayDataOutput(bytes)); + out.writeInt(2048); + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), randomNonNegativeLong()); + + if (reader instanceof TranslogReader) { + ByteBuffer buffer = ByteBuffer.allocate(4); + try { + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * numOps); + fail("read past EOF?"); + } catch (EOFException ex) { + // expected + } + ((TranslogReader) reader).close(); + } else { + // live reader! + ByteBuffer buffer = ByteBuffer.allocate(4); + final long pos = reader.getFirstOperationOffset() + 4 * numOps; + reader.readBytes(buffer, pos); + buffer.flip(); + final int value = buffer.getInt(); + assertEquals(2048, value); + } + IOUtils.close(writer); + } + + public void testTranslogWriterCanFlushInAddOrReadCall() throws IOException { + Path tempDir = createTempDir(); + final TranslogConfig temp = getTranslogConfig(tempDir); + final TranslogConfig config = new TranslogConfig( + temp.getShardId(), + temp.getTranslogPath(), + temp.getIndexSettings(), + temp.getBigArrays(), + new ByteSizeValue(1, ByteSizeUnit.KB) + ); + + final Set persistedSeqNos = new HashSet<>(); + final AtomicInteger writeCalls = new AtomicInteger(); + + final ChannelFactory channelFactory = (file, openOption) -> { + FileChannel delegate = FileChannel.open(file, openOption); + boolean success = false; + try { + // don't do partial writes for checkpoints we rely on the fact that the bytes are written as an atomic operation + final boolean isCkpFile = file.getFileName().toString().endsWith(".ckp"); + + final FileChannel channel; + if (isCkpFile) { + channel = delegate; + } else { + channel = new FilterFileChannel(delegate) { + + @Override + public int write(ByteBuffer src) throws IOException { + writeCalls.incrementAndGet(); + return super.write(src); + } + }; + } + success = true; + return channel; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(delegate); + } + } + }; + + String translogUUID = Translog.createEmptyTranslog( + config.getTranslogPath(), + SequenceNumbers.NO_OPS_PERFORMED, + shardId, + channelFactory, + primaryTerm.get() + ); + + try ( + Translog translog = new RemoteFsTranslog( + config, + translogUUID, + new DefaultTranslogDeletionPolicy(-1, -1, 0), + () -> SequenceNumbers.NO_OPS_PERFORMED, + primaryTerm::get, + persistedSeqNos::add, + repository, + threadPool.executor(ThreadPool.Names.TRANSLOG_TRANSFER) + ) { + @Override + ChannelFactory getChannelFactory() { + return channelFactory; + } + } + ) { + TranslogWriter writer = translog.getCurrent(); + int initialWriteCalls = writeCalls.get(); + byte[] bytes = new byte[256]; + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 1); + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 2); + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 3); + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 4); + assertThat(persistedSeqNos, empty()); + assertEquals(initialWriteCalls, writeCalls.get()); + + if (randomBoolean()) { + // Since the buffer is full, this will flush before performing the add. + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 5); + assertThat(persistedSeqNos, empty()); + assertThat(writeCalls.get(), greaterThan(initialWriteCalls)); + } else { + // Will flush on read + writer.readBytes(ByteBuffer.allocate(256), 0); + assertThat(persistedSeqNos, empty()); + assertThat(writeCalls.get(), greaterThan(initialWriteCalls)); + + // Add after we the read flushed the buffer + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), 5); + } + + writer.sync(); + + // Sequence numbers are marked as persisted after sync + assertThat(persistedSeqNos, contains(1L, 2L, 3L, 4L, 5L)); + } + } + + public void testCloseIntoReader() throws IOException { + try (TranslogWriter writer = translog.createWriter(translog.currentFileGeneration() + 1)) { + final int numOps = randomIntBetween(8, 128); + for (int i = 0; i < numOps; i++) { + final byte[] bytes = new byte[4]; + final DataOutput out = EndiannessReverserUtil.wrapDataOutput(new ByteArrayDataOutput(bytes)); + out.writeInt(i); + writer.add(ReleasableBytesReference.wrap(new BytesArray(bytes)), randomNonNegativeLong()); + } + writer.sync(); + final Checkpoint writerCheckpoint = writer.getCheckpoint(); + TranslogReader reader = writer.closeIntoReader(); + try { + if (randomBoolean()) { + reader.close(); + reader = translog.openReader(reader.path(), writerCheckpoint); + } + for (int i = 0; i < numOps; i++) { + final ByteBuffer buffer = ByteBuffer.allocate(4); + reader.readBytes(buffer, reader.getFirstOperationOffset() + 4 * i); + buffer.flip(); + final int value = buffer.getInt(); + assertEquals(i, value); + } + final Checkpoint readerCheckpoint = reader.getCheckpoint(); + assertThat(readerCheckpoint, equalTo(writerCheckpoint)); + } finally { + IOUtils.close(reader); + } + } + } + + public class ThrowingBlobRepository extends FsRepository { + private final Environment environment; + + private TestTranslog.FailSwitch fail; + + public ThrowingBlobRepository( + RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + RecoverySettings recoverySettings, + TestTranslog.FailSwitch fail + ) { + super(metadata, environment, namedXContentRegistry, clusterService, recoverySettings); + this.environment = environment; + this.fail = fail; + } + + protected BlobStore createBlobStore() throws Exception { + final String location = REPOSITORIES_LOCATION_SETTING.get(getMetadata().settings()); + final Path locationFile = environment.resolveRepoFile(location); + return new ThrowingBlobStore(bufferSize, locationFile, isReadOnly(), fail); + } + } + + private class ThrowingBlobStore extends FsBlobStore { + + private TestTranslog.FailSwitch fail; + + public ThrowingBlobStore(int bufferSizeInBytes, Path path, boolean readonly, TestTranslog.FailSwitch fail) throws IOException { + super(bufferSizeInBytes, path, readonly); + this.fail = fail; + } + + @Override + public BlobContainer blobContainer(BlobPath path) { + try { + return new ThrowingBlobContainer(this, path, buildAndCreate(path), fail); + } catch (IOException ex) { + throw new OpenSearchException("failed to create blob container", ex); + } + } + } + + private class ThrowingBlobContainer extends FsBlobContainer { + + private TestTranslog.FailSwitch fail; + + public ThrowingBlobContainer(FsBlobStore blobStore, BlobPath blobPath, Path path, TestTranslog.FailSwitch fail) { + super(blobStore, blobPath, path); + this.fail = fail; + } + + public void writeBlobAtomic(final String blobName, final InputStream inputStream, final long blobSize, boolean failIfAlreadyExists) + throws IOException { + if (fail.fail()) { + throw new IOException("blob container throwing error"); + } + super.writeBlobAtomic(blobName, inputStream, blobSize, failIfAlreadyExists); + } + } + + class TranslogThread extends Thread { + private final CountDownLatch downLatch; + private final int opsPerThread; + private final int threadId; + private final Collection writtenOperations; + private final Exception[] threadExceptions; + private final Translog translog; + private final AtomicLong seqNoGenerator; + + TranslogThread( + Translog translog, + CountDownLatch downLatch, + int opsPerThread, + int threadId, + Collection writtenOperations, + AtomicLong seqNoGenerator, + Exception[] threadExceptions + ) { + this.translog = translog; + this.downLatch = downLatch; + this.opsPerThread = opsPerThread; + this.threadId = threadId; + this.writtenOperations = writtenOperations; + this.seqNoGenerator = seqNoGenerator; + this.threadExceptions = threadExceptions; + } + + @Override + public void run() { + try { + downLatch.await(); + for (int opCount = 0; opCount < opsPerThread; opCount++) { + Translog.Operation op; + final Translog.Operation.Type type = randomFrom(Translog.Operation.Type.values()); + switch (type) { + case CREATE: + case INDEX: + op = new Translog.Index( + threadId + "_" + opCount, + seqNoGenerator.getAndIncrement(), + primaryTerm.get(), + randomUnicodeOfLengthBetween(1, 20 * 1024).getBytes("UTF-8") + ); + break; + case DELETE: + op = new Translog.Delete( + threadId + "_" + opCount, + seqNoGenerator.getAndIncrement(), + primaryTerm.get(), + 1 + randomInt(100000) + ); + break; + case NO_OP: + op = new Translog.NoOp(seqNoGenerator.getAndIncrement(), primaryTerm.get(), randomAlphaOfLength(16)); + break; + default: + throw new AssertionError("unsupported operation type [" + type + "]"); + } + + Translog.Location loc = add(op); + writtenOperations.add(new TestTranslog.LocationOperation(op, loc)); + if (rarely()) { // lets verify we can concurrently read this + assertEquals(op, translog.readOperation(loc)); + } + afterAdd(); + } + } catch (Exception t) { + threadExceptions[threadId] = t; + } + } + + protected Translog.Location add(Translog.Operation op) throws IOException { + Translog.Location location = translog.add(op); + if (randomBoolean()) { + translog.ensureSynced(location); + } + return location; + } + + protected void afterAdd() {} + } + +} diff --git a/server/src/test/java/org/opensearch/index/translog/TestTranslog.java b/server/src/test/java/org/opensearch/index/translog/TestTranslog.java index 4e9d56aeb9573..3545f2dbf5ffc 100644 --- a/server/src/test/java/org/opensearch/index/translog/TestTranslog.java +++ b/server/src/test/java/org/opensearch/index/translog/TestTranslog.java @@ -38,6 +38,7 @@ import org.apache.lucene.tests.util.LuceneTestCase; import org.opensearch.common.io.stream.InputStreamStreamInput; import org.opensearch.core.internal.io.IOUtils; +import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; import java.nio.ByteBuffer; @@ -283,4 +284,88 @@ static boolean isTranslogHeaderVersionFlipped(Path corruptedFile, FileChannel ch return false; } } + + static class LocationOperation implements Comparable { + final Translog.Operation operation; + final Translog.Location location; + + LocationOperation(Translog.Operation operation, Translog.Location location) { + this.operation = operation; + this.location = location; + } + + @Override + public int compareTo(LocationOperation o) { + return location.compareTo(o.location); + } + } + + static class FailSwitch { + private volatile int failRate; + private volatile boolean onceFailedFailAlways = false; + + public boolean fail() { + final int rnd = OpenSearchTestCase.randomIntBetween(1, 100); + boolean fail = rnd <= failRate; + if (fail && onceFailedFailAlways) { + failAlways(); + } + return fail; + } + + public void failNever() { + failRate = 0; + } + + public void failAlways() { + failRate = 100; + } + + public void failRandomly() { + failRate = OpenSearchTestCase.randomIntBetween(1, 100); + } + + public void failRate(int rate) { + failRate = rate; + } + + public void onceFailedFailAlways() { + onceFailedFailAlways = true; + } + } + + static class SortedSnapshot implements Translog.Snapshot { + private final Translog.Snapshot snapshot; + private List operations = null; + + SortedSnapshot(Translog.Snapshot snapshot) { + this.snapshot = snapshot; + } + + @Override + public int totalOperations() { + return snapshot.totalOperations(); + } + + @Override + public Translog.Operation next() throws IOException { + if (operations == null) { + operations = new ArrayList<>(); + Translog.Operation op; + while ((op = snapshot.next()) != null) { + operations.add(op); + } + operations.sort(Comparator.comparing(Translog.Operation::seqNo)); + } + if (operations.isEmpty()) { + return null; + } + return operations.remove(0); + } + + @Override + public void close() throws IOException { + snapshot.close(); + } + } } diff --git a/server/src/test/java/org/opensearch/index/translog/TranslogManagerTestCase.java b/server/src/test/java/org/opensearch/index/translog/TranslogManagerTestCase.java index 25867cdb666ad..4c3948cbd7b5b 100644 --- a/server/src/test/java/org/opensearch/index/translog/TranslogManagerTestCase.java +++ b/server/src/test/java/org/opensearch/index/translog/TranslogManagerTestCase.java @@ -81,7 +81,7 @@ protected Translog createTranslog(Path translogPath, LongSupplier primaryTermSup shardId, primaryTermSupplier.getAsLong() ); - return new Translog( + return new LocalTranslog( translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), 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 adca47bf64c64..5dc5ac92070ea 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 @@ -54,6 +54,16 @@ public void testUploadBlob() throws IOException { transferService.uploadBlob(transferFileSnapshot, repository.basePath()); } + public void testUploadBlobFromByteArray() throws IOException { + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + "dummy_name", + randomByteArrayOfLength(128), + 1 + ); + TransferService transferService = new BlobStoreTransferService(repository.blobStore(), executorService); + transferService.uploadBlob(transferFileSnapshot, repository.basePath()); + } + public void testUploadBlobAsync() throws IOException, InterruptedException { Path testFile = createTempFile(); Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); 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 new file mode 100644 index 0000000000000..c6b4579f5ddd1 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/transfer/FileTransferTrackerTests.java @@ -0,0 +1,77 @@ +/* + * 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.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; + +public class FileTransferTrackerTests extends OpenSearchTestCase { + + protected final ShardId shardId = new ShardId("index", "_na_", 1); + FileTransferTracker fileTransferTracker; + + @Override + public void setUp() throws Exception { + super.setUp(); + } + + public void testOnSuccess() throws IOException { + fileTransferTracker = new FileTransferTracker(shardId); + Path testFile = createTempFile(); + Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + try ( + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong() + ) + ) { + fileTransferTracker.onSuccess(transferFileSnapshot); + // idempotent + fileTransferTracker.onSuccess(transferFileSnapshot); + assertEquals(fileTransferTracker.allUploaded().size(), 1); + try { + fileTransferTracker.onFailure(transferFileSnapshot, new IOException("random exception")); + fail("failure after succcess invalid"); + } catch (IllegalStateException ex) { + // all good + } + } + } + + public void testOnFailure() throws IOException { + fileTransferTracker = new FileTransferTracker(shardId); + Path testFile = createTempFile(); + Path testFile2 = createTempFile(); + Files.write(testFile, randomByteArrayOfLength(128), StandardOpenOption.APPEND); + try ( + FileSnapshot.TransferFileSnapshot transferFileSnapshot = new FileSnapshot.TransferFileSnapshot( + testFile, + randomNonNegativeLong() + ); + FileSnapshot.TransferFileSnapshot transferFileSnapshot2 = new FileSnapshot.TransferFileSnapshot( + testFile2, + randomNonNegativeLong() + ) + ) { + + fileTransferTracker.onFailure(transferFileSnapshot, new IOException("random exception")); + fileTransferTracker.onSuccess(transferFileSnapshot2); + assertEquals(fileTransferTracker.allUploaded().size(), 1); + + fileTransferTracker.onSuccess(transferFileSnapshot); + assertEquals(fileTransferTracker.allUploaded().size(), 2); + } + } + +} 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 60b7029f18fa6..1a8827ac797a8 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 @@ -12,22 +12,30 @@ import org.mockito.Mockito; import org.opensearch.action.ActionListener; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.util.set.Sets; +import org.opensearch.index.shard.ShardId; import org.opensearch.index.translog.Translog; -import org.opensearch.index.translog.transfer.listener.FileTransferListener; import org.opensearch.index.translog.transfer.listener.TranslogTransferListener; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.index.translog.transfer.FileSnapshot.CheckpointFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TranslogFileSnapshot; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; +import java.io.ByteArrayInputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import static org.mockito.Mockito.any; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; @LuceneTestCase.SuppressFileSystems("*") public class TranslogTransferManagerTests extends OpenSearchTestCase { @@ -66,7 +74,7 @@ public void testTransferSnapshot() throws IOException { TranslogTransferManager translogTransferManager = new TranslogTransferManager( transferService, remoteBaseTransferPath, - new FileTransferListener() { + new FileTransferTracker(new ShardId("index", "indexUUid", 0)) { @Override public void onSuccess(TransferFileSnapshot fileSnapshot) { fileTransferSucceeded.incrementAndGet(); @@ -76,8 +84,7 @@ public void onSuccess(TransferFileSnapshot fileSnapshot) { public void onFailure(TransferFileSnapshot fileSnapshot, Exception e) { fileTransferFailed.incrementAndGet(); } - }, - r -> r + } ); assertTrue(translogTransferManager.transferSnapshot(createTransferSnapshot(), new TranslogTransferListener() { @@ -147,4 +154,141 @@ public TranslogTransferMetadata getTranslogTransferMetadata() { } }; } + + public void testReadMetadataNoFile() throws IOException { + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, null); + + when(transferService.listAll(remoteBaseTransferPath)).thenReturn(Sets.newHashSet()); + assertNull(translogTransferManager.readMetadata()); + } + + public void testReadMetadataSingleFile() throws IOException { + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, null); + + // BlobPath does not have equals method, so we can't use the instance directly in when + when(transferService.listAll(any(BlobPath.class))).thenReturn(Sets.newHashSet("12__234__123456789")); + + TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); + when(transferService.downloadBlob(any(BlobPath.class), eq("12__234__123456789"))).thenReturn( + new ByteArrayInputStream(metadata.createMetadataBytes()) + ); + + assertEquals(metadata, translogTransferManager.readMetadata()); + } + + public void testReadMetadataMultipleFiles() throws IOException { + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, null); + + when(transferService.listAll(any(BlobPath.class))).thenReturn( + Sets.newHashSet("12__234__56789", "12__235__56823", "12__233__56700") + ); + + TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); + when(transferService.downloadBlob(any(BlobPath.class), eq("12__235__56823"))).thenReturn( + new ByteArrayInputStream(metadata.createMetadataBytes()) + ); + + assertEquals(metadata, translogTransferManager.readMetadata()); + } + + public void testReadMetadataException() throws IOException { + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, null); + + when(transferService.listAll(any(BlobPath.class))).thenReturn( + Sets.newHashSet("12__234__56789", "12__235__56823", "12__233__56700") + ); + + when(transferService.downloadBlob(any(BlobPath.class), eq("12__235__56823"))).thenThrow(new IOException("Something went wrong")); + + assertNull(translogTransferManager.readMetadata()); + } + + public void testReadMetadataSamePrimaryTermGeneration() throws IOException { + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, null); + + when(transferService.listAll(any(BlobPath.class))).thenReturn( + Sets.newHashSet("12__234__56789", "12__235__56823", "12__234__56700") + ); + + assertThrows(IllegalArgumentException.class, translogTransferManager::readMetadata); + } + + public void testDownloadTranslog() throws IOException { + Path location = createTempDir(); + TranslogTransferManager translogTransferManager = new TranslogTransferManager( + transferService, + remoteBaseTransferPath, + new FileTransferTracker(new ShardId("index", "indexUuid", 0)) + ); + + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.tlog"))).thenReturn( + new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) + ); + + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.ckp"))).thenReturn( + new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) + ); + + assertFalse(Files.exists(location.resolve("translog-23.tlog"))); + assertFalse(Files.exists(location.resolve("translog-23.ckp"))); + translogTransferManager.downloadTranslog("12", "23", location); + assertTrue(Files.exists(location.resolve("translog-23.tlog"))); + assertTrue(Files.exists(location.resolve("translog-23.ckp"))); + } + + public void testDownloadTranslogAlreadyExists() throws IOException { + FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); + Path location = createTempDir(); + Files.createFile(location.resolve("translog-23.tlog")); + Files.createFile(location.resolve("translog-23.ckp")); + + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, tracker); + + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.tlog"))).thenReturn( + new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) + ); + when(transferService.downloadBlob(any(BlobPath.class), eq("translog-23.ckp"))).thenReturn( + new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) + ); + + translogTransferManager.downloadTranslog("12", "23", location); + + verify(transferService).downloadBlob(any(BlobPath.class), eq("translog-23.tlog")); + verify(transferService).downloadBlob(any(BlobPath.class), eq("translog-23.ckp")); + assertTrue(Files.exists(location.resolve("translog-23.tlog"))); + assertTrue(Files.exists(location.resolve("translog-23.ckp"))); + } + + public void testDownloadTranslogWithTrackerUpdated() throws IOException { + FileTransferTracker tracker = new FileTransferTracker(new ShardId("index", "indexUuid", 0)); + Path location = createTempDir(); + String translogFile = "translog-23.tlog", checkpointFile = "translog-23.ckp"; + Files.createFile(location.resolve(translogFile)); + Files.createFile(location.resolve(checkpointFile)); + + TranslogTransferManager translogTransferManager = new TranslogTransferManager(transferService, remoteBaseTransferPath, tracker); + + when(transferService.downloadBlob(any(BlobPath.class), eq(translogFile))).thenReturn( + new ByteArrayInputStream("Hello Translog".getBytes(StandardCharsets.UTF_8)) + ); + when(transferService.downloadBlob(any(BlobPath.class), eq(checkpointFile))).thenReturn( + new ByteArrayInputStream("Hello Checkpoint".getBytes(StandardCharsets.UTF_8)) + ); + + translogTransferManager.downloadTranslog("12", "23", location); + + verify(transferService).downloadBlob(any(BlobPath.class), eq(translogFile)); + verify(transferService).downloadBlob(any(BlobPath.class), eq(checkpointFile)); + assertTrue(Files.exists(location.resolve(translogFile))); + assertTrue(Files.exists(location.resolve(checkpointFile))); + + // Since the tracker already holds the files with success state, adding them with failed state would throw exception + assertThrows(IllegalStateException.class, () -> tracker.add(translogFile, false)); + assertThrows(IllegalStateException.class, () -> tracker.add(checkpointFile, false)); + + // Since the tracker already holds the files with success state, adding them with success state is allowed + tracker.add(translogFile, true); + tracker.add(checkpointFile, true); + + } } diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index 91953d4db3495..465629406b54b 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -22,6 +22,7 @@ public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLe .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "translog-repo") .build(); public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 663c325db12c2..5732fc5bfa270 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -1797,6 +1797,8 @@ public void onFailure(final Exception e) { ); final BigArrays bigArrays = new BigArrays(new PageCacheRecycler(settings), null, "test"); final MapperRegistry mapperRegistry = new IndicesModule(Collections.emptyList()).getMapperRegistry(); + final SetOnce repositoriesServiceReference = new SetOnce<>(); + repositoriesServiceReference.set(repositoriesService); if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { indicesService = new IndicesService( settings, @@ -1831,7 +1833,8 @@ public void onFailure(final Exception e) { emptyMap(), null, emptyMap(), - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService) + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService), + repositoriesServiceReference::get ); } else { indicesService = new IndicesService( @@ -1866,10 +1869,10 @@ public void onFailure(final Exception e) { emptyMap(), null, emptyMap(), - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService) + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService), + repositoriesServiceReference::get ); } - final RecoverySettings recoverySettings = new RecoverySettings(settings, clusterSettings); snapshotShardsService = new SnapshotShardsService( settings, diff --git a/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java b/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java index a3a7cbecc6958..cd941feb37002 100644 --- a/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java +++ b/server/src/test/java/org/opensearch/threadpool/ScalingThreadPoolTests.java @@ -132,6 +132,7 @@ private int expectedSize(final String threadPoolName, final int numberOfProcesso sizes.put(ThreadPool.Names.SNAPSHOT, ThreadPool::halfAllocatedProcessorsMaxFive); sizes.put(ThreadPool.Names.FETCH_SHARD_STARTED, ThreadPool::twiceAllocatedProcessors); sizes.put(ThreadPool.Names.FETCH_SHARD_STORE, ThreadPool::twiceAllocatedProcessors); + sizes.put(ThreadPool.Names.TRANSLOG_TRANSFER, ThreadPool::halfAllocatedProcessorsMaxTen); return sizes.get(threadPoolName).apply(numberOfProcessors); } diff --git a/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java index a3dfe37bde826..3dff21de33ece 100644 --- a/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java @@ -112,6 +112,7 @@ import org.opensearch.index.shard.ShardId; import org.opensearch.index.store.Store; import org.opensearch.index.translog.InternalTranslogManager; +import org.opensearch.index.translog.LocalTranslog; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogConfig; import org.opensearch.index.translog.TranslogDeletionPolicy; @@ -150,9 +151,9 @@ import static java.util.Collections.shuffle; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; -import static org.hamcrest.Matchers.instanceOf; import static org.opensearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.opensearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.opensearch.index.engine.Engine.Operation.Origin.REPLICA; @@ -528,7 +529,7 @@ protected Translog createTranslog(Path translogPath, LongSupplier primaryTermSup shardId, primaryTermSupplier.getAsLong() ); - return new Translog( + return new LocalTranslog( translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index bc7780ffed131..f01a189e974bf 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -38,6 +38,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.junit.Assert; +import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.Version; @@ -61,6 +62,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.fs.FsBlobContainer; import org.opensearch.common.blobstore.fs.FsBlobStore; import org.opensearch.common.bytes.BytesArray; @@ -96,7 +98,10 @@ import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.index.translog.InternalTranslogFactory; +import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.indices.breaker.HierarchyCircuitBreakerService; import org.opensearch.indices.recovery.AsyncRecoveryTarget; @@ -123,7 +128,9 @@ import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.repositories.IndexId; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.blobstore.OpenSearchBlobStoreRepositoryIntegTestCase; import org.opensearch.snapshots.Snapshot; import org.opensearch.test.DummyShardLock; @@ -546,6 +553,17 @@ protected IndexShard newShard( if (remoteStore == null && indexSettings.isRemoteStoreEnabled()) { remoteStore = createRemoteStore(createTempDir(), routing, indexMetadata); } + + final BiFunction translogFactorySupplier = (settings, shardRouting) -> { + if (settings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { + return new RemoteBlobStoreInternalTranslogFactory( + this::createRepositoriesService, + threadPool, + settings.getRemoteStoreTranslogRepository() + ); + } + return new InternalTranslogFactory(); + }; indexShard = new IndexShard( routing, indexSettings, @@ -567,6 +585,7 @@ protected IndexShard newShard( globalCheckpointSyncer, retentionLeaseSyncer, breakerService, + translogFactorySupplier, checkpointPublisher, remoteStore ); @@ -580,6 +599,18 @@ protected IndexShard newShard( return indexShard; } + protected RepositoriesService createRepositoriesService() { + RepositoriesService repositoriesService = Mockito.mock(RepositoriesService.class); + BlobStoreRepository repository = Mockito.mock(BlobStoreRepository.class); + when(repository.basePath()).thenReturn(new BlobPath()); + BlobStore blobStore = Mockito.mock(BlobStore.class); + BlobContainer blobContainer = Mockito.mock(BlobContainer.class); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + when(repository.blobStore()).thenReturn(blobStore); + when(repositoriesService.repository(any(String.class))).thenReturn(repository); + return repositoriesService; + } + protected Store createRemoteStore(Path path, ShardRouting shardRouting, IndexMetadata metadata) throws IOException { Settings nodeSettings = Settings.builder().put("node.name", shardRouting.currentNodeId()).build();