From 507bdab756f9271d8f9a6f4b9a763daedff5baca Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Tue, 8 Feb 2022 19:23:18 -0800 Subject: [PATCH] Initial POC for segment replication. In this POC replicas are configured as read only by not creating an indexwriter. After primary shards refresh, a checkpoint is sent over the transport layer to replicas. Once received, replicas fetch files in the checkpoint from the primary shard. This initial commit ignores failover, retention leases, and shard allocation. Signed-off-by: Marc Handalian --- .../org/opensearch/action/ActionModule.java | 3 + .../flush/TransportShardFlushAction.java | 2 +- .../indices/refresh/RefreshResponse.java | 4 +- .../opensearch/client/IndicesAdminClient.java | 8 + .../client/support/AbstractClient.java | 7 + .../org/opensearch/index/IndexModule.java | 8 +- .../org/opensearch/index/IndexService.java | 15 +- .../org/opensearch/index/IndexSettings.java | 2 +- .../org/opensearch/index/engine/Engine.java | 29 ++ .../opensearch/index/engine/EngineConfig.java | 14 +- .../index/engine/EngineConfigFactory.java | 4 +- .../index/engine/InternalEngine.java | 119 +++++- .../index/engine/OpenSearchReaderManager.java | 43 +- .../index/seqno/ReplicationTracker.java | 31 +- .../shard/CheckpointRefreshListener.java | 44 ++ .../opensearch/index/shard/IndexShard.java | 120 +++++- .../org/opensearch/index/store/Store.java | 52 ++- .../org/opensearch/indices/IndicesModule.java | 2 + .../opensearch/indices/IndicesService.java | 12 +- .../cluster/IndicesClusterStateService.java | 40 +- .../indices/flush/SyncedFlushService.java | 3 + .../indices/recovery/MultiFileWriter.java | 1 + .../recovery/PeerRecoveryTargetService.java | 6 +- .../indices/recovery/RecoveryState.java | 71 ---- .../opensearch/indices/recovery/Timer.java | 94 ++++ .../SegmentReplicationService.java | 182 ++++++++ .../checkpoint/PublishCheckpointAction.java | 19 + .../checkpoint/PublishCheckpointRequest.java | 48 +++ .../ShardPublishCheckpointRequest.java | 46 ++ .../TransportCheckpointInfoResponse.java | 58 +++ .../TransportCheckpointPublisher.java | 46 ++ .../TransportPublishCheckpointAction.java | 63 +++ ...TransportPublishShardCheckpointAction.java | 90 ++++ .../indices/replication/copy/CopyState.java | 73 ++++ .../replication/copy/GetFilesRequest.java | 56 +++ .../replication/copy/GetFilesResponse.java | 30 ++ .../copy/PrimaryShardReplicationHandler.java | 400 ++++++++++++++++++ .../copy/PrimaryShardReplicationSource.java | 272 ++++++++++++ .../replication/copy/ReplicaClient.java | 212 ++++++++++ .../copy/ReplicationCheckpoint.java | 87 ++++ .../copy/ReplicationCollection.java | 291 +++++++++++++ .../copy/ReplicationFailedException.java | 70 +++ .../copy/ReplicationFileChunkRequest.java | 156 +++++++ .../replication/copy/ReplicationState.java | 79 ++++ .../replication/copy/ReplicationTarget.java | 354 ++++++++++++++++ .../SegmentReplicationPrimaryService.java | 184 ++++++++ .../SegmentReplicationTransportRequest.java | 82 ++++ .../copy/StartReplicationRequest.java | 44 ++ .../main/java/org/opensearch/node/Node.java | 8 + 49 files changed, 3525 insertions(+), 159 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java create mode 100644 server/src/main/java/org/opensearch/indices/recovery/Timer.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/SegmentReplicationService.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/ShardPublishCheckpointRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointInfoResponse.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointPublisher.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishCheckpointAction.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishShardCheckpointAction.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/CopyState.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/GetFilesRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/GetFilesResponse.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationHandler.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicaClient.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCheckpoint.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFailedException.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFileChunkRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationPrimaryService.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTransportRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/copy/StartReplicationRequest.java diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index c9b3360e92e87..7905abe4b5533 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -270,6 +270,8 @@ import org.opensearch.index.seqno.RetentionLeaseActions; import org.opensearch.indices.SystemIndices; import org.opensearch.indices.breaker.CircuitBreakerService; +import org.opensearch.indices.replication.checkpoint.PublishCheckpointAction; +import org.opensearch.indices.replication.checkpoint.TransportPublishCheckpointAction; import org.opensearch.persistent.CompletionPersistentTaskAction; import org.opensearch.persistent.RemovePersistentTaskAction; import org.opensearch.persistent.StartPersistentTaskAction; @@ -588,6 +590,7 @@ public void reg actions.register(SimulateTemplateAction.INSTANCE, TransportSimulateTemplateAction.class); actions.register(ValidateQueryAction.INSTANCE, TransportValidateQueryAction.class); actions.register(RefreshAction.INSTANCE, TransportRefreshAction.class); + actions.register(PublishCheckpointAction.INSTANCE, TransportPublishCheckpointAction.class); actions.register(FlushAction.INSTANCE, TransportFlushAction.class); actions.register(SyncedFlushAction.INSTANCE, TransportSyncedFlushAction.class); actions.register(ForceMergeAction.INSTANCE, TransportForceMergeAction.class); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/flush/TransportShardFlushAction.java b/server/src/main/java/org/opensearch/action/admin/indices/flush/TransportShardFlushAction.java index e267c5e224581..8c5604dfac220 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/flush/TransportShardFlushAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/flush/TransportShardFlushAction.java @@ -98,7 +98,7 @@ protected void shardOperationOnPrimary( @Override protected void shardOperationOnReplica(ShardFlushRequest request, IndexShard replica, ActionListener listener) { ActionListener.completeWith(listener, () -> { - replica.flush(request.getRequest()); +// replica.flush(request.getRequest()); logger.trace("{} flush request executed on replica", replica.shardId()); return new ReplicaResult(); }); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/refresh/RefreshResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/refresh/RefreshResponse.java index deae44b1c4676..c6c7b7c59bfd7 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/refresh/RefreshResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/refresh/RefreshResponse.java @@ -61,11 +61,11 @@ public class RefreshResponse extends BroadcastResponse { declareBroadcastFields(PARSER); } - RefreshResponse(StreamInput in) throws IOException { + public RefreshResponse(StreamInput in) throws IOException { super(in); } - RefreshResponse(int totalShards, int successfulShards, int failedShards, List shardFailures) { + public RefreshResponse(int totalShards, int successfulShards, int failedShards, List shardFailures) { super(totalShards, successfulShards, failedShards, shardFailures); } diff --git a/server/src/main/java/org/opensearch/client/IndicesAdminClient.java b/server/src/main/java/org/opensearch/client/IndicesAdminClient.java index 886ce5776eb20..7cf81967d2755 100644 --- a/server/src/main/java/org/opensearch/client/IndicesAdminClient.java +++ b/server/src/main/java/org/opensearch/client/IndicesAdminClient.java @@ -135,6 +135,7 @@ import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.cluster.metadata.IndexMetadata.APIBlock; import org.opensearch.common.Nullable; +import org.opensearch.indices.replication.checkpoint.PublishCheckpointRequest; /** * Administrative actions/operations against indices. @@ -405,6 +406,13 @@ public interface IndicesAdminClient extends OpenSearchClient { */ void refresh(RefreshRequest request, ActionListener listener); + /** + * Publish the latest primary checkpoint to replica shards. + * @param request {@link PublishCheckpointRequest} The PublishCheckpointRequest + * @param listener A listener to be notified with a result + */ + void publishCheckpoint(PublishCheckpointRequest request, ActionListener listener); + /** * Explicitly refresh one or more indices (making the content indexed since the last refresh searchable). */ diff --git a/server/src/main/java/org/opensearch/client/support/AbstractClient.java b/server/src/main/java/org/opensearch/client/support/AbstractClient.java index a796185e2145c..031e4507e08cd 100644 --- a/server/src/main/java/org/opensearch/client/support/AbstractClient.java +++ b/server/src/main/java/org/opensearch/client/support/AbstractClient.java @@ -372,6 +372,8 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.indices.replication.checkpoint.PublishCheckpointAction; +import org.opensearch.indices.replication.checkpoint.PublishCheckpointRequest; import org.opensearch.tasks.TaskId; import org.opensearch.threadpool.ThreadPool; @@ -1632,6 +1634,11 @@ public void refresh(final RefreshRequest request, final ActionListener listener) { + execute(PublishCheckpointAction.INSTANCE, request, listener); + } + @Override public RefreshRequestBuilder prepareRefresh(String... indices) { return new RefreshRequestBuilder(this, RefreshAction.INSTANCE).setIndices(indices); diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 61f6c9689308f..89b0587906ac8 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -75,6 +75,7 @@ import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointPublisher; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.script.ScriptService; import org.opensearch.search.aggregations.support.ValuesSourceRegistry; @@ -466,7 +467,8 @@ public IndexService newIndexService( IndicesFieldDataCache indicesFieldDataCache, NamedWriteableRegistry namedWriteableRegistry, BooleanSupplier idFieldDataEnabled, - ValuesSourceRegistry valuesSourceRegistry + ValuesSourceRegistry valuesSourceRegistry, + TransportCheckpointPublisher checkpointPublisher ) throws IOException { final IndexEventListener eventListener = freeze(); Function> readerWrapperFactory = indexReaderWrapper @@ -520,8 +522,8 @@ public IndexService newIndexService( allowExpensiveQueries, expressionResolver, valuesSourceRegistry, - recoveryStateFactory - ); + recoveryStateFactory, + checkpointPublisher); success = true; return indexService; } finally { diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index a14afde7ebafb..ec75a1fc6b69a 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -94,6 +94,7 @@ import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointPublisher; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.script.ScriptService; import org.opensearch.search.aggregations.support.ValuesSourceRegistry; @@ -165,6 +166,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final IndexNameExpressionResolver expressionResolver; private final Supplier indexSortSupplier; private final ValuesSourceRegistry valuesSourceRegistry; + private final TransportCheckpointPublisher checkpointPublisher; public IndexService( IndexSettings indexSettings, @@ -195,8 +197,8 @@ public IndexService( BooleanSupplier allowExpensiveQueries, IndexNameExpressionResolver expressionResolver, ValuesSourceRegistry valuesSourceRegistry, - IndexStorePlugin.RecoveryStateFactory recoveryStateFactory - ) { + IndexStorePlugin.RecoveryStateFactory recoveryStateFactory, + TransportCheckpointPublisher checkpointPublisher) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; this.indexSettings = indexSettings; @@ -206,6 +208,7 @@ public IndexService( this.circuitBreakerService = circuitBreakerService; this.expressionResolver = expressionResolver; this.valuesSourceRegistry = valuesSourceRegistry; + this.checkpointPublisher = checkpointPublisher; if (needsMapperService(indexSettings, indexCreationContext)) { assert indexAnalyzers != null; this.mapperService = new MapperService( @@ -520,8 +523,8 @@ public synchronized IndexShard createShard( indexingOperationListeners, () -> globalCheckpointSyncer.accept(shardId), retentionLeaseSyncer, - circuitBreakerService - ); + circuitBreakerService, + checkpointPublisher); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); @@ -911,7 +914,9 @@ private void maybeRefreshEngine(boolean force) { if (indexSettings.getRefreshInterval().millis() > 0 || force) { for (IndexShard shard : this.shards.values()) { try { - shard.scheduledRefresh(); + if (shard.routingEntry().primary()) { + shard.scheduledRefresh(); + } } catch (IndexShardClosedException | AlreadyClosedException ex) { // fine - continue; } diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 36fcae1d93d62..a48bc0f311ed7 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -508,7 +508,7 @@ public final class IndexSettings { */ public static final Setting INDEX_SEGMENT_REPLICATION_SETTING = Setting.boolSetting( "index.replication.segment_replication", - false, + true, Property.IndexScope, Property.Final ); diff --git a/server/src/main/java/org/opensearch/index/engine/Engine.java b/server/src/main/java/org/opensearch/index/engine/Engine.java index ed55beb04f0e6..8faa418d13644 100644 --- a/server/src/main/java/org/opensearch/index/engine/Engine.java +++ b/server/src/main/java/org/opensearch/index/engine/Engine.java @@ -249,6 +249,9 @@ public void verifyEngineBeforeIndexClosing() throws IllegalStateException { } } + public void updateCurrentInfos(byte[] infosBytes, long gen) throws IOException {}; + + /** * A throttling class that can be activated, causing the * {@code acquireThrottle} method to block on a lock when throttling @@ -1196,6 +1199,10 @@ public abstract void forceMerge( */ public abstract IndexCommitRef acquireLastIndexCommit(boolean flushFirst) throws EngineException; + public SegmentInfosRef getLatestSegmentInfosSafe() { return null; }; + + public SegmentInfos getLatestSegmentInfos() { return null; }; + /** * Snapshots the most recent safe index commit from the engine. */ @@ -1999,6 +2006,28 @@ public IndexCommit getIndexCommit() { } } + public static class SegmentInfosRef implements Closeable { + private final AtomicBoolean closed = new AtomicBoolean(); + private final CheckedRunnable onClose; + private final SegmentInfos segmentInfos; + + public SegmentInfosRef(SegmentInfos segmentInfos, CheckedRunnable onClose) { + this.segmentInfos = segmentInfos; + this.onClose = onClose; + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + onClose.run(); + } + } + + public SegmentInfos getSegmentInfos() { + return segmentInfos; + } + } + public void onSettingsChanged(TimeValue translogRetentionAge, ByteSizeValue translogRetentionSize, long softDeletesRetentionOps) { } 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 fd02f3049cc8e..67cc6d3b643ef 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -95,6 +95,7 @@ public final class EngineConfig { private final CircuitBreakerService circuitBreakerService; private final LongSupplier globalCheckpointSupplier; private final Supplier retentionLeasesSupplier; + private boolean isPrimary; /** * A supplier of the outstanding retention leases. This is used during merged operations to determine which operations that have been @@ -169,8 +170,8 @@ public EngineConfig( LongSupplier globalCheckpointSupplier, Supplier retentionLeasesSupplier, LongSupplier primaryTermSupplier, - TombstoneDocSupplier tombstoneDocSupplier - ) { + TombstoneDocSupplier tombstoneDocSupplier, + boolean isPrimary) { this( shardId, threadPool, @@ -193,7 +194,7 @@ public EngineConfig( circuitBreakerService, globalCheckpointSupplier, retentionLeasesSupplier, - primaryTermSupplier, + isPrimary, primaryTermSupplier, tombstoneDocSupplier ); } @@ -223,7 +224,7 @@ public EngineConfig( CircuitBreakerService circuitBreakerService, LongSupplier globalCheckpointSupplier, Supplier retentionLeasesSupplier, - LongSupplier primaryTermSupplier, + boolean isPrimary, LongSupplier primaryTermSupplier, TombstoneDocSupplier tombstoneDocSupplier ) { this.shardId = shardId; @@ -237,6 +238,7 @@ public EngineConfig( this.codecService = codecService; this.eventListener = eventListener; codecName = indexSettings.getValue(INDEX_CODEC_SETTING); + this.isPrimary = isPrimary; // We need to make the indexing buffer for this shard at least as large // as the amount of memory that is available for all engines on the // local node so that decisions to flush segments to disk are made by @@ -458,6 +460,10 @@ public LongSupplier getPrimaryTermSupplier() { return primaryTermSupplier; } + public boolean isPrimary() { + return isPrimary; + } + /** * 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 dffdb58bfec1e..869c274ba4841 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java @@ -112,7 +112,8 @@ public EngineConfig newEngineConfig( LongSupplier globalCheckpointSupplier, Supplier retentionLeasesSupplier, LongSupplier primaryTermSupplier, - EngineConfig.TombstoneDocSupplier tombstoneDocSupplier + EngineConfig.TombstoneDocSupplier tombstoneDocSupplier, + Boolean isPrimary ) { return new EngineConfig( @@ -137,6 +138,7 @@ public EngineConfig newEngineConfig( circuitBreakerService, globalCheckpointSupplier, retentionLeasesSupplier, + isPrimary, primaryTermSupplier, tombstoneDocSupplier ); 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 9d36f170d1df6..1fc998164ac12 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -48,6 +48,7 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.ShuffleForcedMergePolicy; import org.apache.lucene.index.SoftDeletesRetentionMergePolicy; +import org.apache.lucene.index.StandardDirectoryReader; import org.apache.lucene.index.Term; import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; @@ -61,6 +62,10 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.Weight; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.BufferedChecksumIndexInput; +import org.apache.lucene.store.ByteBuffersDataInput; +import org.apache.lucene.store.ByteBuffersIndexInput; +import org.apache.lucene.store.ChecksumIndexInput; import org.apache.lucene.store.Directory; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; @@ -114,6 +119,7 @@ import java.io.Closeable; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.file.Path; import java.util.Arrays; import java.util.HashMap; @@ -271,11 +277,19 @@ public InternalEngine(EngineConfig engineConfig) { translog::getLastSyncedGlobalCheckpoint ); this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); - writer = createWriter(); - bootstrapAppendOnlyInfoFromWriter(writer); - final Map commitData = commitDataAsMap(writer); - historyUUID = loadHistoryUUID(commitData); - forceMergeUUID = commitData.get(FORCE_MERGE_UUID_KEY); + // TODO: Segrep - should have a separate read only engine rather than all this conditional logic. + if (engineConfig.isPrimary()) { + writer = createWriter(); + bootstrapAppendOnlyInfoFromWriter(writer); + final Map commitData = commitDataAsMap(writer); + historyUUID = loadHistoryUUID(commitData); + forceMergeUUID = commitData.get(FORCE_MERGE_UUID_KEY); + } else { + // Segrep - hack to make this engine read only and not use + writer = null; + historyUUID = null; + forceMergeUUID = null; + } indexWriter = writer; } catch (IOException | TranslogCorruptedException e) { throw new EngineCreationFailureException(shardId, "failed to create engine", e); @@ -331,6 +345,24 @@ public InternalEngine(EngineConfig engineConfig) { logger.trace("created new InternalEngine"); } + @Override + public void updateCurrentInfos(byte[] infosBytes, long gen) throws IOException { + assert engineConfig.isPrimary() == false : "Only replicas should update Infos"; + SegmentInfos infos = SegmentInfos.readCommit(this.store.directory(), + toIndexInput(infosBytes), + gen); + assert gen == infos.getGeneration(); + externalReaderManager.internalReaderManager.setCurrentInfos(infos); + externalReaderManager.maybeRefresh(); + } + + private ChecksumIndexInput toIndexInput(byte[] input) { + return new BufferedChecksumIndexInput( + new ByteBuffersIndexInput( + new ByteBuffersDataInput( + Arrays.asList(ByteBuffer.wrap(input))), "SegmentInfos")); + } + private LocalCheckpointTracker createLocalCheckpointTracker( BiFunction localCheckpointTrackerSupplier ) throws IOException { @@ -553,7 +585,9 @@ private void recoverFromTranslogInternal(TranslogRecoveryRunner translogRecovery translog.currentFileGeneration() ) ); - flush(false, true); + if (engineConfig.isPrimary()) { + flush(false, true); + } translog.trimUnreferencedReaders(); } @@ -662,7 +696,9 @@ public Translog.Location getTranslogLastWriteLocation() { private void revisitIndexDeletionPolicyOnTranslogSynced() throws IOException { if (combinedDeletionPolicy.hasUnreferencedCommits()) { - indexWriter.deleteUnusedFiles(); + if (engineConfig.isPrimary()) { + indexWriter.deleteUnusedFiles(); + } } translog.trimUnreferencedReaders(); } @@ -701,7 +737,7 @@ private ExternalReaderManager createReaderManager(RefreshWarmerListener external try { try { final OpenSearchDirectoryReader directoryReader = OpenSearchDirectoryReader.wrap( - DirectoryReader.open(indexWriter), + getDirectoryReader(), shardId ); internalReaderManager = new OpenSearchReaderManager( @@ -728,6 +764,14 @@ private ExternalReaderManager createReaderManager(RefreshWarmerListener external } } + private DirectoryReader getDirectoryReader() throws IOException { + // replicas should create the reader from store, we don't want an open IW on replicas. + if (engineConfig.isPrimary() == false) { + return DirectoryReader.open(store.directory()); + } + return DirectoryReader.open(indexWriter); + } + @Override public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { assert Objects.equals(get.uid().field(), IdFieldMapper.NAME) : get.uid().field(); @@ -1105,20 +1149,22 @@ public IndexResult index(Index index) throws IOException { @Override public Engine.IndexResult addIndexOperationToTranslog(Index index) throws IOException { - IndexingStrategy plan = indexingStrategyForOperation(index); - /** - * Matches the logic in {@link #indexIntoLucene(Index, IndexingStrategy)} - */ - IndexResult indexResult = new IndexResult( - plan.versionForIndexing, - index.primaryTerm(), - index.seqNo(), - plan.currentNotFoundOrDeleted - ); - addIndexOperationToTranslog(index, indexResult); - indexResult.setTook(System.nanoTime() - index.startTime()); - indexResult.freeze(); - return indexResult; + try (Releasable ignored = versionMap.acquireLock(index.uid().bytes())) { + IndexingStrategy plan = indexingStrategyForOperation(index); + /** + * Matches the logic in {@link #indexIntoLucene(Index, IndexingStrategy)} + */ + IndexResult indexResult = new IndexResult( + plan.versionForIndexing, + index.primaryTerm(), + index.seqNo(), + plan.currentNotFoundOrDeleted + ); + addIndexOperationToTranslog(index, indexResult); + indexResult.setTook(System.nanoTime() - index.startTime()); + indexResult.freeze(); + return indexResult; + } } private void addIndexOperationToTranslog(Index index, IndexResult indexResult) throws IOException { @@ -2357,6 +2403,35 @@ public IndexCommitRef acquireLastIndexCommit(final boolean flushFirst) throws En return new Engine.IndexCommitRef(lastCommit, () -> releaseIndexCommit(lastCommit)); } + @Override + public SegmentInfosRef getLatestSegmentInfosSafe() { + assert (engineConfig.isPrimary()); + final SegmentInfos segmentInfos = getLatestSegmentInfos(); + try { + indexWriter.incRefDeleter(segmentInfos); + } catch (IOException e) { + throw new EngineException(shardId, e.getMessage(), e); + } + return new Engine.SegmentInfosRef(segmentInfos, () -> indexWriter.decRefDeleter(segmentInfos)); + } + + @Override + public SegmentInfos getLatestSegmentInfos() { + OpenSearchDirectoryReader reader = null; + try { + reader = externalReaderManager.internalReaderManager.acquire(); + return ((StandardDirectoryReader) reader.getDelegate()).getSegmentInfos(); + } catch (IOException e) { + throw new EngineException(shardId, e.getMessage(), e); + } finally { + try { + externalReaderManager.internalReaderManager.release(reader); + } catch (IOException e) { + throw new EngineException(shardId, e.getMessage(), e); + } + } + } + @Override public IndexCommitRef acquireSafeIndexCommit() throws EngineException { final IndexCommit safeCommit = combinedDeletionPolicy.acquireIndexCommit(true); diff --git a/server/src/main/java/org/opensearch/index/engine/OpenSearchReaderManager.java b/server/src/main/java/org/opensearch/index/engine/OpenSearchReaderManager.java index d68183f9cd07a..a6447c2aab4a1 100644 --- a/server/src/main/java/org/opensearch/index/engine/OpenSearchReaderManager.java +++ b/server/src/main/java/org/opensearch/index/engine/OpenSearchReaderManager.java @@ -33,9 +33,17 @@ package org.opensearch.index.engine; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.function.BiConsumer; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.StandardDirectoryReader; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.SearcherManager; @@ -52,8 +60,12 @@ */ @SuppressForbidden(reason = "reference counting is required here") class OpenSearchReaderManager extends ReferenceManager { + protected static Logger logger = LogManager.getLogger(OpenSearchReaderManager.class); + private final BiConsumer refreshListener; + private volatile SegmentInfos currentInfos; + /** * Creates and returns a new OpenSearchReaderManager from the given * already-opened {@link OpenSearchDirectoryReader}, stealing @@ -78,13 +90,42 @@ protected void decRef(OpenSearchDirectoryReader reference) throws IOException { @Override protected OpenSearchDirectoryReader refreshIfNeeded(OpenSearchDirectoryReader referenceToRefresh) throws IOException { - final OpenSearchDirectoryReader reader = (OpenSearchDirectoryReader) DirectoryReader.openIfChanged(referenceToRefresh); + // This is directly from Lucene's SegmentInfosSearcherManager... + List subs; + if (referenceToRefresh == null) { + subs = null; + } else { + subs = new ArrayList<>(); + for (LeafReaderContext ctx : referenceToRefresh.getDelegate().leaves()) { + subs.add(ctx.reader()); + } + } + + final OpenSearchDirectoryReader reader; + // If not using NRT repl. + if (currentInfos == null) { + reader = (OpenSearchDirectoryReader) DirectoryReader.openIfChanged(referenceToRefresh); + } else { + // Open a new reader, sharing any common segment readers with the old one: + DirectoryReader innerReader = StandardDirectoryReader.open(referenceToRefresh.directory(), currentInfos, subs, null); + reader = OpenSearchDirectoryReader.wrap(innerReader, referenceToRefresh.shardId()); + logger.trace("updated to SegmentInfosVersion=" + currentInfos.getVersion() + " reader=" + innerReader); + } if (reader != null) { refreshListener.accept(reader, referenceToRefresh); } return reader; } + /** + * Switch to new segments, refreshing if necessary. Note that it's the caller job to ensure + * there's a held refCount for the incoming infos, so all files exist. + */ + public synchronized void setCurrentInfos(SegmentInfos infos) throws IOException { + currentInfos = infos; + maybeRefresh(); + } + @Override protected boolean tryIncRef(OpenSearchDirectoryReader reference) { return reference.tryIncRef(); diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index b2986ee3fccdd..91be5def453fa 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -904,22 +904,23 @@ private boolean invariant() { } if (primaryMode && indexSettings.isSoftDeleteEnabled() && hasAllPeerRecoveryRetentionLeases) { + // TODO: Segrep - This blows up during segrep because we don't have a retention lease for replicas on the primary - we can ignore for poc. // all tracked shard copies have a corresponding peer-recovery retention lease - for (final ShardRouting shardRouting : routingTable.assignedShards()) { - if (checkpoints.get(shardRouting.allocationId().getId()).tracked) { - assert retentionLeases.contains( - getPeerRecoveryRetentionLeaseId(shardRouting) - ) : "no retention lease for tracked shard [" + shardRouting + "] in " + retentionLeases; - assert PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals( - retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source() - ) : "incorrect source [" - + retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source() - + "] for [" - + shardRouting - + "] in " - + retentionLeases; - } - } +// for (final ShardRouting shardRouting : routingTable.assignedShards()) { +// if (checkpoints.get(shardRouting.allocationId().getId()).tracked) { +// assert retentionLeases.contains( +// getPeerRecoveryRetentionLeaseId(shardRouting) +// ) : "no retention lease for tracked shard [" + shardRouting + "] in " + retentionLeases; +// assert PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals( +// retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source() +// ) : "incorrect source [" +// + retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source() +// + "] for [" +// + shardRouting +// + "] in " +// + retentionLeases; +// } +// } } return true; diff --git a/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java new file mode 100644 index 0000000000000..9563bcd9100ce --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java @@ -0,0 +1,44 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.shard; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.ReferenceManager; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointPublisher; + +import java.io.IOException; + +/** + * A {@link ReferenceManager.RefreshListener} that publishes a checkpoint to be consumed by replicas. + */ +public class CheckpointRefreshListener implements ReferenceManager.RefreshListener { + + protected static Logger logger = LogManager.getLogger(CheckpointRefreshListener.class); + + private final IndexShard shard; + private final TransportCheckpointPublisher publisher; + + public CheckpointRefreshListener(IndexShard shard, TransportCheckpointPublisher publisher) { + this.shard = shard; + this.publisher = publisher; + } + + @Override + public void beforeRefresh() throws IOException { + // Do nothing + } + + @Override + public void afterRefresh(boolean didRefresh) throws IOException { + if (shard.routingEntry().primary()) { + publisher.publish(shard.getLatestReplicationCheckpoint()); + } + } +} 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 0ad3c2cb0478f..c0e0177fbd3fd 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -160,6 +160,13 @@ import org.opensearch.indices.recovery.RecoveryFailedException; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.checkpoint.PublishCheckpointRequest; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointPublisher; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; +import org.opensearch.indices.replication.copy.ReplicationCheckpoint; +import org.opensearch.indices.replication.copy.ReplicationFailedException; +import org.opensearch.indices.replication.copy.ReplicationState; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.rest.RestStatus; @@ -253,6 +260,8 @@ Runnable getGlobalCheckpointSyncer() { @Nullable private volatile RecoveryState recoveryState; + private volatile ReplicationState replicationState; + private final RecoveryStats recoveryStats = new RecoveryStats(); private final MeanMetric refreshMetric = new MeanMetric(); private final MeanMetric externalRefreshMetric = new MeanMetric(); @@ -295,6 +304,8 @@ Runnable getGlobalCheckpointSyncer() { private final RefreshPendingLocationListener refreshPendingLocationListener; private volatile boolean useRetentionLeasesInPeerRecovery; + private final CheckpointRefreshListener checkpointRefreshListener; + public IndexShard( final ShardRouting shardRouting, final IndexSettings indexSettings, @@ -315,8 +326,8 @@ public IndexShard( final List listeners, final Runnable globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final CircuitBreakerService circuitBreakerService - ) throws IOException { + final CircuitBreakerService circuitBreakerService, + final TransportCheckpointPublisher checkpointPublisher) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); this.shardRouting = shardRouting; @@ -398,6 +409,8 @@ public boolean shouldCache(Query query) { persistMetadata(path, indexSettings, shardRouting, null, logger); this.useRetentionLeasesInPeerRecovery = replicationTracker.hasAllPeerRecoveryRetentionLeases(); this.refreshPendingLocationListener = new RefreshPendingLocationListener(); + this.checkpointRefreshListener = new CheckpointRefreshListener(this, checkpointPublisher); + this.replicationState = new ReplicationState(); } public ThreadPool getThreadPool() { @@ -1267,6 +1280,10 @@ public void refresh(String source) { * Returns how many bytes we are currently moving from heap to disk */ public long getWritingBytes() { + // TODO: Segrep: hack - if not the primary our IW is null and this blows up. + if (shardRouting.primary() == false) { + return 0L; + } Engine engine = getEngineOrNull(); if (engine == null) { return 0; @@ -1489,10 +1506,26 @@ public Engine.IndexCommitRef acquireLastIndexCommit(boolean flushFirst) throws E } } - /** - * Snapshots the most recent safe index commit from the currently running engine. - * All index files referenced by this index commit won't be freed until the commit/snapshot is closed. - */ + public Engine.SegmentInfosRef getLatestSegmentInfosSafe() throws EngineException { + return getEngine().getLatestSegmentInfosSafe(); + } + + public SegmentInfos getLatestSegmentInfos() { + return getEngine().getLatestSegmentInfos(); + } + + public ReplicationCheckpoint getLatestReplicationCheckpoint() { + return new ReplicationCheckpoint(this.shardId, getOperationPrimaryTerm(), getLatestSegmentInfos().getGeneration(), getLocalCheckpoint()); + } + + public void updateCurrentInfos(long gen, byte[] infosBytes) throws IOException { + getEngine().updateCurrentInfos(infosBytes, gen); + } + + /** + * Snapshots the most recent safe index commit from the currently running engine. + * All index files referenced by this index commit won't be freed until the commit/snapshot is closed. + */ public Engine.IndexCommitRef acquireSafeIndexCommit() throws EngineException { final IndexShardState state = this.state; // one time volatile read // we allow snapshot on closed index shard, since we want to do one after we close the shard and before we close the engine @@ -1993,7 +2026,8 @@ public void openEngineAndRecoverFromTranslog() throws IOException { */ public void openEngineAndSkipTranslogRecovery() throws IOException { assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]"; - recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); + // TODO: Segrep - fix initial recovery stages from ReplicationTarget. +// recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); loadGlobalCheckpointToReplicationTracker(); innerOpenEngineAndTranslog(replicationTracker); getEngine().skipTranslogRecovery(); @@ -2029,7 +2063,8 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t // time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during // which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine. onSettingsChanged(); - assert assertSequenceNumbersInCommit(); + // TODO: Segrep - Fix +// assert assertSequenceNumbersInCommit(); recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); } @@ -2202,6 +2237,9 @@ protected final void verifyActive() throws IllegalIndexShardStateException { * Returns number of heap bytes used by the indexing buffer for this shard, or 0 if the shard is closed */ public long getIndexBufferRAMBytesUsed() { + if (shardRouting.primary() == false) { + return 0; + } Engine engine = getEngineOrNull(); if (engine == null) { return 0; @@ -2681,7 +2719,8 @@ public boolean assertRetentionLeasesPersisted() throws IOException { public void syncRetentionLeases() { assert assertPrimaryMode(); verifyNotClosed(); - replicationTracker.renewPeerRecoveryRetentionLeases(); + // TODO: Segrep - Fix retention leases +// replicationTracker.renewPeerRecoveryRetentionLeases(); final Tuple retentionLeases = getRetentionLeases(true); if (retentionLeases.v1()) { logger.trace("syncing retention leases [{}] after expiration check", retentionLeases.v2()); @@ -3008,7 +3047,10 @@ protected Engine getEngineOrNull() { public void startRecovery( RecoveryState recoveryState, - PeerRecoveryTargetService recoveryTargetService, + SegmentReplicationService segmentReplicationService, + SegmentReplicationService.ReplicationListener replicationListener, + PrimaryShardReplicationSource replicationSource, + PeerRecoveryTargetService peerRecoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, BiConsumer mappingUpdateConsumer, @@ -3039,8 +3081,10 @@ public void startRecovery( case PEER: try { markAsRecovering("from " + recoveryState.getSourceNode(), recoveryState); - recoveryTargetService.startRecovery(this, recoveryState.getSourceNode(), recoveryListener); + markAsReplicating(); + segmentReplicationService.startReplication(new ReplicationCheckpoint(this.shardId, -10, -10, 0), this, replicationSource, replicationListener); } catch (Exception e) { + logger.error("Error", e); failShard("corrupted preexisting index", e); recoveryListener.onRecoveryFailure(recoveryState, new RecoveryFailedException(recoveryState, null, e), true); } @@ -3253,13 +3297,14 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) { translogConfig, IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Arrays.asList(refreshListeners, refreshPendingLocationListener), - Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), + Arrays.asList(new RefreshMetricUpdater(refreshMetric), checkpointRefreshListener), indexSort, circuitBreakerService, globalCheckpointSupplier, replicationTracker::getRetentionLeases, () -> getOperationPrimaryTerm(), - tombstoneDocSupplier() + tombstoneDocSupplier(), + shardRouting.primary() ); } @@ -3644,6 +3689,50 @@ public void sync() throws IOException { getEngine().syncTranslog(); } + /** + * Invoked when a new checkpoint is received from a primary shard. Starts the copy process. + */ + public void onNewCheckpoint(final PublishCheckpointRequest request, + final PrimaryShardReplicationSource source, + final SegmentReplicationService segmentReplicationService) { + if (this.replicationState.getStage() == ReplicationState.Stage.INACTIVE) { + try { + final ReplicationCheckpoint checkpoint = request.getCheckpoint(); + logger.trace("Received new checkpoint {}", checkpoint); + // TODO: segrep - these are the states set after we perform our initial store recovery. + if (RecoveryState.Stage.TRANSLOG != this.recoveryState.getStage() && this.state.equals(IndexShardState.RECOVERING) == false) { + logger.debug("Ignore - shard is not started {} {}", recoveryState.getStage(), this.state); + return; + } + segmentReplicationService.startReplication(checkpoint, this, source, new SegmentReplicationService.ReplicationListener() { + @Override + public void onReplicationDone(ReplicationState state) { + logger.trace("Replication complete."); + } + + @Override + public void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + logger.error("Failure", e); + } + }); + } catch (Exception e) { + logger.error("Error", e); + } + } + } + + public ReplicationState getReplicationState() { + return this.replicationState; + } + + public void markAsReplicating() { + replicationState.setStage(ReplicationState.Stage.ACTIVE); + } + + public void finalizeReplication() { + replicationState.setStage(ReplicationState.Stage.INACTIVE); + } + /** * Checks if the underlying storage sync is required. */ @@ -3778,6 +3867,11 @@ ReplicationTracker getReplicationTracker() { * @return true iff the engine got refreshed otherwise false */ public boolean scheduledRefresh() { + // skip if not primary shard. + // TODO: Segrep - should split into primary/replica classes. + if (shardRouting.primary() == false) { + return false; + } verifyNotClosed(); boolean listenerNeedsRefresh = refreshListeners.refreshNeeded(); if (isReadAllowed() && (listenerNeedsRefresh || getEngine().refreshNeeded())) { diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index d822a824b8147..04825ea701fe9 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -104,6 +104,7 @@ import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Iterator; @@ -313,6 +314,10 @@ public MetadataSnapshot getMetadata(IndexCommit commit, boolean lockDirectory) t } } + public MetadataSnapshot getMetadata(SegmentInfos segmentInfos) throws IOException { + return new MetadataSnapshot(segmentInfos, directory, logger); + } + /** * Renames all the given files from the key of the map to the * value of the map. All successfully renamed files are removed from the map in-place. @@ -475,7 +480,7 @@ public static MetadataSnapshot readMetadataSnapshot( Directory dir = new NIOFSDirectory(indexLocation) ) { failIfCorrupted(dir); - return new MetadataSnapshot(null, dir, logger); + return new MetadataSnapshot((IndexCommit) null, dir, logger); } catch (IndexNotFoundException ex) { // that's fine - happens all the time no need to log } catch (FileNotFoundException | NoSuchFileException ex) { @@ -682,7 +687,7 @@ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetadata) thr } } directory.syncMetaData(); - final Store.MetadataSnapshot metadataOrEmpty = getMetadata(null); + final Store.MetadataSnapshot metadataOrEmpty = getMetadata((IndexCommit) null); verifyAfterCleanup(sourceMetadata, metadataOrEmpty); } finally { metadataLock.writeLock().unlock(); @@ -780,6 +785,11 @@ public Set getPendingDeletions() throws IOException { // to be removed once fixed in FilterDirectory. return unwrap(this).getPendingDeletions(); } + + @Override + public void sync(Collection names) throws IOException { + // Do nothing. + } } /** @@ -822,6 +832,14 @@ public MetadataSnapshot(Map metadata, Map builder = new HashMap<>(); Map commitUserDataBuilder = new HashMap<>(); try { - final SegmentInfos segmentCommitInfos = Store.readSegmentsInfo(commit, directory); - numDocs = Lucene.getNumDocs(segmentCommitInfos); - commitUserDataBuilder.putAll(segmentCommitInfos.getUserData()); + numDocs = Lucene.getNumDocs(infos); + commitUserDataBuilder.putAll(infos.getUserData()); // we don't know which version was used to write so we take the max version. - Version maxVersion = segmentCommitInfos.getMinSegmentLuceneVersion(); - for (SegmentCommitInfo info : segmentCommitInfos) { + Version maxVersion = infos.getMinSegmentLuceneVersion(); + for (SegmentCommitInfo info : infos) { final Version version = info.info.getVersion(); if (version == null) { // version is written since 3.1+: we should have already hit IndexFormatTooOld. throw new IllegalArgumentException("expected valid version value: " + info.info.toString()); } - if (version.onOrAfter(maxVersion)) { + // TODO: Version is null here from getMinSegmentLuceneVersion... not sure why + if (maxVersion == null || version.onOrAfter(maxVersion)) { maxVersion = version; } for (String file : info.files()) { @@ -910,7 +928,7 @@ static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, Logg if (maxVersion == null) { maxVersion = org.opensearch.Version.CURRENT.minimumIndexCompatibilityVersion().luceneVersion; } - final String segmentsFile = segmentCommitInfos.getSegmentsFileName(); + final String segmentsFile = infos.getSegmentsFileName(); checksumFromLuceneFile(directory, segmentsFile, builder, logger, maxVersion, true); } catch (CorruptIndexException | IndexNotFoundException | IndexFormatTooOldException | IndexFormatTooNewException ex) { // we either know the index is corrupted or it's just not there @@ -922,8 +940,8 @@ static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, Logg // TODO we should check the checksum in lucene if we hit an exception logger.warn( () -> new ParameterizedMessage( - "failed to build store metadata. checking segment info integrity " + "(with commit [{}])", - commit == null ? "no" : "yes" + "failed to build store metadata. checking segment info integrity " + "(with infos [{}])", + infos == null ? "no" : "yes" ), ex ); @@ -940,6 +958,16 @@ static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, Logg return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserDataBuilder), numDocs); } + static LoadedMetadata loadMetadata(IndexCommit commit, Directory directory, Logger logger) throws IOException { + SegmentInfos infos; + try { + infos = Store.readSegmentsInfo(commit, directory); + } catch (Exception e) { + throw e; + } + return loadMetadata(infos, directory, logger); + } + private static void checksumFromLuceneFile( Directory directory, String file, diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index 6f777f4833c6a..49cedb6a3723c 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -74,6 +74,7 @@ import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.flush.SyncedFlushService; import org.opensearch.indices.mapper.MapperRegistry; +import org.opensearch.indices.replication.checkpoint.TransportPublishShardCheckpointAction; import org.opensearch.indices.store.IndicesStore; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; import org.opensearch.plugins.MapperPlugin; @@ -277,6 +278,7 @@ protected void configure() { bind(PrimaryReplicaSyncer.class).asEagerSingleton(); bind(RetentionLeaseSyncAction.class).asEagerSingleton(); bind(RetentionLeaseBackgroundSyncAction.class).asEagerSingleton(); + bind(TransportPublishShardCheckpointAction.class).asEagerSingleton(); bind(RetentionLeaseSyncer.class).asEagerSingleton(); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 0fa57bd3fb33f..341df65205f9c 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -137,6 +137,9 @@ import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointPublisher; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.node.Node; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.plugins.PluginsService; @@ -737,8 +740,8 @@ private synchronized IndexService createIndexService( indicesFieldDataCache, namedWriteableRegistry, this::isIdFieldDataEnabled, - valuesSourceRegistry - ); + valuesSourceRegistry, + new TransportCheckpointPublisher(client)); } private EngineConfigFactory getEngineConfigFactory(final IndexSettings idxSettings) { @@ -833,6 +836,9 @@ public synchronized void verifyIndexMetadata(IndexMetadata metadata, IndexMetada @Override public IndexShard createShard( final ShardRouting shardRouting, + final SegmentReplicationService replicaService, + final SegmentReplicationService.ReplicationListener replicationListener, + final PrimaryShardReplicationSource replicationSource, final PeerRecoveryTargetService recoveryTargetService, final PeerRecoveryTargetService.RecoveryListener recoveryListener, final RepositoriesService repositoriesService, @@ -849,7 +855,7 @@ public IndexShard createShard( RecoveryState recoveryState = indexService.createRecoveryState(shardRouting, targetNode, sourceNode); IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer, retentionLeaseSyncer); indexShard.addShardFailureCallback(onShardFailure); - indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, (type, mapping) -> { + indexShard.startRecovery(recoveryState, replicaService, replicationListener, replicationSource, recoveryTargetService, recoveryListener, repositoriesService, (type, mapping) -> { assert recoveryState.getRecoverySource() .getType() == RecoverySource.Type.LOCAL_SHARDS : "mapping update consumer only required by local shards recovery"; client.admin() diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index ee0d76b4a9117..a71fac14df98f 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -81,6 +81,10 @@ import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.recovery.RecoveryFailedException; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; +import org.opensearch.indices.replication.copy.ReplicationFailedException; +import org.opensearch.indices.replication.copy.ReplicationState; import org.opensearch.repositories.RepositoriesService; import org.opensearch.search.SearchService; import org.opensearch.snapshots.SnapshotShardsService; @@ -113,6 +117,9 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple final AllocatedIndices> indicesService; private final ClusterService clusterService; private final ThreadPool threadPool; + + private final SegmentReplicationService replicationReplicaService; + private final PrimaryShardReplicationSource replicationSource; private final PeerRecoveryTargetService recoveryTargetService; private final ShardStateAction shardStateAction; private final NodeMappingRefreshAction nodeMappingRefreshAction; @@ -149,13 +156,16 @@ public IndicesClusterStateService( final SnapshotShardsService snapshotShardsService, final PrimaryReplicaSyncer primaryReplicaSyncer, final GlobalCheckpointSyncAction globalCheckpointSyncAction, - final RetentionLeaseSyncer retentionLeaseSyncer - ) { + final RetentionLeaseSyncer retentionLeaseSyncer, + final SegmentReplicationService replicationReplicaService, + final PrimaryShardReplicationSource replicationSource) { this( settings, indicesService, clusterService, threadPool, + replicationReplicaService, + replicationSource, recoveryTargetService, shardStateAction, nodeMappingRefreshAction, @@ -176,6 +186,8 @@ public IndicesClusterStateService( final AllocatedIndices> indicesService, final ClusterService clusterService, final ThreadPool threadPool, + final SegmentReplicationService replicationReplicaService, + final PrimaryShardReplicationSource replicationSource, final PeerRecoveryTargetService recoveryTargetService, final ShardStateAction shardStateAction, final NodeMappingRefreshAction nodeMappingRefreshAction, @@ -189,6 +201,8 @@ public IndicesClusterStateService( final RetentionLeaseSyncer retentionLeaseSyncer ) { this.settings = settings; + this.replicationReplicaService = replicationReplicaService; + this.replicationSource = replicationSource; this.buildInIndexListener = Arrays.asList( peerRecoverySourceService, recoveryTargetService, @@ -624,6 +638,9 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR logger.debug("{} creating shard with primary term [{}]", shardRouting.shardId(), primaryTerm); indicesService.createShard( shardRouting, + replicationReplicaService, + new ReplicationListener(), + replicationSource, recoveryTargetService, new RecoveryListener(shardRouting, primaryTerm), repositoriesService, @@ -741,6 +758,22 @@ private static DiscoveryNode findSourceNodeForPeerRecovery( return sourceNode; } + + private class ReplicationListener implements SegmentReplicationService.ReplicationListener { + + private ReplicationListener() {} + + @Override + public void onReplicationDone(final ReplicationState state) { + logger.info("Replication Done"); + } + + @Override + public void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + logger.error("Replication failed", e); + } + } + private class RecoveryListener implements PeerRecoveryTargetService.RecoveryListener { /** @@ -990,6 +1023,9 @@ U createIndex(IndexMetadata indexMetadata, List builtInIndex */ T createShard( ShardRouting shardRouting, + SegmentReplicationService replicaService, + SegmentReplicationService.ReplicationListener replicationListener, + PrimaryShardReplicationSource replicationSource, PeerRecoveryTargetService recoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, diff --git a/server/src/main/java/org/opensearch/indices/flush/SyncedFlushService.java b/server/src/main/java/org/opensearch/indices/flush/SyncedFlushService.java index 88c1fd03d9c54..e96d700b992ce 100644 --- a/server/src/main/java/org/opensearch/indices/flush/SyncedFlushService.java +++ b/server/src/main/java/org/opensearch/indices/flush/SyncedFlushService.java @@ -164,6 +164,9 @@ public void onFailure(Exception e) { } private void performNormalFlushOnInactive(IndexShard shard) { + if (shard.routingEntry().primary() == false) { + return; + } logger.debug("flushing shard {} on inactive", shard.routingEntry()); shard.getThreadPool().executor(ThreadPool.Names.FLUSH).execute(new AbstractRunnable() { @Override diff --git a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java index f0295fdd2fbef..f10fc6e834703 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java @@ -79,6 +79,7 @@ public MultiFileWriter(Store store, RecoveryState.Index indexState, String tempF final Map tempFileNames = ConcurrentCollections.newConcurrentMap(); + public void writeFileChunk(StoreFileMetadata fileMetadata, long position, BytesReference content, boolean lastChunk) throws IOException { assert Transports.assertNotTransportThread("multi_file_writer"); diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 8c959ab02b7b3..50f377b6f40a1 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -215,7 +215,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi final String actionName; final TransportRequest requestToSend; final StartRecoveryRequest startRequest; - final RecoveryState.Timer timer; + final Timer timer; CancellableThreads cancellableThreads; try (RecoveryRef recoveryRef = onGoingRecoveries.getRecovery(recoveryId)) { if (recoveryRef == null) { @@ -640,9 +640,9 @@ private class RecoveryResponseHandler implements TransportResponseHandler= 0) { - return time; - } - return Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startNanoTime)); - } - - /** Returns stop time in millis */ - public synchronized long stopTime() { - return stopTime; - } - - public synchronized void stop() { - assert stopTime == 0 : "already stopped"; - stopTime = Math.max(System.currentTimeMillis(), startTime); - time = TimeValue.nsecToMSec(System.nanoTime() - startNanoTime); - assert time >= 0; - } - - public synchronized void reset() { - startTime = 0; - startNanoTime = 0; - time = -1; - stopTime = 0; - } - - // for tests - public long getStartNanoTime() { - return startNanoTime; - } - } - public static class VerifyIndex extends Timer implements ToXContentFragment, Writeable { private volatile long checkIndexTime; diff --git a/server/src/main/java/org/opensearch/indices/recovery/Timer.java b/server/src/main/java/org/opensearch/indices/recovery/Timer.java new file mode 100644 index 0000000000000..23b5d671567e0 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/recovery/Timer.java @@ -0,0 +1,94 @@ +/* + * 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.indices.recovery; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.unit.TimeValue; + +import java.io.IOException; + +public class Timer implements Writeable { + public long startTime = 0; + public long startNanoTime = 0; + public long time = -1; + public long stopTime = 0; + + public Timer() { + } + + public Timer(StreamInput in) throws IOException { + startTime = in.readVLong(); + startNanoTime = in.readVLong(); + stopTime = in.readVLong(); + time = in.readVLong(); + } + + @Override + public synchronized void writeTo(StreamOutput out) throws IOException { + out.writeVLong(startTime); + out.writeVLong(startNanoTime); + out.writeVLong(stopTime); + // write a snapshot of current time, which is not per se the time field + out.writeVLong(time()); + } + + public synchronized void start() { + assert startTime == 0 : "already started"; + startTime = System.currentTimeMillis(); + startNanoTime = System.nanoTime(); + } + + /** + * Returns start time in millis + */ + public synchronized long startTime() { + return startTime; + } + + /** + * Returns elapsed time in millis, or 0 if timer was not started + */ + public synchronized long time() { + if (startNanoTime == 0) { + return 0; + } + if (time >= 0) { + return time; + } + return Math.max(0, TimeValue.nsecToMSec(System.nanoTime() - startNanoTime)); + } + + /** + * Returns stop time in millis + */ + public synchronized long stopTime() { + return stopTime; + } + + public synchronized void stop() { + assert stopTime == 0 : "already stopped"; + stopTime = Math.max(System.currentTimeMillis(), startTime); + time = TimeValue.nsecToMSec(System.nanoTime() - startNanoTime); + assert time >= 0; + } + + public synchronized void reset() { + startTime = 0; + startNanoTime = 0; + time = -1; + stopTime = 0; + } + + // for tests + public long getStartNanoTime() { + return startNanoTime; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationService.java new file mode 100644 index 0000000000000..3edad128e7495 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationService.java @@ -0,0 +1,182 @@ +/* + * 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.indices.replication; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.common.util.concurrent.AbstractRunnable; +import org.opensearch.index.shard.IndexEventListener; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.Timer; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; +import org.opensearch.indices.replication.copy.ReplicationCheckpoint; +import org.opensearch.indices.replication.copy.ReplicationCollection; +import org.opensearch.indices.replication.copy.ReplicationFailedException; +import org.opensearch.indices.replication.copy.ReplicationState; +import org.opensearch.indices.replication.copy.ReplicationTarget; +import org.opensearch.threadpool.ThreadPool; + +/** + * Orchestrator of replication events. + */ +public class SegmentReplicationService implements IndexEventListener { + + private static final Logger logger = LogManager.getLogger(SegmentReplicationService.class); + + private final ThreadPool threadPool; + private final RecoverySettings recoverySettings; + + public ReplicationCollection getOnGoingReplications() { + return onGoingReplications; + } + + private final ReplicationCollection onGoingReplications; + + public SegmentReplicationService(final ThreadPool threadPool, + final RecoverySettings recoverySettings) { + this.threadPool = threadPool; + this.recoverySettings = recoverySettings; + this.onGoingReplications = new ReplicationCollection(logger, threadPool); + } + + @Override + public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { + if (indexShard != null) { + onGoingReplications.cancelRecoveriesForShard(shardId, "shard closed"); + } + } + + public void startReplication(final ReplicationCheckpoint checkpoint, final IndexShard indexShard, PrimaryShardReplicationSource source, final ReplicationListener listener) { + final long replicationId = onGoingReplications.startReplication(checkpoint, indexShard, source, listener, recoverySettings.activityTimeout()); + threadPool.generic().execute(new ReplicationRunner(replicationId)); + } + + private void doReplication(final long replicationId) { + final Timer timer; + try (ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.getReplication(replicationId)) { + if (replicationRef == null) { + logger.trace("not running replication with id [{}] - can not find it (probably finished)", replicationId); + return; + } + final ReplicationTarget replicationTarget = replicationRef.target(); + timer = replicationTarget.state().getTimer(); + final IndexShard indexShard = replicationTarget.getIndexShard(); + + try { + logger.trace("{} preparing shard for replication", indexShard.shardId()); + } catch (final Exception e) { + // this will be logged as warning later on... + logger.error("unexpected error while preparing shard for peer replication, failing replication", e); + onGoingReplications.failReplication( + replicationId, + new ReplicationFailedException(indexShard, "failed to prepare shard for replication", e), + true + ); + return; + } + ReplicationResponseHandler listener = new ReplicationResponseHandler(replicationId, indexShard, timer); + replicationTarget.startReplication(listener); + } + } + + class ReplicationRunner extends AbstractRunnable { + + final long replicationId; + + ReplicationRunner(long replicationId) { + this.replicationId = replicationId; + } + + @Override + public void onFailure(Exception e) { + try (ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.getReplication(replicationId)) { + if (replicationRef != null) { + logger.error(() -> new ParameterizedMessage("unexpected error during replication [{}], failing shard", replicationId), e); + onGoingReplications.failReplication( + replicationId, + new ReplicationFailedException(replicationRef.target().getIndexShard(), "unexpected error", e), + true // be safe + ); + } else { + logger.debug( + () -> new ParameterizedMessage("unexpected error during replication, but replication id [{}] is finished", replicationId), + e + ); + } + } + } + + @Override + public void doRun() { + doReplication(replicationId); + } + } + + private class ReplicationResponseHandler implements ActionListener { + + private final long replicationId; + private final IndexShard shard; + private final Timer timer; + + private ReplicationResponseHandler(final long id, final IndexShard shard, final Timer timer) { + this.replicationId = id; + this.timer = timer; + this.shard = shard; + } + + @Override + public void onResponse(ReplicationResponse replicationResponse) { +// final TimeValue replicationTime = new TimeValue(timer.time()); + logger.trace("Replication complete {}", replicationId); + onGoingReplications.markReplicationAsDone(replicationId); + shard.finalizeReplication(); + } + + @Override + public void onFailure(Exception e) { + if (logger.isTraceEnabled()) { + logger.trace( + () -> new ParameterizedMessage( + "[{}][{}] Got exception on replication", + shard.shardId().getIndex().getName(), + shard.shardId().id() + ), + e + ); + } + Throwable cause = ExceptionsHelper.unwrapCause(e); + if (cause instanceof CancellableThreads.ExecutionCancelledException) { + // this can also come from the source wrapped in a RemoteTransportException + onGoingReplications.failReplication( + replicationId, + new ReplicationFailedException(shard, "source has canceled the replication", cause), + false + ); + return; + } + onGoingReplications.failReplication(replicationId, new ReplicationFailedException(shard, e), true); + } + } + + public interface ReplicationListener { + void onReplicationDone(ReplicationState state); + + void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure); + } +} + diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java new file mode 100644 index 0000000000000..1e8379df555d7 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -0,0 +1,19 @@ +/* + * 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.indices.replication.checkpoint; + +import org.opensearch.action.ActionType; +import org.opensearch.action.admin.indices.refresh.RefreshResponse; + +public class PublishCheckpointAction extends ActionType { + public static final PublishCheckpointAction INSTANCE = new PublishCheckpointAction(); + public static final String NAME = "indices:admin/publishCheckpoint"; + + private PublishCheckpointAction() {super(NAME, RefreshResponse::new);} +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointRequest.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointRequest.java new file mode 100644 index 0000000000000..440ced422b8b9 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointRequest.java @@ -0,0 +1,48 @@ +/* + * 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.indices.replication.checkpoint; + +import org.opensearch.action.support.broadcast.BroadcastRequest; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.indices.replication.copy.ReplicationCheckpoint; + +import java.io.IOException; + +public class PublishCheckpointRequest extends BroadcastRequest { + + private ReplicationCheckpoint checkpoint; + + public PublishCheckpointRequest(ReplicationCheckpoint checkpoint, String... indices) { + super(indices); + this.checkpoint = checkpoint; + } + + public PublishCheckpointRequest(StreamInput in) throws IOException { + super(in); + this.checkpoint = new ReplicationCheckpoint(in); + } + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + checkpoint.writeTo(out); + } + + @Override + public String toString() { + return "PublishCheckpointRequest{" + + "checkpoint=" + checkpoint + + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ShardPublishCheckpointRequest.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ShardPublishCheckpointRequest.java new file mode 100644 index 0000000000000..61bfac16812b3 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ShardPublishCheckpointRequest.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.opensearch.action.support.replication.ReplicationRequest; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.index.shard.ShardId; + +import java.io.IOException; + +public class ShardPublishCheckpointRequest extends ReplicationRequest { + + private final PublishCheckpointRequest request; + + public ShardPublishCheckpointRequest(PublishCheckpointRequest request, ShardId shardId) { + super(shardId); + this.request = request; + } + + public ShardPublishCheckpointRequest(StreamInput in) throws IOException { + super(in); + this.request = new PublishCheckpointRequest(in); + } + + PublishCheckpointRequest getRequest() { + return request; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + request.writeTo(out); + } + + @Override + public String toString() { + return "ShardPublishCheckpointRequest{" + shardId + "}"; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointInfoResponse.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointInfoResponse.java new file mode 100644 index 0000000000000..132e760ed94c3 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointInfoResponse.java @@ -0,0 +1,58 @@ +/* + * 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.indices.replication.checkpoint; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.index.store.Store; +import org.opensearch.indices.replication.copy.ReplicationCheckpoint; +import org.opensearch.transport.TransportResponse; + +import java.io.IOException; + +public class TransportCheckpointInfoResponse extends TransportResponse { + + private final ReplicationCheckpoint checkpoint; + private final Store.MetadataSnapshot snapshot; + private final byte[] infosBytes; + + public TransportCheckpointInfoResponse(final ReplicationCheckpoint checkpoint, + final Store.MetadataSnapshot snapshot, + final byte[] infosBytes) { + this.checkpoint = checkpoint; + this.snapshot = snapshot; + this.infosBytes = infosBytes; + } + + public TransportCheckpointInfoResponse(StreamInput in) throws IOException { + this.checkpoint = new ReplicationCheckpoint(in); + this.snapshot = new Store.MetadataSnapshot(in); + this.infosBytes = in.readByteArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + checkpoint.writeTo(out); + snapshot.writeTo(out); + out.writeByteArray(infosBytes); + } + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + public Store.MetadataSnapshot getSnapshot() { + return snapshot; + } + + public byte[] getInfosBytes() { + return infosBytes; + } + +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointPublisher.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointPublisher.java new file mode 100644 index 0000000000000..bbd9eff4d230b --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportCheckpointPublisher.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.indices.refresh.RefreshResponse; +import org.opensearch.client.Client; +import org.opensearch.indices.replication.copy.ReplicationCheckpoint; + +public class TransportCheckpointPublisher { + + protected static Logger logger = LogManager.getLogger(TransportCheckpointPublisher.class); + + private final Client client; + + public TransportCheckpointPublisher( + Client client) { + this.client = client; + } + + public void publish(ReplicationCheckpoint checkpoint) { + logger.trace("Publishing Checkpoint {}", checkpoint); + client.admin() + .indices() + .publishCheckpoint(new PublishCheckpointRequest(checkpoint), + new ActionListener() { + @Override + public void onResponse(RefreshResponse response) { + logger.trace("Successfully published checkpoints"); + } + + @Override + public void onFailure(Exception e) { + logger.error("Publishing Checkpoints from primary to replicas failed", e); + } + }); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishCheckpointAction.java new file mode 100644 index 0000000000000..1a8d0afc20b5e --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishCheckpointAction.java @@ -0,0 +1,63 @@ +/* + * 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.indices.replication.checkpoint; + +import org.opensearch.action.admin.indices.refresh.RefreshResponse; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.DefaultShardOperationFailedException; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.action.support.replication.TransportBroadcastReplicationAction; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.index.shard.ShardId; +import org.opensearch.transport.TransportService; + +import java.util.List; + +public class TransportPublishCheckpointAction extends TransportBroadcastReplicationAction< + PublishCheckpointRequest, + RefreshResponse, + ShardPublishCheckpointRequest, + ReplicationResponse> { + + @Inject + public TransportPublishCheckpointAction( + ClusterService clusterService, + TransportService transportService, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, + TransportPublishShardCheckpointAction shardCheckpointAction + ) { + super( + PublishCheckpointAction.NAME, + PublishCheckpointRequest::new, + clusterService, + transportService, + actionFilters, + indexNameExpressionResolver, + shardCheckpointAction + ); + } + + @Override + protected ReplicationResponse newShardResponse() { + return new ReplicationResponse(); + } + + @Override + protected ShardPublishCheckpointRequest newShardRequest(PublishCheckpointRequest request, ShardId shardId) { + return new ShardPublishCheckpointRequest(request, shardId); + } + + @Override + protected RefreshResponse newResponse(int successfulShards, int failedShards, int totalNumCopies, List shardFailures) { + return new RefreshResponse(totalNumCopies, successfulShards, failedShards, shardFailures); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishShardCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishShardCheckpointAction.java new file mode 100644 index 0000000000000..84c4bbb1aaaad --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/TransportPublishShardCheckpointAction.java @@ -0,0 +1,90 @@ +/* + * 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.indices.replication.checkpoint; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.action.support.replication.TransportReplicationAction; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; + +public class TransportPublishShardCheckpointAction extends TransportReplicationAction< + ShardPublishCheckpointRequest, + ShardPublishCheckpointRequest, + ReplicationResponse> { + + protected static Logger logger = LogManager.getLogger(TransportPublishShardCheckpointAction.class); + + public static final String ACTION_NAME = PublishCheckpointAction.NAME + "[s]"; + + private final SegmentReplicationService replicationService; + private final PrimaryShardReplicationSource source; + + @Inject + public TransportPublishShardCheckpointAction( + Settings settings, + TransportService transportService, + ClusterService clusterService, + IndicesService indicesService, + ThreadPool threadPool, + ShardStateAction shardStateAction, + ActionFilters actionFilters, + SegmentReplicationService segmentCopyService, + PrimaryShardReplicationSource source) { + super( + settings, + ACTION_NAME, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + actionFilters, + ShardPublishCheckpointRequest::new, + ShardPublishCheckpointRequest::new, + ThreadPool.Names.SNAPSHOT + ); + this.replicationService = segmentCopyService; + this.source = source; + } + + @Override + protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { + return new ReplicationResponse(in); + } + + @Override + protected void shardOperationOnPrimary(ShardPublishCheckpointRequest shardRequest, IndexShard primary, ActionListener> listener) { + ActionListener.completeWith(listener, () -> new PrimaryResult<>(shardRequest, new ReplicationResponse())); + } + + @Override + protected void shardOperationOnReplica(ShardPublishCheckpointRequest shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + PublishCheckpointRequest request = shardRequest.getRequest(); + logger.trace("Checkpoint received on replica {}", request); + replica.onNewCheckpoint(request, source, replicationService); + return new ReplicaResult(); + }); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/CopyState.java b/server/src/main/java/org/opensearch/indices/replication/copy/CopyState.java new file mode 100644 index 0000000000000..d47751786d339 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/CopyState.java @@ -0,0 +1,73 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.opensearch.common.util.concurrent.AbstractRefCounted; +import org.opensearch.index.engine.Engine; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.Store; + +import java.io.IOException; +import java.io.UncheckedIOException; + +public class CopyState extends AbstractRefCounted { + + private final Engine.SegmentInfosRef segmentInfosRef; + private final ReplicationCheckpoint checkpoint; + private final Store.MetadataSnapshot metadataSnapshot; + private final byte[] infosBytes; + + CopyState(IndexShard shard) throws IOException { + super("replication-nrt-state"); + this.segmentInfosRef = shard.getLatestSegmentInfosSafe(); + final SegmentInfos segmentInfos = segmentInfosRef.getSegmentInfos(); + this.checkpoint = new ReplicationCheckpoint(shard.shardId(), shard.getOperationPrimaryTerm(), segmentInfos.getGeneration(), shard.getLocalCheckpoint()); + this.metadataSnapshot = shard.store().getMetadata(segmentInfos); + ByteBuffersDataOutput buffer = new ByteBuffersDataOutput(); + try (ByteBuffersIndexOutput tmpIndexOutput = + new ByteBuffersIndexOutput(buffer, "temporary", "temporary")) { + segmentInfos.write(tmpIndexOutput); + } + this.infosBytes = buffer.toArrayCopy(); + } + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + public Store.MetadataSnapshot getMetadataSnapshot() { + return metadataSnapshot; + } + + public byte[] getInfosBytes() { + return infosBytes; + } + + @Override + protected void closeInternal() { + try { + segmentInfosRef.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public String toString() { + return "CopyState{" + + "SegmentInfosRef=" + segmentInfosRef + + ", checkpoint=" + checkpoint + + ", metadataSnapshot=" + metadataSnapshot + + ", refcount=" + refCount() + + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesRequest.java b/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesRequest.java new file mode 100644 index 0000000000000..d38b7c59f5985 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesRequest.java @@ -0,0 +1,56 @@ +/* + * 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.indices.replication.copy; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.index.store.StoreFileMetadata; + +import java.io.IOException; +import java.util.List; + +public class GetFilesRequest extends SegmentReplicationTransportRequest { + + private final List filesToFetch; + private final ReplicationCheckpoint checkpoint; + + public GetFilesRequest(long replicationId, + String targetAllocationId, + DiscoveryNode targetNode, + List filesToFetch, + ReplicationCheckpoint checkpoint) { + super(replicationId, targetAllocationId, targetNode); + this.filesToFetch = filesToFetch; + this.checkpoint = checkpoint; + } + + public GetFilesRequest(StreamInput in) throws IOException { + super(in); + this.filesToFetch = in.readList(StoreFileMetadata::new); + this.checkpoint = new ReplicationCheckpoint(in); + } + + public List getFilesToFetch() { + return filesToFetch; + } + + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeList(filesToFetch); + checkpoint.writeTo(out); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesResponse.java b/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesResponse.java new file mode 100644 index 0000000000000..b5125d637ac49 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/GetFilesResponse.java @@ -0,0 +1,30 @@ +/* + * 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.indices.replication.copy; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.transport.TransportResponse; + +import java.io.IOException; + +public class GetFilesResponse extends TransportResponse { + + public GetFilesResponse(StreamInput in) { + } + + public GetFilesResponse() { + + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationHandler.java b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationHandler.java new file mode 100644 index 0000000000000..2bbf59821a385 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationHandler.java @@ -0,0 +1,400 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.ArrayUtil; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.ActionListener; +import org.opensearch.action.StepListener; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.logging.Loggers; +import org.opensearch.common.lucene.store.InputStreamIndexInput; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.common.util.concurrent.FutureUtils; +import org.opensearch.common.util.concurrent.ListenableFuture; +import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.core.internal.io.IOUtils; +import org.opensearch.index.engine.RecoveryEngineException; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardRelocatedException; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.DelayRecoveryException; +import org.opensearch.indices.recovery.MultiChunkTransfer; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.RemoteTransportException; +import org.opensearch.transport.Transports; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.Consumer; + +public class PrimaryShardReplicationHandler { + + private final long replicationId; + private final Logger logger; + // Shard that is going to be recovered (the "source" on the current Node) + private final IndexShard shard; + private final String targetAllocationId; + private final DiscoveryNode targetNode; + private final int shardId; + // Request containing source and target node information + private final GetFilesRequest request; + private final int chunkSizeInBytes; + private final ReplicaClient targetClient; + private final int maxConcurrentFileChunks; + private final int maxConcurrentOperations; + private final ThreadPool threadPool; + private final CancellableThreads cancellableThreads = new CancellableThreads(); + private final List resources = new CopyOnWriteArrayList<>(); + private final ListenableFuture future = new ListenableFuture<>(); + + public PrimaryShardReplicationHandler( + long replicationId, + IndexShard shard, + DiscoveryNode targetNode, + String targetAllocationId, + ReplicaClient client, + ThreadPool threadPool, + GetFilesRequest request, + int fileChunkSizeInBytes, + int maxConcurrentFileChunks, + int maxConcurrentOperations) { + this.replicationId = replicationId; + this.shard = shard; + this.targetNode = targetNode; + this.targetAllocationId = targetAllocationId; + this.targetClient = client; + this.threadPool = threadPool; + this.request = request; + this.shardId = shard.shardId().id(); + this.logger = Loggers.getLogger(getClass(), shard.shardId(), "recover to " + request.getTargetNode().getName()); + this.chunkSizeInBytes = fileChunkSizeInBytes; + // if the target is on an old version, it won't be able to handle out-of-order file chunks. + this.maxConcurrentFileChunks = maxConcurrentFileChunks; + this.maxConcurrentOperations = maxConcurrentOperations; + } + + /** + * performs the recovery from the local engine to the target + */ + public void sendFiles(CopyState copyState, ActionListener listener) { + future.addListener(listener, OpenSearchExecutors.newDirectExecutorService()); + final Closeable releaseResources = () -> IOUtils.close(resources); + try { + + final Consumer onFailure = e -> { + assert Transports.assertNotTransportThread(PrimaryShardReplicationHandler.this + "[onFailure]"); + IOUtils.closeWhileHandlingException(releaseResources, () -> future.onFailure(e)); + }; + + runUnderPrimaryPermit(() -> { + final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); + ShardRouting targetShardRouting = routingTable.getByAllocationId(request.getTargetAllocationId()); + if (targetShardRouting == null) { + logger.debug( + "delaying replication of {} as it is not listed as assigned to target node {}", + shard.shardId(), + request.getTargetNode() + ); + throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); + } + }, + shardId + " validating recovery target [" + request.getTargetAllocationId() + "] registered ", + shard, + cancellableThreads, + logger + ); + + final StepListener sendFileStep = new StepListener<>(); + try { + // TODO: Segrep - Need validation here. + final Store.MetadataSnapshot metadataSnapshot = copyState.getMetadataSnapshot(); + final Map metadataMap = metadataSnapshot.asMap(); + final StoreFileMetadata[] storeFileMetadata = request.getFilesToFetch().stream() + .filter(file -> metadataMap.containsKey(file.name())) + .toArray(StoreFileMetadata[]::new); + sendFiles(shard.store(), storeFileMetadata, sendFileStep); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); + } + + // TODO: This only needs to happen on the initial setup. + sendFileStep.whenComplete(r -> { + runUnderPrimaryPermit(() -> shard.initiateTracking(targetAllocationId), + shardId + " initiating tracking of " + targetAllocationId, shard, cancellableThreads, logger); + + try { + future.onResponse(new GetFilesResponse()); + } finally { + IOUtils.close(resources); + } + }, onFailure); + } catch (Exception e) { + IOUtils.closeWhileHandlingException(releaseResources, () -> future.onFailure(e)); + } + } + + static void runUnderPrimaryPermit( + CancellableThreads.Interruptible runnable, + String reason, + IndexShard primary, + CancellableThreads cancellableThreads, + Logger logger + ) { + cancellableThreads.execute(() -> { + CompletableFuture permit = new CompletableFuture<>(); + final ActionListener onAcquired = new ActionListener() { + @Override + public void onResponse(Releasable releasable) { + if (permit.complete(releasable) == false) { + releasable.close(); + } + } + + @Override + public void onFailure(Exception e) { + permit.completeExceptionally(e); + } + }; + primary.acquirePrimaryOperationPermit(onAcquired, ThreadPool.Names.SAME, reason); + try (Releasable ignored = FutureUtils.get(permit)) { + // check that the IndexShard still has the primary authority. This needs to be checked under operation permit to prevent + // races, as IndexShard will switch its authority only when it holds all operation permits, see IndexShard.relocated() + if (primary.isRelocatedPrimary()) { + throw new IndexShardRelocatedException(primary.shardId()); + } + runnable.run(); + } finally { + // just in case we got an exception (likely interrupted) while waiting for the get + permit.whenComplete((r, e) -> { + if (r != null) { + r.close(); + } + if (e != null) { + logger.trace("suppressing exception on completion (it was already bubbled up or the operation was aborted)", e); + } + }); + } + }); + } + + /** + * Cancels the recovery and interrupts all eligible threads. + */ + public void cancel(String reason) { + cancellableThreads.cancel(reason); + } + + @Override + public String toString() { + return "SegmentReplicationSourceHandler{" + + "shardId=" + + shard.shardId() + + ", targetNode=" + + request.getTargetNode() + + '}'; + } + + private static class FileChunk implements MultiChunkTransfer.ChunkRequest, Releasable { + final StoreFileMetadata md; + final BytesReference content; + final long position; + final boolean lastChunk; + final Releasable onClose; + + FileChunk(StoreFileMetadata md, BytesReference content, long position, boolean lastChunk, Releasable onClose) { + this.md = md; + this.content = content; + this.position = position; + this.lastChunk = lastChunk; + this.onClose = onClose; + } + + @Override + public boolean lastChunk() { + return lastChunk; + } + + @Override + public void close() { + onClose.close(); + } + } + + void sendFiles(Store store, StoreFileMetadata[] files, ActionListener listener) { + + ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetadata::length)); // send smallest first + + final MultiChunkTransfer multiFileSender = new MultiChunkTransfer( + logger, + threadPool.getThreadContext(), + listener, + maxConcurrentFileChunks, + Arrays.asList(files) + ) { + + final Deque buffers = new ConcurrentLinkedDeque<>(); + InputStreamIndexInput currentInput = null; + long offset = 0; + + @Override + protected void onNewResource(StoreFileMetadata md) throws IOException { + offset = 0; + IOUtils.close(currentInput, () -> currentInput = null); + final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); + currentInput = new InputStreamIndexInput(indexInput, md.length()) { + @Override + public void close() throws IOException { + IOUtils.close(indexInput, super::close); // InputStreamIndexInput's close is a noop + } + }; + } + + private byte[] acquireBuffer() { + final byte[] buffer = buffers.pollFirst(); + if (buffer != null) { + return buffer; + } + return new byte[chunkSizeInBytes]; + } + + @Override + protected FileChunk nextChunkRequest(StoreFileMetadata md) throws IOException { + assert Transports.assertNotTransportThread("read file chunk"); + cancellableThreads.checkForCancel(); + final byte[] buffer = acquireBuffer(); + final int bytesRead = currentInput.read(buffer); + if (bytesRead == -1) { + throw new CorruptIndexException("file truncated; length=" + md.length() + " offset=" + offset, md.name()); + } + final boolean lastChunk = offset + bytesRead == md.length(); + final FileChunk chunk = new FileChunk( + md, + new BytesArray(buffer, 0, bytesRead), + offset, + lastChunk, + () -> buffers.addFirst(buffer) + ); + offset += bytesRead; + return chunk; + } + + @Override + protected void executeChunkRequest(FileChunk request, ActionListener listener) { + cancellableThreads.checkForCancel(); + targetClient.writeFileChunk( + replicationId, + request.md, + request.position, + request.content, + request.lastChunk, + ActionListener.runBefore(listener, request::close) + ); + } + + @Override + protected void handleError(StoreFileMetadata md, Exception e) throws Exception { + handleErrorOnSendFiles(store, e, new StoreFileMetadata[]{md}); + } + + @Override + public void close() throws IOException { + IOUtils.close(currentInput, () -> currentInput = null); + } + }; + resources.add(multiFileSender); + multiFileSender.start(); + } + + private void handleErrorOnSendFiles(Store store, Exception e, StoreFileMetadata[] mds) throws Exception { + final IOException corruptIndexException = ExceptionsHelper.unwrapCorruption(e); + assert Transports.assertNotTransportThread(PrimaryShardReplicationHandler.this + "[handle error on send/clean files]"); + if (corruptIndexException != null) { + Exception localException = null; + for (StoreFileMetadata md : mds) { + cancellableThreads.checkForCancel(); + logger.debug("checking integrity for file {} after remove corruption exception", md); + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md); + if (localException == null) { + localException = corruptIndexException; + } + failEngine(corruptIndexException); + } + } + if (localException != null) { + throw localException; + } else { // corruption has happened on the way to replica + RemoteTransportException remoteException = new RemoteTransportException( + "File corruption occurred on recovery but checksums are ok", + null + ); + remoteException.addSuppressed(e); + logger.warn( + () -> new ParameterizedMessage( + "{} Remote file corruption on node {}, recovering {}. local checksum OK", + shardId, + request.getTargetNode(), + mds + ), + corruptIndexException + ); + throw remoteException; + } + } + throw e; + } + + protected void failEngine(IOException cause) { + shard.failShard("recovery", cause); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java new file mode 100644 index 0000000000000..566a7921830e5 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java @@ -0,0 +1,272 @@ +/* + * 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.indices.replication.copy; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.RateLimiter; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.ActionListener; +import org.opensearch.action.ActionListenerResponseHandler; +import org.opensearch.action.StepListener; +import org.opensearch.action.support.ChannelActionListener; +import org.opensearch.action.support.RetryableAction; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.CheckedFunction; +import org.opensearch.common.breaker.CircuitBreakingException; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.IndexService; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; +import org.opensearch.tasks.Task; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.ConnectTransportException; +import org.opensearch.transport.RemoteTransportException; +import org.opensearch.transport.SendRequestTransportException; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportRequestHandler; +import org.opensearch.transport.TransportRequestOptions; +import org.opensearch.transport.TransportResponse; +import org.opensearch.transport.TransportService; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import static org.opensearch.indices.replication.copy.SegmentReplicationPrimaryService.Actions.GET_CHECKPOINT_INFO; +import static org.opensearch.indices.replication.copy.SegmentReplicationPrimaryService.Actions.GET_FILES; + +/** + * The source for replication where the source is the primary shard of a cluster. + */ +public class PrimaryShardReplicationSource { + + private static final Logger logger = LogManager.getLogger(PrimaryShardReplicationSource.class); + private final TransportService transportService; + private final ClusterService clusterService; + private final IndicesService indicesService; + private final Map> onGoingRetryableActions = ConcurrentCollections.newConcurrentMap(); + private final ThreadPool threadPool; + private final RecoverySettings recoverySettings; + private final SegmentReplicationService segmentReplicationService; + + // TODO: Segrep - Cancellation doesn't make sense here as it should be per replication event. + private volatile boolean isCancelled = false; + + public static class Actions { + public static final String FILE_CHUNK = "internal:index/shard/segrep/file_chunk"; + } + + public PrimaryShardReplicationSource(TransportService transportService, + ClusterService clusterService, + IndicesService indicesService, + RecoverySettings recoverySettings, + SegmentReplicationService segmentReplicationReplicaShardService) { + this.transportService = transportService; + this.clusterService = clusterService; + this.indicesService = indicesService; + this.recoverySettings = recoverySettings; + this.threadPool = transportService.getThreadPool(); + this.segmentReplicationService = segmentReplicationReplicaShardService; + + transportService.registerRequestHandler( + Actions.FILE_CHUNK, + ThreadPool.Names.GENERIC, + ReplicationFileChunkRequest::new, + new FileChunkTransportRequestHandler() + ); + } + + public void getCheckpointInfo(long replicationId, ReplicationCheckpoint checkpoint, StepListener listener) { + final ShardId shardId = checkpoint.getShardId(); + DiscoveryNode primaryNode = getPrimaryNode(shardId); + final DiscoveryNodes nodes = clusterService.state().nodes(); + final Writeable.Reader reader = TransportCheckpointInfoResponse::new; + final ActionListener responseListener = ActionListener.map(listener, r -> r); + StartReplicationRequest request = new StartReplicationRequest(replicationId, getAllocationId(shardId), nodes.getLocalNode(), checkpoint); + executeRetryableAction(primaryNode, GET_CHECKPOINT_INFO, request, responseListener, reader); + } + + public void getFiles(long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, StepListener listener) { + final ShardId shardId = checkpoint.getShardId(); + DiscoveryNode primaryNode = getPrimaryNode(shardId); + final DiscoveryNodes nodes = clusterService.state().nodes(); + final Writeable.Reader reader = GetFilesResponse::new; + final ActionListener responseListener = ActionListener.map(listener, r -> r); + + GetFilesRequest request = new GetFilesRequest(replicationId, getAllocationId(shardId), nodes.getLocalNode(), filesToFetch, checkpoint); + executeRetryableAction(primaryNode, GET_FILES, request, responseListener, reader); + } + + private String getAllocationId(ShardId shardId) { + final IndexService indexService = indicesService.indexService(shardId.getIndex()); + final IndexShard shard = indexService.getShard(shardId.getId()); + return shard.routingEntry().allocationId().getId(); + } + + private DiscoveryNode getPrimaryNode(ShardId shardId) { + ClusterState state = clusterService.state(); + ShardRouting primaryShard = state.routingTable().shardRoutingTable(shardId).primaryShard(); + return state.nodes().get(primaryShard.currentNodeId()); + } + + private void executeRetryableAction( + DiscoveryNode sourceNode, + String action, + TransportRequest request, + ActionListener actionListener, + Writeable.Reader reader + ) { + final Object key = new Object(); + final ActionListener removeListener = ActionListener.runBefore(actionListener, () -> onGoingRetryableActions.remove(key)); + final TimeValue initialDelay = TimeValue.timeValueMillis(200); + final TimeValue timeout = recoverySettings.internalActionRetryTimeout(); + final TransportRequestOptions options = TransportRequestOptions.builder() + .withTimeout(recoverySettings.internalActionTimeout()) + .build(); + final RetryableAction retryableAction = new RetryableAction(logger, threadPool, initialDelay, timeout, removeListener) { + + @Override + public void tryAction(ActionListener listener) { + transportService.sendRequest( + sourceNode, + action, + request, + options, + new ActionListenerResponseHandler<>(listener, reader, ThreadPool.Names.GENERIC) + ); + } + + @Override + public boolean shouldRetry(Exception e) { + return retryableException(e); + } + }; + onGoingRetryableActions.put(key, retryableAction); + retryableAction.run(); + if (isCancelled) { + retryableAction.cancel(new CancellableThreads.ExecutionCancelledException("recovery was cancelled")); + } + } + + private static boolean retryableException(Exception e) { + if (e instanceof ConnectTransportException) { + return true; + } else if (e instanceof SendRequestTransportException) { + final Throwable cause = ExceptionsHelper.unwrapCause(e); + return cause instanceof ConnectTransportException; + } else if (e instanceof RemoteTransportException) { + final Throwable cause = ExceptionsHelper.unwrapCause(e); + return cause instanceof CircuitBreakingException || cause instanceof OpenSearchRejectedExecutionException; + } + return false; + } + + public void cancel() { + isCancelled = true; + if (onGoingRetryableActions.isEmpty()) { + return; + } + final RuntimeException exception = new CancellableThreads.ExecutionCancelledException("recovery was cancelled"); + // Dispatch to generic as cancellation calls can come on the cluster state applier thread + threadPool.generic().execute(() -> { + for (RetryableAction action : onGoingRetryableActions.values()) { + action.cancel(exception); + } + onGoingRetryableActions.clear(); + }); + } + + class FileChunkTransportRequestHandler implements TransportRequestHandler { + + // How many bytes we've copied since we last called RateLimiter.pause + final AtomicLong bytesSinceLastPause = new AtomicLong(); + + @Override + public void messageReceived(final ReplicationFileChunkRequest request, TransportChannel channel, Task task) throws Exception { + try (ReplicationCollection.ReplicationRef replicationRef = segmentReplicationService.getOnGoingReplications().getReplicationSafe(request.getReplicationId(), request.shardId())) { + final ReplicationTarget replicationTarget = replicationRef.target(); + final ActionListener listener = createOrFinishListener(replicationRef, channel, Actions.FILE_CHUNK, request); + if (listener == null) { + return; + } + +// final ReplicationState.Index indexState = replicationTarget.state().getIndex(); +// if (request.sourceThrottleTimeInNanos() != ReplicationState.Index.UNKNOWN) { +// indexState.addSourceThrottling(request.sourceThrottleTimeInNanos()); +// } + + RateLimiter rateLimiter = recoverySettings.rateLimiter(); + if (rateLimiter != null) { + long bytes = bytesSinceLastPause.addAndGet(request.content().length()); + if (bytes > rateLimiter.getMinPauseCheckBytes()) { + // Time to pause + bytesSinceLastPause.addAndGet(-bytes); +// long throttleTimeInNanos = rateLimiter.pause(bytes); +// indexState.addTargetThrottling(throttleTimeInNanos); +// replicationTarget.getIndexShard().replicationStats().addThrottleTime(throttleTimeInNanos); + } + } + replicationTarget.writeFileChunk( + request.metadata(), + request.position(), + request.content(), + request.lastChunk(), + listener + ); + } + } + } + + private ActionListener createOrFinishListener( + final ReplicationCollection.ReplicationRef replicationRef, + final TransportChannel channel, + final String action, + final ReplicationFileChunkRequest request + ) { + return createOrFinishListener(replicationRef, channel, action, request, nullVal -> TransportResponse.Empty.INSTANCE); + } + + private ActionListener createOrFinishListener( + final ReplicationCollection.ReplicationRef replicationRef, + final TransportChannel channel, + final String action, + final ReplicationFileChunkRequest request, + final CheckedFunction responseFn + ) { + final ReplicationTarget replicationTarget = replicationRef.target(); + final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); + final ActionListener voidListener = ActionListener.map(channelListener, responseFn); + + final long requestSeqNo = request.requestSeqNo(); + final ActionListener listener; + if (requestSeqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { + listener = replicationTarget.markRequestReceivedAndCreateListener(requestSeqNo, voidListener); + } else { + listener = voidListener; + } + + return listener; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicaClient.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicaClient.java new file mode 100644 index 0000000000000..0acfc8fb2ad3c --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicaClient.java @@ -0,0 +1,212 @@ +/* + * 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.indices.replication.copy; + +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.RateLimiter; +import org.opensearch.ExceptionsHelper; +import org.opensearch.LegacyESVersion; +import org.opensearch.OpenSearchException; +import org.opensearch.action.ActionListener; +import org.opensearch.action.ActionListenerResponseHandler; +import org.opensearch.action.support.RetryableAction; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.breaker.CircuitBreakingException; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.*; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.*; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; + +/** + * This class serves as a client for primary shard's interaction with a target shard during replication. + */ +public class ReplicaClient { + + private static final Logger logger = LogManager.getLogger(org.opensearch.indices.recovery.RemoteRecoveryTargetHandler.class); + + private final TransportService transportService; + private final ThreadPool threadPool; + private final ShardId shardId; + private final DiscoveryNode targetNode; + private final RecoverySettings recoverySettings; + private final Map> onGoingRetryableActions = ConcurrentCollections.newConcurrentMap(); + + private final TransportRequestOptions fileChunkRequestOptions; + + private final AtomicLong bytesSinceLastPause = new AtomicLong(); + private final AtomicLong requestSeqNoGenerator = new AtomicLong(0); + + private final Consumer onSourceThrottle; + private final boolean retriesSupported; + private volatile boolean isCancelled = false; + + public ReplicaClient( + ShardId shardId, + TransportService transportService, + DiscoveryNode targetNode, + RecoverySettings recoverySettings, + Consumer onSourceThrottle + ) { + this.transportService = transportService; + this.threadPool = transportService.getThreadPool(); + this.shardId = shardId; + this.targetNode = targetNode; + this.recoverySettings = recoverySettings; + this.onSourceThrottle = onSourceThrottle; + this.fileChunkRequestOptions = TransportRequestOptions.builder() + .withType(TransportRequestOptions.Type.RECOVERY) + .withTimeout(recoverySettings.internalActionTimeout()) + .build(); + this.retriesSupported = targetNode.getVersion().onOrAfter(LegacyESVersion.V_7_9_0); + } + + public void writeFileChunk( + long replicationId, + StoreFileMetadata fileMetadata, + long position, + BytesReference content, + boolean lastChunk, + + ActionListener listener + ) { + // Pause using the rate limiter, if desired, to throttle the recovery + final long throttleTimeInNanos; + // always fetch the ratelimiter - it might be updated in real-time on the recovery settings + final RateLimiter rl = recoverySettings.rateLimiter(); + if (rl != null) { + long bytes = bytesSinceLastPause.addAndGet(content.length()); + if (bytes > rl.getMinPauseCheckBytes()) { + // Time to pause + bytesSinceLastPause.addAndGet(-bytes); + try { + throttleTimeInNanos = rl.pause(bytes); + onSourceThrottle.accept(throttleTimeInNanos); + } catch (IOException e) { + throw new OpenSearchException("failed to pause recovery", e); + } + } else { + throttleTimeInNanos = 0; + } + } else { + throttleTimeInNanos = 0; + } + + final String action = PrimaryShardReplicationSource.Actions.FILE_CHUNK; + final long requestSeqNo = requestSeqNoGenerator.getAndIncrement(); + /* we send estimateTotalOperations with every request since we collect stats on the target and that way we can + * see how many translog ops we accumulate while copying files across the network. A future optimization + * would be in to restart file copy again (new deltas) if we have too many translog ops are piling up. + */ + final ReplicationFileChunkRequest request = new ReplicationFileChunkRequest( + replicationId, + requestSeqNo, + shardId, + fileMetadata, + position, + content, + lastChunk, + throttleTimeInNanos); + final Writeable.Reader reader = in -> TransportResponse.Empty.INSTANCE; + executeRetryableAction(action, request, fileChunkRequestOptions, ActionListener.map(listener, r -> null), reader); + } + + private void executeRetryableAction( + String action, + TransportRequest request, + TransportRequestOptions options, + ActionListener actionListener, + Writeable.Reader reader + ) { + final Object key = new Object(); + final ActionListener removeListener = ActionListener.runBefore(actionListener, () -> onGoingRetryableActions.remove(key)); + final TimeValue initialDelay = TimeValue.timeValueMillis(200); + final TimeValue timeout = recoverySettings.internalActionRetryTimeout(); + final RetryableAction retryableAction = new RetryableAction(logger, threadPool, initialDelay, timeout, removeListener) { + + @Override + public void tryAction(ActionListener listener) { + transportService.sendRequest( + targetNode, + action, + request, + options, + new ActionListenerResponseHandler<>(listener, reader, ThreadPool.Names.GENERIC) + ); + } + + @Override + public boolean shouldRetry(Exception e) { + return retriesSupported && retryableException(e); + } + }; + onGoingRetryableActions.put(key, retryableAction); + retryableAction.run(); + if (isCancelled) { + retryableAction.cancel(new CancellableThreads.ExecutionCancelledException("recovery was cancelled")); + } + } + + private static boolean retryableException(Exception e) { + if (e instanceof ConnectTransportException) { + return true; + } else if (e instanceof SendRequestTransportException) { + final Throwable cause = ExceptionsHelper.unwrapCause(e); + return cause instanceof ConnectTransportException; + } else if (e instanceof RemoteTransportException) { + final Throwable cause = ExceptionsHelper.unwrapCause(e); + return cause instanceof CircuitBreakingException || cause instanceof OpenSearchRejectedExecutionException; + } + return false; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCheckpoint.java new file mode 100644 index 0000000000000..95d2c723f3b25 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCheckpoint.java @@ -0,0 +1,87 @@ +/* + * 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.indices.replication.copy; + +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.index.shard.ShardId; + +import java.io.IOException; +import java.util.Objects; + +public class ReplicationCheckpoint implements Writeable { + + private final ShardId shardId; + private final long primaryTerm; + private final long segmentsGen; + private final long seqNo; + + public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segments_gen, long seqNo) { + this.shardId = shardId; + this.primaryTerm = primaryTerm; + this.segmentsGen = segments_gen; + this.seqNo = seqNo; + } + + public ReplicationCheckpoint(StreamInput in) throws IOException { + shardId = new ShardId(in); + primaryTerm = in.readLong(); + segmentsGen = in.readLong(); + seqNo = in.readLong(); + } + + public long getPrimaryTerm() { + return primaryTerm; + } + + public long getSegmentsGen() { + return segmentsGen; + } + + public long getSeqNo() { + return seqNo; + } + + public ShardId getShardId() { + return shardId; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + out.writeLong(primaryTerm); + out.writeLong(segmentsGen); + out.writeLong(seqNo); + } + + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReplicationCheckpoint that = (ReplicationCheckpoint) o; + return primaryTerm == that.primaryTerm && segmentsGen == that.segmentsGen && seqNo == that.seqNo && Objects.equals(shardId, that.shardId); + } + + @Override + public int hashCode() { + return Objects.hash(shardId, primaryTerm, segmentsGen, seqNo); + } + + @Override + public String toString() { + return "ReplicationCheckpoint{" + + "shardId=" + shardId + + ", primaryTerm=" + primaryTerm + + ", segmentsGen=" + segmentsGen + + ", seqNo=" + seqNo + + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java new file mode 100644 index 0000000000000..1ac429ff8d6a4 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -0,0 +1,291 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.OpenSearchTimeoutException; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractRunnable; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.ShardId; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.threadpool.ThreadPool; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * This class holds a collection of all on going recoveries on the current node (i.e., the node is the target node + * of those recoveries). The class is used to guarantee concurrent semantics such that once a recoveries was done/cancelled/failed + * no other thread will be able to find it. Last, the {@link ReplicationRef} inner class verifies that replication temporary files + * and store will only be cleared once on going usage is finished. + */ +public class ReplicationCollection { + + /** + * This is the single source of truth for ongoing recoveries. If it's not here, it was canceled or done + */ + private final ConcurrentMap onGoingReplications = ConcurrentCollections.newConcurrentMap(); + + private final Logger logger; + private final ThreadPool threadPool; + + public ReplicationCollection(Logger logger, ThreadPool threadPool) { + this.logger = logger; + this.threadPool = threadPool; + } + + public long startReplication( + ReplicationCheckpoint checkpoint, + IndexShard indexShard, + PrimaryShardReplicationSource source, + SegmentReplicationService.ReplicationListener listener, + TimeValue activityTimeout + ) { + ReplicationTarget replicationTarget = new ReplicationTarget(checkpoint, indexShard, source, listener); + startReplicationInternal(replicationTarget, activityTimeout); + return replicationTarget.getReplicationId(); + } + + private void startReplicationInternal(ReplicationTarget replicationTarget, TimeValue activityTimeout) { + ReplicationTarget existingTarget = onGoingReplications.putIfAbsent(replicationTarget.getReplicationId(), replicationTarget); + assert existingTarget == null : "found two ReplicationStatus instances with the same id"; + logger.trace( + "{} started segment replication id [{}]", + replicationTarget.getIndexShard().shardId(), + replicationTarget.getReplicationId() + ); + threadPool.schedule( + new ReplicationMonitor(replicationTarget.getReplicationId(), replicationTarget.lastAccessTime(), activityTimeout), + activityTimeout, + ThreadPool.Names.GENERIC + ); + } + + public ReplicationTarget getReplicationTarget(long id) { + return onGoingReplications.get(id); + } + + /** + * gets the {@link ReplicationTarget } for a given id. The ReplicationStatus returned has it's ref count already incremented + * to make sure it's safe to use. However, you must call {@link ReplicationTarget#decRef()} when you are done with it, typically + * by using this method in a try-with-resources clause. + *

+ * Returns null if replication is not found + */ + public ReplicationRef getReplication(long id) { + ReplicationTarget status = onGoingReplications.get(id); + if (status != null && status.tryIncRef()) { + return new ReplicationRef(status); + } + return null; + } + + /** + * Similar to {@link #getReplication(long)} but throws an exception if no replication is found + */ + public ReplicationRef getReplicationSafe(long id, ShardId shardId) { + ReplicationRef replicationRef = getReplication(id); + if (replicationRef == null) { + throw new IndexShardClosedException(shardId); + } + assert replicationRef.target().getIndexShard().shardId().equals(shardId); + return replicationRef; + } + +// /** cancel the replication with the given id (if found) and remove it from the replication collection */ +// public boolean cancelReplication(long id, String reason) { +// ReplicationTarget removed = onGoingRecoveries.remove(id); +// boolean cancelled = false; +// if (removed != null) { +// logger.trace( +// "{} canceled replication from {}, id [{}] (reason [{}])", +// removed.getIndexShard().shardId(), +// removed.getSource().name(), +// removed.getReplicationId(), +// reason +// ); +// removed.cancel(reason); +// cancelled = true; +// } +// return cancelled; +// } + + /** + * fail the replication with the given id (if found) and remove it from the replication collection + * + * @param id id of the replication to fail + * @param e exception with reason for the failure + * @param sendShardFailure true a shard failed message should be sent to the master + */ + public void failReplication(long id, ReplicationFailedException e, boolean sendShardFailure) { + ReplicationTarget removed = onGoingReplications.remove(id); + if (removed != null) { + logger.trace( + "{} failing segment replication id [{}]. Send shard failure: [{}]", + removed.getIndexShard().shardId(), + removed.getReplicationId(), + sendShardFailure + ); + removed.fail(e, sendShardFailure); + } + } + + /** + * mark the replication with the given id as done (if found) + */ + public void markReplicationAsDone(long id) { + ReplicationTarget removed = onGoingReplications.remove(id); + if (removed != null) { + logger.trace("{} marking replication as done, id [{}]", removed.getIndexShard().shardId(), removed.getReplicationId()); + removed.markAsDone(); + } + } + + /** + * the number of ongoing recoveries + */ + public int size() { + return onGoingReplications.size(); + } + + /** + * cancel all ongoing recoveries for the given shard + * + * @param reason reason for cancellation + * @param shardId shardId for which to cancel recoveries + * @return true if a replication was cancelled + */ + public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { + boolean cancelled = false; + List matchedRecoveries = new ArrayList<>(); + synchronized (onGoingReplications) { + for (Iterator it = onGoingReplications.values().iterator(); it.hasNext(); ) { + ReplicationTarget status = it.next(); + if (status.getIndexShard().shardId().equals(shardId)) { + matchedRecoveries.add(status); + it.remove(); + } + } + } + for (ReplicationTarget removed : matchedRecoveries) { + logger.trace( + "{} canceled segment replication id [{}] (reason [{}])", + removed.getIndexShard().shardId(), + removed.getReplicationId(), + reason + ); + removed.cancel(reason); + cancelled = true; + } + return cancelled; + } + + /** + * a reference to {@link ReplicationTarget}, which implements {@link AutoCloseable}. closing the reference + * causes {@link ReplicationTarget#decRef()} to be called. This makes sure that the underlying resources + * will not be freed until {@link ReplicationRef#close()} is called. + */ + public static class ReplicationRef implements AutoCloseable { + + private final ReplicationTarget target; + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** + * Important: {@link ReplicationTarget#tryIncRef()} should + * be *successfully* called on status before + */ + public ReplicationRef(ReplicationTarget target) { + this.target = target; + this.target.setLastAccessTime(); + } + + @Override + public void close() { + if (closed.compareAndSet(false, true)) { + target.decRef(); + } + } + + public ReplicationTarget target() { + return target; + } + } + + private class ReplicationMonitor extends AbstractRunnable { + private final long replicationId; + private final TimeValue checkInterval; + + private volatile long lastSeenAccessTime; + + private ReplicationMonitor(long replicationId, long lastSeenAccessTime, TimeValue checkInterval) { + this.replicationId = replicationId; + this.checkInterval = checkInterval; + this.lastSeenAccessTime = lastSeenAccessTime; + } + + @Override + public void onFailure(Exception e) { + logger.error(() -> new ParameterizedMessage("unexpected error while monitoring replication [{}]", replicationId), e); + } + + @Override + protected void doRun() throws Exception { + ReplicationTarget replicationTarget = onGoingReplications.get(replicationId); + if (replicationTarget == null) { + logger.trace("[monitor] no replicationTarget found for [{}], shutting down", replicationId); + return; + } + long accessTime = replicationTarget.lastAccessTime(); + if (accessTime == lastSeenAccessTime) { + String message = "no activity after [" + checkInterval + "]"; + failReplication( + replicationId, + new ReplicationFailedException(replicationTarget.getIndexShard(), message, new OpenSearchTimeoutException(message)), + true // to be safe, we don't know what go stuck + ); + return; + } + lastSeenAccessTime = accessTime; + logger.trace("[monitor] rescheduling check for [{}]. last access time is [{}]", replicationId, lastSeenAccessTime); + threadPool.schedule(this, checkInterval, ThreadPool.Names.GENERIC); + } + } + +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFailedException.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFailedException.java new file mode 100644 index 0000000000000..1860ebc3e347c --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFailedException.java @@ -0,0 +1,70 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.opensearch.OpenSearchException; +import org.opensearch.common.Nullable; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; + +import java.io.IOException; + +public class ReplicationFailedException extends OpenSearchException { + + public ReplicationFailedException(IndexShard shard, Throwable cause) { + this(shard, null, cause); + } + + public ReplicationFailedException(IndexShard shard, @Nullable String extraInfo, Throwable cause) { + this(shard.shardId(), extraInfo, cause); + } + + public ReplicationFailedException( + ShardId shardId, + @Nullable String extraInfo, + Throwable cause + ) { + super( + shardId + + ": Replication failed " + + ("on ") + + (extraInfo == null ? "" : " (" + extraInfo + ")"), + cause + ); + } + + public ReplicationFailedException(StreamInput in) throws IOException { + super(in); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFileChunkRequest.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFileChunkRequest.java new file mode 100644 index 0000000000000..08ea8e47146f8 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationFileChunkRequest.java @@ -0,0 +1,156 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.lucene.util.Version; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.transport.TransportRequest; + +import java.io.IOException; + +public final class ReplicationFileChunkRequest extends TransportRequest { + private final boolean lastChunk; + private final ShardId shardId; + private final long position; + private final BytesReference content; + private final StoreFileMetadata metadata; + private final long sourceThrottleTimeInNanos; + + private final long replicationId; + private final long requestSeqNo; + + public ReplicationFileChunkRequest(StreamInput in) throws IOException { + super(in); + shardId = new ShardId(in); + replicationId = in.readLong(); + requestSeqNo = in.readLong(); + String name = in.readString(); + position = in.readVLong(); + long length = in.readVLong(); + String checksum = in.readString(); + content = in.readBytesReference(); + Version writtenBy = Lucene.parseVersionLenient(in.readString(), null); + assert writtenBy != null; + metadata = new StoreFileMetadata(name, length, checksum, writtenBy); + lastChunk = in.readBoolean(); + sourceThrottleTimeInNanos = in.readLong(); + } + + public ReplicationFileChunkRequest( + long replicationId, + final long requestSeqNo, + ShardId shardId, + StoreFileMetadata metadata, + long position, + BytesReference content, + boolean lastChunk, + long sourceThrottleTimeInNanos) { + this.replicationId = replicationId; + this.requestSeqNo = requestSeqNo; + this.shardId = shardId; + this.metadata = metadata; + this.position = position; + this.content = content; + this.lastChunk = lastChunk; + this.sourceThrottleTimeInNanos = sourceThrottleTimeInNanos; + } + + public ShardId shardId() { + return shardId; + } + + public String name() { + return metadata.name(); + } + + public long position() { + return position; + } + + public long length() { + return metadata.length(); + } + + public BytesReference content() { + return content; + } + + public long sourceThrottleTimeInNanos() { + return sourceThrottleTimeInNanos; + } + + public long getReplicationId() { + return replicationId; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + out.writeLong(replicationId); + out.writeLong(requestSeqNo); + out.writeString(metadata.name()); + out.writeVLong(position); + out.writeVLong(metadata.length()); + out.writeString(metadata.checksum()); + out.writeBytesReference(content); + out.writeString(metadata.writtenBy().toString()); + out.writeBoolean(lastChunk); + out.writeLong(sourceThrottleTimeInNanos); + } + + @Override + public String toString() { + return shardId + ": name='" + name() + '\'' + ", position=" + position + ", length=" + length(); + } + + public StoreFileMetadata metadata() { + return metadata; + } + + /** + * Returns true if this chunk is the last chunk in the stream. + */ + public boolean lastChunk() { + return lastChunk; + } + + public long requestSeqNo() { + return requestSeqNo; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java new file mode 100644 index 0000000000000..c2c999b8ae53b --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java @@ -0,0 +1,79 @@ +/* + * 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.indices.replication.copy; + +import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.recovery.Timer; + +public class ReplicationState { + + private Timer timer; + private RecoveryState.Index index; + private Stage stage; + + public ReplicationState(RecoveryState.Index index) { + this.index = index; + this.timer = new Timer(); + stage = Stage.INACTIVE; + timer.start(); + } + + public ReplicationState() { + stage = Stage.INACTIVE; + } + + public Timer getTimer() { + return timer; + } + + public RecoveryState.Index getIndex() { + return index; + } + + public enum Stage { + // TODO: Add more steps here. + INACTIVE((byte) 0), + + ACTIVE((byte) 1); + + private static final ReplicationState.Stage[] STAGES = new ReplicationState.Stage[ReplicationState.Stage.values().length]; + + static { + for (ReplicationState.Stage stage : ReplicationState.Stage.values()) { + assert stage.id() < STAGES.length && stage.id() >= 0; + STAGES[stage.id] = stage; + } + } + + private final byte id; + + Stage(byte id) { + this.id = id; + } + + public byte id() { + return id; + } + + public static ReplicationState.Stage fromId(byte id) { + if (id < 0 || id >= STAGES.length) { + throw new IllegalArgumentException("No mapping for id [" + id + "]"); + } + return STAGES[id]; + } + } + + public synchronized Stage getStage() { + return this.stage; + } + // synchronized is strictly speaking not needed (this is called by a single thread), but just to be safe + public synchronized void setStage(Stage stage) { + this.stage = stage; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java new file mode 100644 index 0000000000000..ee253ebaa05aa --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java @@ -0,0 +1,354 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.opensearch.OpenSearchException; +import org.opensearch.action.ActionListener; +import org.opensearch.action.StepListener; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.common.UUIDs; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.logging.Loggers; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.common.util.concurrent.AbstractRefCounted; +import org.opensearch.index.engine.EngineException; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardRecoveryException; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.index.translog.Translog; +import org.opensearch.indices.recovery.MultiFileWriter; +import org.opensearch.indices.recovery.RecoveryRequestTracker; +import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; + + +/** + * Orchestrates a replication event for a replica shard. + */ +public class ReplicationTarget extends AbstractRefCounted { + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + private final ReplicationCheckpoint checkpoint; + private static final AtomicLong idGenerator = new AtomicLong(); + private final AtomicBoolean finished = new AtomicBoolean(); + private final long replicationId; + private final IndexShard indexShard; + private final Logger logger; + private final PrimaryShardReplicationSource source; + private final SegmentReplicationService.ReplicationListener listener; + private final Store store; + private final MultiFileWriter multiFileWriter; + private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker(); + private final ReplicationState state; + private volatile long lastAccessTime = System.nanoTime(); + + private static final String REPLICATION_PREFIX = "replication."; + + /** + * Creates a new replication target object that represents a replication to the provided source. + * + * @param indexShard local shard where we want to recover to + * @param source source of the recovery where we recover from + * @param listener called when recovery is completed/failed + */ + public ReplicationTarget(ReplicationCheckpoint checkpoint, IndexShard indexShard, PrimaryShardReplicationSource source, SegmentReplicationService.ReplicationListener listener) { + super("replication_status"); + this.checkpoint = checkpoint; + this.indexShard = indexShard; + this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); + this.replicationId = idGenerator.incrementAndGet(); + this.source = source; + this.listener = listener; + this.store = indexShard.store(); + final String tempFilePrefix = REPLICATION_PREFIX + UUIDs.randomBase64UUID() + "."; + state = new ReplicationState(new RecoveryState.Index()); + this.multiFileWriter = new MultiFileWriter( + indexShard.store(), + state.getIndex(), + tempFilePrefix, + logger, + this::ensureRefCount + ); + ; + // make sure the store is not released until we are done. + store.incRef(); + } + + public void startReplication(ActionListener listener) { + final StepListener shardStartedListener = new StepListener<>(); + final StepListener checkpointInfoListener = new StepListener<>(); + final StepListener getFilesListener = new StepListener<>(); + final StepListener finalizeListener = new StepListener<>(); + + ensureShardStarted(shardStartedListener); + + // Get list of files to copy from this checkpoint. + shardStartedListener.whenComplete(r -> source.getCheckpointInfo(replicationId, checkpoint, checkpointInfoListener), listener::onFailure); + + checkpointInfoListener.whenComplete(checkpointInfo -> getFiles(checkpointInfo, getFilesListener), listener::onFailure); + getFilesListener.whenComplete(response -> finalizeReplication(checkpointInfoListener.result(), finalizeListener), listener::onFailure); + finalizeListener.whenComplete(r -> listener.onResponse(new ReplicationResponse()), listener::onFailure); + } + + private void ensureShardStarted(StepListener shardStartedListener) { + if (indexShard.recoveryState().getStage() == RecoveryState.Stage.INIT) { + setupReplicaShard(indexShard, shardStartedListener); + } else { + shardStartedListener.onResponse(true); + } + } + + private void setupReplicaShard(IndexShard indexShard, StepListener shardStartedListener) throws IndexShardRecoveryException { + indexShard.prepareForIndexRecovery(); + final Store store = indexShard.store(); + store.incRef(); + try { + store.createEmpty(indexShard.indexSettings().getIndexVersionCreated().luceneVersion); + final String translogUUID = Translog.createEmptyTranslog( + indexShard.shardPath().resolveTranslog(), SequenceNumbers.NO_OPS_PERFORMED, indexShard.shardId(), + indexShard.getPendingPrimaryTerm()); + store.associateIndexWithNewTranslog(translogUUID); + indexShard.persistRetentionLeases(); + indexShard.openEngineAndRecoverFromTranslog(); + shardStartedListener.onResponse(true); + } catch (EngineException | IOException e) { + throw new IndexShardRecoveryException(indexShard.shardId(), "failed to start replica shard", e); + } finally { + store.decRef(); + } + } + + public Store store() { + ensureRefCount(); + return store; + } + + public void finalizeReplication(TransportCheckpointInfoResponse checkpointInfo, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + // first, we go and move files that were created with the recovery id suffix to + // the actual names, its ok if we have a corrupted index here, since we have replicas + // to recover from in case of a full cluster shutdown just when this code executes... + multiFileWriter.renameAllTempFiles(); + final Store store = store(); + store.incRef(); + try { + store.cleanupAndVerify("recovery CleanFilesRequestHandler", checkpointInfo.getSnapshot()); + if (indexShard.getRetentionLeases().leases().isEmpty()) { + // if empty, may be a fresh IndexShard, so write an empty leases file to disk + indexShard.persistRetentionLeases(); + assert indexShard.loadRetentionLeases().leases().isEmpty(); + } else { + assert indexShard.assertRetentionLeasesPersisted(); + } + indexShard.updateCurrentInfos(checkpointInfo.getCheckpoint().getSegmentsGen(), checkpointInfo.getInfosBytes()); + } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { + // this is a fatal exception at this stage. + // this means we transferred files from the remote that have not be checksummed and they are + // broken. We have to clean up this shard entirely, remove all files and bubble it up to the + // source shard since this index might be broken there as well? The Source can handle this and checks + // its content on disk if possible. + try { + try { + store.removeCorruptionMarker(); + } finally { + Lucene.cleanLuceneIndex(store.directory()); // clean up and delete all files + } + } catch (Exception e) { + logger.debug("Failed to clean lucene index", e); + ex.addSuppressed(e); + } + ReplicationFailedException rfe = new ReplicationFailedException(indexShard.shardId(), "failed to clean after recovery", ex); + fail(rfe, true); + throw rfe; + } catch (Exception ex) { + ReplicationFailedException rfe = new ReplicationFailedException(indexShard.shardId(), "failed to clean after recovery", ex); + fail(rfe, true); + throw rfe; + } finally { + store.decRef(); + } + return null; + }); + } + + public long getReplicationId() { + return replicationId; + } + + public IndexShard getIndexShard() { + return indexShard; + } + + @Override + protected void closeInternal() { + store.decRef(); + } + + public ReplicationState state() { + return state; + } + + private void getFiles(TransportCheckpointInfoResponse checkpointInfo, StepListener getFilesListener) throws IOException { + final Store.MetadataSnapshot snapshot = checkpointInfo.getSnapshot(); + Store.MetadataSnapshot localMetadata = getMetadataSnapshot(); + final Store.RecoveryDiff diff = snapshot.recoveryDiff(localMetadata); + logger.debug("Recovery diff {}", diff); + final List filesToFetch = Stream.concat(diff.missing.stream(), diff.different.stream()) + .collect(Collectors.toList()); + for (StoreFileMetadata file : filesToFetch) { + state.getIndex().addFileDetail(file.name(), file.length(), false); + } + if (filesToFetch.isEmpty()) { + getFilesListener.onResponse(new GetFilesResponse()); + } + source.getFiles(replicationId, checkpointInfo.getCheckpoint(), filesToFetch, getFilesListener); + } + + private Store.MetadataSnapshot getMetadataSnapshot() throws IOException { + if (indexShard.recoveryState().getStage() == RecoveryState.Stage.INIT) { + return Store.MetadataSnapshot.EMPTY; + } + return store.getMetadata(indexShard.getLatestSegmentInfos()); + } + + public ActionListener markRequestReceivedAndCreateListener(long requestSeqNo, ActionListener listener) { + return requestTracker.markReceivedAndCreateListener(requestSeqNo, listener); + } + + public void writeFileChunk( + StoreFileMetadata fileMetadata, + long position, + BytesReference content, + boolean lastChunk, + ActionListener listener + ) { + try { + multiFileWriter.writeFileChunk(fileMetadata, position, content, lastChunk); + listener.onResponse(null); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * return the last time this RecoveryStatus was used (based on System.nanoTime() + */ + public long lastAccessTime() { + return lastAccessTime; + } + + /** + * sets the lasAccessTime flag to now + */ + public void setLastAccessTime() { + lastAccessTime = System.nanoTime(); + } + + private void ensureRefCount() { + if (refCount() <= 0) { + throw new OpenSearchException( + "RecoveryStatus is used but it's refcount is 0. Probably a mismatch between incRef/decRef " + "calls" + ); + } + } + + /** + * mark the current recovery as done + */ + public void markAsDone() { + if (finished.compareAndSet(false, true)) { + try { + // might need to do something on index shard here. + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + listener.onReplicationDone(state()); + } + } + + /** + * fail the recovery and call listener + * + * @param e exception that encapsulating the failure + * @param sendShardFailure indicates whether to notify the master of the shard failure + */ + public void fail(ReplicationFailedException e, boolean sendShardFailure) { + if (finished.compareAndSet(false, true)) { + try { + listener.onReplicationFailure(state(), e, sendShardFailure); + } finally { + try { +// cancellableThreads.cancel("failed recovery [" + ExceptionsHelper.stackTrace(e) + "]"); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + } + } + } + + /** + * cancel the recovery. calling this method will clean temporary files and release the store + * unless this object is in use (in which case it will be cleaned once all ongoing users call + * {@link #decRef()} + */ + public void cancel(String reason) { + if (finished.compareAndSet(false, true)) { + try { + logger.debug("recovery canceled (reason: [{}])", reason); +// cancellableThreads.cancel(reason); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationPrimaryService.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationPrimaryService.java new file mode 100644 index 0000000000000..979a136795151 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationPrimaryService.java @@ -0,0 +1,184 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.ChannelActionListener; +import org.opensearch.common.inject.Inject; +import org.opensearch.index.IndexService; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.ShardId; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; +import org.opensearch.tasks.Task; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportRequestHandler; +import org.opensearch.transport.TransportResponse; +import org.opensearch.transport.TransportService; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +/** + * Request handlers for the primary shard during segment copy. + */ +public class SegmentReplicationPrimaryService { + + private static final Logger logger = LogManager.getLogger(SegmentReplicationPrimaryService.class); + + public static class Actions { + public static final String GET_CHECKPOINT_INFO = "internal:index/shard/segrep/checkpoint_info"; + public static final String GET_FILES = "internal:index/shard/segrep/get_files"; + } + + private final TransportService transportService; + private final IndicesService indicesService; + private final RecoverySettings recoverySettings; + + final CopyStateCache commitCache = new CopyStateCache(); + + @Inject + public SegmentReplicationPrimaryService(TransportService transportService, IndicesService indicesService, RecoverySettings recoverySettings) { + this.transportService = transportService; + this.indicesService = indicesService; + this.recoverySettings = recoverySettings; + // When the target node wants to start a peer recovery it sends a START_RECOVERY request to the source + // node. Upon receiving START_RECOVERY, the source node will initiate the peer recovery. + transportService.registerRequestHandler( + Actions.GET_CHECKPOINT_INFO, + ThreadPool.Names.GENERIC, + StartReplicationRequest::new, + new StartReplicationRequestHandler() + ); + + transportService.registerRequestHandler( + Actions.GET_FILES, + ThreadPool.Names.GENERIC, + GetFilesRequest::new, + new GetFilesRequestHandler() + ); + } + + private static final class CopyStateCache { + private final Map checkpointCopyState = Collections.synchronizedMap(new HashMap<>()); + + public void addCopyState(CopyState copyState) { + checkpointCopyState.putIfAbsent(copyState.getCheckpoint(), copyState); + } + + public CopyState getCopyStateForCheckpoint(ReplicationCheckpoint checkpoint) { + return checkpointCopyState.get(checkpoint); + } + + public boolean hasCheckpoint(ReplicationCheckpoint checkpoint) { + return checkpointCopyState.containsKey(checkpoint); + } + + public void removeCopyState(ReplicationCheckpoint checkpoint) { + final Optional nrtCopyState = Optional.ofNullable(checkpointCopyState.get(checkpoint)); + nrtCopyState.ifPresent((state) -> { + if (state.decRef()) { + checkpointCopyState.remove(checkpoint); + } + }); + } + } + + private class StartReplicationRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(StartReplicationRequest request, TransportChannel channel, Task task) throws Exception { + final ReplicationCheckpoint checkpoint = request.getCheckpoint(); + final ShardId shardId = checkpoint.getShardId(); + final IndexService indexService = indicesService.indexService(shardId.getIndex()); + final IndexShard shard = indexService.getShard(shardId.id()); + // If we don't have the requested checkpoint, create a new one from the latest commit on the shard. + // TODO: Segrep - need checkpoint validation. + CopyState nrtCopyState = new CopyState(shard); + commitCache.addCopyState(nrtCopyState); + channel.sendResponse(new TransportCheckpointInfoResponse(nrtCopyState.getCheckpoint(), nrtCopyState.getMetadataSnapshot(), nrtCopyState.getInfosBytes())); + } + } + + class GetFilesRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(GetFilesRequest request, TransportChannel channel, Task task) throws Exception { + if (commitCache.hasCheckpoint(request.getCheckpoint())) { + sendFiles(request, new ChannelActionListener<>(channel, Actions.GET_FILES, request)); + } else { + channel.sendResponse(TransportResponse.Empty.INSTANCE); + } + } + } + + private void sendFiles(GetFilesRequest request, ActionListener listener) { + final ShardId shardId = request.getCheckpoint().getShardId(); + logger.trace("Requested checkpoint {}", request.getCheckpoint()); + + final CopyState copyState = commitCache.getCopyStateForCheckpoint(request.getCheckpoint()); + + final IndexService indexService = indicesService.indexService(shardId.getIndex()); + final IndexShard shard = indexService.getShard(shardId.id()); + final ReplicaClient replicationTargetHandler = new ReplicaClient( + shardId, + transportService, + request.getTargetNode(), + recoverySettings, + throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime) + ); + PrimaryShardReplicationHandler handler = new PrimaryShardReplicationHandler( + request.getReplicationId(), + shard, + request.getTargetNode(), + request.getTargetAllocationId(), + replicationTargetHandler, + shard.getThreadPool(), + request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + recoverySettings.getMaxConcurrentFileChunks(), + recoverySettings.getMaxConcurrentOperations()); + logger.info( + "[{}][{}] fetching files for {}", + shardId.getIndex().getName(), + shardId.id(), + request.getTargetNode() + ); + // TODO: The calling shard could die between requests without finishing. + handler.sendFiles(copyState, ActionListener.runAfter(listener, () -> commitCache.removeCopyState(request.getCheckpoint()))); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTransportRequest.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTransportRequest.java new file mode 100644 index 0000000000000..f2ee2f514f4ce --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTransportRequest.java @@ -0,0 +1,82 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.copy; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.transport.TransportRequest; + +import java.io.IOException; + +public abstract class SegmentReplicationTransportRequest extends TransportRequest { + + private final long replicationId; + private final String targetAllocationId; + private final DiscoveryNode targetNode; + + protected SegmentReplicationTransportRequest(StreamInput in) throws IOException { + super(in); + replicationId = in.readLong(); + targetAllocationId = in.readString(); + targetNode = new DiscoveryNode(in); + } + + protected SegmentReplicationTransportRequest(long replicationId, + String targetAllocationId, + DiscoveryNode discoveryNode) { + this.replicationId = replicationId; + this.targetAllocationId = targetAllocationId; + this.targetNode = discoveryNode; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeLong(replicationId); + out.writeString(targetAllocationId); + targetNode.writeTo(out); + } + + public long getReplicationId() { + return replicationId; + } + + public String getTargetAllocationId() { + return targetAllocationId; + } + + public DiscoveryNode getTargetNode() { + return targetNode; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/StartReplicationRequest.java b/server/src/main/java/org/opensearch/indices/replication/copy/StartReplicationRequest.java new file mode 100644 index 0000000000000..82416441bd9b4 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/copy/StartReplicationRequest.java @@ -0,0 +1,44 @@ +/* + * 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.indices.replication.copy; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; + +import java.io.IOException; + +public class StartReplicationRequest extends SegmentReplicationTransportRequest { + + private final ReplicationCheckpoint checkpoint; + + public StartReplicationRequest(StreamInput in) throws IOException { + super(in); + checkpoint = new ReplicationCheckpoint(in); + } + + public StartReplicationRequest(long replicationId, + String targetAllocationId, + DiscoveryNode targetNode, + ReplicationCheckpoint checkpoint) { + super(replicationId, targetAllocationId, targetNode); + this.checkpoint = checkpoint; + } + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + checkpoint.writeTo(out); + } + +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index ae2872789819f..53c322aca7bfe 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -37,6 +37,9 @@ import org.apache.lucene.util.Constants; import org.apache.lucene.util.SetOnce; import org.opensearch.index.IndexingPressureService; +import org.opensearch.indices.replication.SegmentReplicationService; +import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; +import org.opensearch.indices.replication.copy.SegmentReplicationPrimaryService; import org.opensearch.watcher.ResourceWatcherService; import org.opensearch.Assertions; import org.opensearch.Build; @@ -926,6 +929,11 @@ protected Node( b.bind(PeerRecoveryTargetService.class) .toInstance(new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService)); } + b.bind(SegmentReplicationPrimaryService.class) + .toInstance(new SegmentReplicationPrimaryService(transportService, indicesService, recoverySettings)); + final SegmentReplicationService segmentReplicationService = new SegmentReplicationService(threadPool, recoverySettings); + b.bind(SegmentReplicationService.class).toInstance(segmentReplicationService); + b.bind(PrimaryShardReplicationSource.class).toInstance(new PrimaryShardReplicationSource(transportService, clusterService, indicesService, recoverySettings, segmentReplicationService)); b.bind(HttpServerTransport.class).toInstance(httpServerTransport); pluginComponents.stream().forEach(p -> b.bind((Class) p.getClass()).toInstance(p)); b.bind(PersistentTasksService.class).toInstance(persistentTasksService);