From 40eb0bd75a14089fcf4496081b6d91fe7b004f33 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 5 Apr 2023 15:12:14 +0530 Subject: [PATCH 1/2] Allow replica to fetch segments from remote store instead of leader node Signed-off-by: Ankit Kala --- .../opensearch/index/shard/IndexShardIT.java | 3 +- .../org/opensearch/index/IndexService.java | 7 +- .../index/engine/NRTReplicationEngine.java | 2 +- .../shard/CheckpointRefreshListener.java | 3 +- .../opensearch/index/shard/IndexShard.java | 39 +++--- .../shard/RemoteStoreRefreshListener.java | 13 +- ...oreSegmentUploadNotificationPublisher.java | 37 ++++++ .../opensearch/index/shard/StoreRecovery.java | 2 +- .../store/RemoteSegmentStoreDirectory.java | 7 +- .../org/opensearch/index/store/Store.java | 3 +- .../org/opensearch/indices/IndicesModule.java | 6 + .../opensearch/indices/IndicesService.java | 7 +- .../cluster/IndicesClusterStateService.java | 18 ++- .../recovery/PeerRecoveryTargetService.java | 2 +- .../PrimaryShardReplicationSource.java | 5 +- .../RemoteStoreReplicationSource.java | 92 ++++++++++++++ .../replication/SegmentReplicationSource.java | 6 +- .../SegmentReplicationSourceFactory.java | 19 +-- .../replication/SegmentReplicationTarget.java | 119 ++++++++++-------- .../RemoteStoreRefreshListenerTests.java | 2 +- .../SegmentReplicationIndexShardTests.java | 8 +- ...dicesLifecycleListenerSingleNodeTests.java | 4 +- ...actIndicesClusterStateServiceTestCase.java | 25 ++-- ...ClusterStateServiceRandomUpdatesTests.java | 4 +- .../PrimaryShardReplicationSourceTests.java | 6 +- .../SegmentReplicationTargetServiceTests.java | 2 +- .../SegmentReplicationTargetTests.java | 14 +-- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../replication/TestReplicationSource.java | 3 +- .../index/shard/IndexShardTestCase.java | 5 +- 30 files changed, 329 insertions(+), 137 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index 020985cc9668c..ab4b772641ba9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -678,7 +678,8 @@ public static final IndexShard newIndexShard( cbs, (indexSettings, shardRouting) -> new InternalTranslogFactory(), SegmentReplicationCheckpointPublisher.EMPTY, - null + null, + RemoteStoreSegmentUploadNotificationPublisher.EMPTY ); } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 7d791ace44682..86d3ea746a158 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -81,6 +81,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardClosedException; import org.opensearch.index.shard.IndexingOperationListener; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardNotFoundException; @@ -438,7 +439,8 @@ public synchronized IndexShard createShard( final ShardRouting routing, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final SegmentReplicationCheckpointPublisher checkpointPublisher + final SegmentReplicationCheckpointPublisher checkpointPublisher, + final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); /* @@ -506,7 +508,8 @@ public synchronized IndexShard createShard( circuitBreakerService, translogFactorySupplier, this.indexSettings.isSegRepEnabled() ? checkpointPublisher : null, - remoteStore + remoteStore, + remoteSegmentNotificationPublisher ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java index da3f914d8bd7e..67bc95d0db0f9 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java @@ -485,7 +485,7 @@ public int fillSeqNoGaps(long primaryTerm) throws IOException { @Override public Engine recoverFromTranslog(TranslogRecoveryRunner translogRecoveryRunner, long recoverUpToSeqNo) throws IOException { - throw new UnsupportedOperationException("Read only replicas do not have an IndexWriter and cannot recover from a translog."); + return this; } @Override diff --git a/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java index 66d095878d123..8ee56b68f8ef8 100644 --- a/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/CheckpointRefreshListener.java @@ -40,7 +40,8 @@ public void beforeRefresh() throws IOException { @Override public void afterRefresh(boolean didRefresh) throws IOException { - if (didRefresh && shard.state() == IndexShardState.STARTED && shard.getReplicationTracker().isPrimaryMode()) { + if (didRefresh && shard.state() == IndexShardState.STARTED && shard.getReplicationTracker().isPrimaryMode() + && !shard.indexSettings.isRemoteStoreEnabled()) { publisher.publish(shard, 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 9d06ce7c6a391..1b1fdd72e9f46 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -327,6 +327,7 @@ Runnable getGlobalCheckpointSyncer() { private final Store remoteStore; private final BiFunction translogFactorySupplier; + private final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher; public IndexShard( final ShardRouting shardRouting, @@ -351,7 +352,8 @@ public IndexShard( final CircuitBreakerService circuitBreakerService, final BiFunction translogFactorySupplier, @Nullable final SegmentReplicationCheckpointPublisher checkpointPublisher, - @Nullable final Store remoteStore + @Nullable final Store remoteStore, + RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -403,6 +405,7 @@ public IndexShard( this.pendingPrimaryTerm = primaryTerm; this.globalCheckpointListeners = new GlobalCheckpointListeners(shardId, threadPool.scheduler(), logger); this.pendingReplicationActions = new PendingReplicationActions(shardId, threadPool); + this.remoteSegmentNotificationPublisher = remoteSegmentNotificationPublisher; this.replicationTracker = new ReplicationTracker( shardId, aId, @@ -2182,7 +2185,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false); + syncSegmentsFromRemoteSegmentStore(false, true, false); } // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). final Engine newEngine = engineFactory.newReadWriteEngine(config); @@ -3090,6 +3093,7 @@ public void updateGlobalCheckpointOnReplica(final long globalCheckpoint, final S * When remote translog is enabled for an index, replication operation is limited to primary term validation and does not * update local checkpoint at replica, so the local checkpoint at replica can be less than globalCheckpoint. */ + assert (state() != IndexShardState.POST_RECOVERY && state() != IndexShardState.STARTED) || indexSettings.isRemoteTranslogStoreEnabled() : "supposedly in-sync shard copy received a global checkpoint [" + globalCheckpoint @@ -3499,11 +3503,12 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro final List internalRefreshListener = new ArrayList<>(); internalRefreshListener.add(new RefreshMetricUpdater(refreshMetric)); if (isRemoteStoreEnabled()) { - internalRefreshListener.add(new RemoteStoreRefreshListener(this)); + internalRefreshListener.add(new RemoteStoreRefreshListener(this, remoteSegmentNotificationPublisher)); } - if (this.checkpointPublisher != null && indexSettings.isSegRepEnabled() && shardRouting.primary()) { + if (this.checkpointPublisher != null && indexSettings.isSegRepEnabled() && shardRouting.primary() && !indexSettings.isRemoteStoreEnabled()) { internalRefreshListener.add(new CheckpointRefreshListener(this, this.checkpointPublisher)); } + /** * With segment replication enabled for primary relocation, recover replica shard initially as read only and * change to a writeable engine during relocation handoff after a round of segment replication. @@ -4078,7 +4083,7 @@ EngineConfigFactory getEngineConfigFactory() { } // for tests - ReplicationTracker getReplicationTracker() { + public ReplicationTracker getReplicationTracker() { return replicationTracker; } @@ -4347,7 +4352,7 @@ public void close() throws IOException { }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false); + syncSegmentsFromRemoteSegmentStore(false, true, false); } newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker))); onNewEngine(newEngineReference.get()); @@ -4380,23 +4385,14 @@ public void close() throws IOException { onSettingsChanged(); } - /** - * Downloads segments from remote segment store. This method will download segments till - * last refresh checkpoint. - * @param overrideLocal flag to override local segment files with those in remote store - * @throws IOException if exception occurs while reading segments from remote store - */ - public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal) throws IOException { - syncSegmentsFromRemoteSegmentStore(overrideLocal, true); - } - /** * Downloads segments from remote segment store. * @param overrideLocal flag to override local segment files with those in remote store * @param refreshLevelSegmentSync last refresh checkpoint is used if true, commit checkpoint otherwise + * @param shouldCommit if the shard requires committing the changes after sync from remote. * @throws IOException if exception occurs while reading segments from remote store */ - public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync) throws IOException { + public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync, boolean shouldCommit) throws IOException { assert indexSettings.isRemoteStoreEnabled(); logger.info("Downloading segments from remote segment store"); assert remoteStore.directory() instanceof FilterDirectory : "Store.directory is not an instance of FilterDirectory"; @@ -4448,6 +4444,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re skippedSegments.add(file); } } + if (refreshLevelSegmentSync && segmentInfosSnapshotFilename != null) { try ( ChecksumIndexInput indexInput = new BufferedChecksumIndexInput( @@ -4460,7 +4457,13 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re Long.parseLong(segmentInfosSnapshotFilename.split("__")[1]) ); long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + if (shouldCommit) { + finalizeReplication(infosSnapshot); + store.cleanupAndPreserveLatestCommitPoint("finalize - clean with in memory infos", infosSnapshot); + } + else { + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + } } } } catch (IOException e) { diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index ee3b392472fa0..e653cba2cf4c8 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -25,6 +25,8 @@ import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; import java.util.Collection; @@ -57,14 +59,16 @@ public final class RemoteStoreRefreshListener implements ReferenceManager.Refres private final RemoteSegmentStoreDirectory remoteDirectory; private final Map localSegmentChecksumMap; private long primaryTerm; + private final RemoteStoreSegmentUploadNotificationPublisher notificationPublisher; private static final Logger logger = LogManager.getLogger(RemoteStoreRefreshListener.class); - public RemoteStoreRefreshListener(IndexShard indexShard) { + public RemoteStoreRefreshListener(IndexShard indexShard, RemoteStoreSegmentUploadNotificationPublisher notificationPublisher) { this.indexShard = indexShard; this.storeDirectory = indexShard.store().directory(); this.remoteDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()) .getDelegate()).getDelegate(); this.primaryTerm = indexShard.getOperationPrimaryTerm(); + this.notificationPublisher = notificationPublisher; localSegmentChecksumMap = new HashMap<>(); if (indexShard.shardRouting.primary()) { try { @@ -103,6 +107,9 @@ public void afterRefresh(boolean didRefresh) { deleteStaleCommits(); } + // Capture replication checkpoint before uploading the segments as upload can take some time and checkpoint can move. + ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); + String segmentInfoSnapshotFilename = null; try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { SegmentInfos segmentInfos = segmentInfosGatedCloseable.get(); @@ -148,6 +155,10 @@ public void afterRefresh(boolean didRefresh) { .lastRefreshedCheckpoint(); ((InternalEngine) indexShard.getEngine()).translogManager() .setMinSeqNoToKeep(lastRefreshedCheckpoint + 1); + + if (!RemoteStoreSegmentUploadNotificationPublisher.EMPTY.equals(notificationPublisher)) { + notificationPublisher.notifySegmentUpload(indexShard, checkpoint); + } } } } catch (EngineException e) { diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java new file mode 100644 index 0000000000000..81ad3abae8354 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java @@ -0,0 +1,37 @@ +/* + * 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.opensearch.common.inject.Inject; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; + + +/** + * Hook to publish notification after primary uploads segments to the remote store. + * + * @opensearch.internal + */ +public class RemoteStoreSegmentUploadNotificationPublisher { + private final SegmentReplicationCheckpointPublisher segRepPublisher; + + @Inject + public RemoteStoreSegmentUploadNotificationPublisher(SegmentReplicationCheckpointPublisher segRepPublisher) { + this.segRepPublisher = segRepPublisher; + } + + public void notifySegmentUpload(IndexShard indexShard, ReplicationCheckpoint checkpoint) { + // TODO: Add separate publisher for CCR. + // we don't call indexShard.getLatestReplicationCheckpoint() as it might have a newer refreshed checkpoint. + // Instead we send the one which has been uploaded to remote store. + if (segRepPublisher != null) segRepPublisher.publish(indexShard, checkpoint); + } + + public static final RemoteStoreSegmentUploadNotificationPublisher EMPTY = new RemoteStoreSegmentUploadNotificationPublisher(null); +} diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 31a863129cc8c..6c2336501e64f 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -457,7 +457,7 @@ private void recoverFromRemoteStore(IndexShard indexShard, Repository repository remoteStore.incRef(); try { // Download segments from remote segment store - indexShard.syncSegmentsFromRemoteSegmentStore(true); + indexShard.syncSegmentsFromRemoteSegmentStore(true, true, false); if (store.directory().listAll().length == 0) { store.createEmpty(indexShard.indexSettings().getIndexVersionCreated().luceneVersion); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index c385303813844..ffb83f9febe28 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -118,7 +118,7 @@ public void init() throws IOException { * @return Map of segment filename to uploaded filename with checksum * @throws IOException if there were any failures in reading the metadata file */ - private Map readLatestMetadataFile() throws IOException { + public Map readLatestMetadataFile() throws IOException { Map segmentMetadataMap = new HashMap<>(); Collection metadataFiles = remoteMetadataDirectory.listFilesByPrefix(MetadataFilenameUtils.METADATA_PREFIX); @@ -149,7 +149,6 @@ private Map readMetadataFile(String metadataFil public static class UploadedSegmentMetadata { // Visible for testing static final String SEPARATOR = "::"; - private final String originalFilename; private final String uploadedFilename; private final String checksum; @@ -179,6 +178,10 @@ public static UploadedSegmentMetadata fromString(String uploadedFilename) { String[] values = uploadedFilename.split(SEPARATOR); return new UploadedSegmentMetadata(values[0], values[1], values[2], Long.parseLong(values[3])); } + + public String getOriginalFilename() { + return originalFilename; + } } /** 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 f923532b3d9ad..0c9437682c07d 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -399,7 +399,8 @@ public static RecoveryDiff segmentReplicationDiff(Map missing.add(value); } else { final StoreFileMetadata fileMetadata = target.get(value.name()); - if (fileMetadata.isSame(value)) { + // match segments using checksum + if (fileMetadata.checksum().equals(value.checksum())) { identical.add(value); } else { different.add(value); diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index 5310e1b1e8397..9da05c10408ca 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -73,6 +73,7 @@ import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.shard.PrimaryReplicaSyncer; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; @@ -287,6 +288,11 @@ protected void configure() { } else { bind(SegmentReplicationCheckpointPublisher.class).toInstance(SegmentReplicationCheckpointPublisher.EMPTY); } + if (FeatureFlags.isEnabled(FeatureFlags.REMOTE_STORE)) { + bind(RemoteStoreSegmentUploadNotificationPublisher.class).asEagerSingleton(); + } else { + bind(RemoteStoreSegmentUploadNotificationPublisher.class).toInstance(RemoteStoreSegmentUploadNotificationPublisher.EMPTY); + } } /** diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index b0d488a5b2cf7..7a249c35e0e74 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -135,6 +135,7 @@ import org.opensearch.index.shard.IndexingStats; import org.opensearch.index.shard.ShardId; import org.opensearch.index.store.remote.filecache.FileCacheCleaner; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.TranslogFactory; @@ -161,7 +162,6 @@ import org.opensearch.search.query.QueryPhase; import org.opensearch.search.query.QuerySearchResult; import org.opensearch.threadpool.ThreadPool; - import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -1023,14 +1023,15 @@ public IndexShard createShard( final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final DiscoveryNode targetNode, - final DiscoveryNode sourceNode + final DiscoveryNode sourceNode, + final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); ensureChangesAllowed(); IndexService indexService = indexService(shardRouting.index()); assert indexService != null; RecoveryState recoveryState = indexService.createRecoveryState(shardRouting, targetNode, sourceNode); - IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer, retentionLeaseSyncer, checkpointPublisher); + IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer, retentionLeaseSyncer, checkpointPublisher, remoteSegmentNotificationPublisher); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, mapping -> { assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS 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 439191f7635d9..299a4ff03df87 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -74,6 +74,7 @@ import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.shard.PrimaryReplicaSyncer; import org.opensearch.index.shard.PrimaryReplicaSyncer.ResyncTask; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.indices.IndicesService; @@ -144,6 +145,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final RetentionLeaseSyncer retentionLeaseSyncer; private final SegmentReplicationTargetService segmentReplicationTargetService; + private final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher; private final SegmentReplicationCheckpointPublisher checkpointPublisher; @@ -165,7 +167,8 @@ public IndicesClusterStateService( final PrimaryReplicaSyncer primaryReplicaSyncer, final GlobalCheckpointSyncAction globalCheckpointSyncAction, final RetentionLeaseSyncer retentionLeaseSyncer, - final SegmentReplicationCheckpointPublisher checkpointPublisher + final SegmentReplicationCheckpointPublisher checkpointPublisher, + final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) { this( settings, @@ -184,7 +187,8 @@ public IndicesClusterStateService( snapshotShardsService, primaryReplicaSyncer, globalCheckpointSyncAction::updateGlobalCheckpointForShard, - retentionLeaseSyncer + retentionLeaseSyncer, + remoteSegmentNotificationPublisher ); } @@ -206,7 +210,8 @@ public IndicesClusterStateService( final SnapshotShardsService snapshotShardsService, final PrimaryReplicaSyncer primaryReplicaSyncer, final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer + final RetentionLeaseSyncer retentionLeaseSyncer, + final RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) { this.settings = settings; this.checkpointPublisher = checkpointPublisher; @@ -232,6 +237,7 @@ public IndicesClusterStateService( this.globalCheckpointSyncer = globalCheckpointSyncer; this.retentionLeaseSyncer = Objects.requireNonNull(retentionLeaseSyncer); this.sendRefreshMapping = settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); + this.remoteSegmentNotificationPublisher = remoteSegmentNotificationPublisher; } @Override @@ -661,7 +667,8 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR globalCheckpointSyncer, retentionLeaseSyncer, nodes.getLocalNode(), - sourceNode + sourceNode, + remoteSegmentNotificationPublisher ); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); @@ -1019,7 +1026,8 @@ T createShard( Consumer globalCheckpointSyncer, RetentionLeaseSyncer retentionLeaseSyncer, DiscoveryNode targetNode, - @Nullable DiscoveryNode sourceNode + @Nullable DiscoveryNode sourceNode, + @Nullable RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher ) throws IOException; /** 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 22614ee7ca063..6693f1f66e8c6 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -246,7 +246,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi indexShard.prepareForIndexRecovery(); final boolean hasRemoteSegmentStore = indexShard.indexSettings().isRemoteStoreEnabled(); if (hasRemoteSegmentStore) { - indexShard.syncSegmentsFromRemoteSegmentStore(false, false); + indexShard.syncSegmentsFromRemoteSegmentStore(false, false, false); } final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false && indexShard.isRemoteTranslogEnabled(); final boolean hasNoTranslog = indexShard.indexSettings().isRemoteSnapshot(); diff --git a/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java index b211d81c1c76a..5e793d0d0f214 100644 --- a/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java @@ -13,6 +13,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.recovery.RecoverySettings; @@ -79,7 +80,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { final Writeable.Reader reader = GetSegmentFilesResponse::new; @@ -99,7 +100,7 @@ public void getSegmentFiles( @Override public String getDescription() { - return sourceNode.getName(); + return "remote store"; } @Override diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java new file mode 100644 index 0000000000000..8d3ffa9461c08 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -0,0 +1,92 @@ +/* + * 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.lucene.index.SegmentCommitInfo; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.Version; +import org.opensearch.action.ActionListener; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.RetryableTransportClient; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.transport.TransportResponse; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.opensearch.indices.replication.SegmentReplicationSourceService.Actions.GET_CHECKPOINT_INFO; + +/** + * Implementation of a {@link SegmentReplicationSource} where the source is remote store. + * + * @opensearch.internal + */ +public class RemoteStoreReplicationSource implements SegmentReplicationSource { + + private static final Logger logger = LogManager.getLogger(PrimaryShardReplicationSource.class); + + private final IndexShard indexShard; + + public RemoteStoreReplicationSource(IndexShard indexShard) { + this.indexShard = indexShard; + } + + @Override + public void getCheckpointMetadata(long replicationId, ReplicationCheckpoint checkpoint, ActionListener listener) { + FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); + FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); + RemoteSegmentStoreDirectory remoteDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory.getDelegate(); + + Map metadataMap = null; + // TODO: Need to figure out a way to pass this information for segment metadata via remote store. + final Version version = indexShard.getSegmentInfosSnapshot().get().getCommitLuceneVersion(); + try { + metadataMap = remoteDirectory.readLatestMetadataFile().entrySet().stream().collect(Collectors.toMap( + e -> e.getKey(), + e -> new StoreFileMetadata(e.getValue().getOriginalFilename(), e.getValue().getLength(), + Store.digestToString(Long.valueOf(e.getValue().getChecksum())), version, null))); + } catch (IOException e) { + logger.error("Error fetching checkpoint metadata from remote store {}", e); + e.printStackTrace(); + } + // TODO: GET current checkpoint from remote store. + listener.onResponse(new CheckpointInfoResponse(checkpoint, metadataMap, null)); + } + + @Override + public void getSegmentFiles(long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, IndexShard indexShard, ActionListener listener) { + try { + indexShard.syncSegmentsFromRemoteSegmentStore(false, true, true); + } catch (IOException e) { + logger.error("Failed to sync segments {}", e); + listener.onFailure(e); + return; + } + listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + } + + @Override + public String getDescription() { + return null; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java index 2fa74819fe4de..fbdba81f31837 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java @@ -10,10 +10,12 @@ import org.opensearch.action.ActionListener; import org.opensearch.common.util.CancellableThreads.ExecutionCancelledException; +import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import java.io.IOException; import java.util.List; /** @@ -38,14 +40,14 @@ public interface SegmentReplicationSource { * @param replicationId {@link long} - ID of the replication event. * @param checkpoint {@link ReplicationCheckpoint} Checkpoint to fetch metadata for. * @param filesToFetch {@link List} List of files to fetch. - * @param store {@link Store} Reference to the local store. + * @param indexShard {@link IndexShard} Reference to the IndexShard. * @param listener {@link ActionListener} Listener that completes with the list of files copied. */ void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceFactory.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceFactory.java index 1867fc59c5a56..af447d3df3a48 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceFactory.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceFactory.java @@ -38,13 +38,18 @@ public SegmentReplicationSourceFactory( } public SegmentReplicationSource get(IndexShard shard) { - return new PrimaryShardReplicationSource( - shard.recoveryState().getTargetNode(), - shard.routingEntry().allocationId().getId(), - transportService, - recoverySettings, - getPrimaryNode(shard.shardId()) - ); + if(shard.indexSettings().isRemoteStoreEnabled()) { + return new RemoteStoreReplicationSource(shard); + } + else { + return new PrimaryShardReplicationSource( + shard.recoveryState().getTargetNode(), + shard.routingEntry().allocationId().getId(), + transportService, + recoverySettings, + getPrimaryNode(shard.shardId()) + ); + } } private DiscoveryNode getPrimaryNode(ShardId shardId) { diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 995ec58d8768f..8a1229bd9831a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -38,6 +38,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Map; /** * Represents the target of a replication event. @@ -209,73 +210,81 @@ private void getFiles(CheckpointInfoResponse checkpointInfo, StepListener listener) { - ActionListener.completeWith(listener, () -> { - cancellableThreads.checkForCancel(); - state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); - Store store = null; - try { - multiFileWriter.renameAllTempFiles(); - store = store(); - store.incRef(); - // Deserialize the new SegmentInfos object sent from the primary. - final ReplicationCheckpoint responseCheckpoint = checkpointInfoResponse.getCheckpoint(); - SegmentInfos infos = SegmentInfos.readCommit( - store.directory(), - toIndexInput(checkpointInfoResponse.getInfosBytes()), - responseCheckpoint.getSegmentsGen() - ); + if (source instanceof RemoteStoreReplicationSource) { + ActionListener.completeWith(listener, () -> { + state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + return null; + }); + } else { + ActionListener.completeWith(listener, () -> { cancellableThreads.checkForCancel(); - indexShard.finalizeReplication(infos); - store.cleanupAndPreserveLatestCommitPoint("finalize - clean with in memory infos", infos); - } 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. + state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + Store store = null; try { + multiFileWriter.renameAllTempFiles(); + store = store(); + store.incRef(); + // Deserialize the new SegmentInfos object sent from the primary. + final ReplicationCheckpoint responseCheckpoint = checkpointInfoResponse.getCheckpoint(); + SegmentInfos infos = SegmentInfos.readCommit( + store.directory(), + toIndexInput(checkpointInfoResponse.getInfosBytes()), + responseCheckpoint.getSegmentsGen() + ); + cancellableThreads.checkForCancel(); + indexShard.finalizeReplication(infos); + store.cleanupAndPreserveLatestCommitPoint("finalize - clean with in memory infos", infos); + } 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 { - store.removeCorruptionMarker(); - } finally { - Lucene.cleanLuceneIndex(store.directory()); // clean up and delete all files + 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); } - } catch (Exception e) { - logger.debug("Failed to clean lucene index", e); - ex.addSuppressed(e); - } - ReplicationFailedException rfe = new ReplicationFailedException( - indexShard.shardId(), - "failed to clean after replication", - ex - ); - fail(rfe, true); - throw rfe; - } catch (OpenSearchException ex) { + ReplicationFailedException rfe = new ReplicationFailedException( + indexShard.shardId(), + "failed to clean after replication", + ex + ); + fail(rfe, true); + throw rfe; + } catch (OpenSearchException ex) { /* Ignore closed replication target as it can happen due to index shard closed event in a separate thread. In such scenario, ignore the exception */ - assert cancellableThreads.isCancelled() : "Replication target closed but segment replication not cancelled"; - logger.info("Replication target closed", ex); - } catch (Exception ex) { - ReplicationFailedException rfe = new ReplicationFailedException( - indexShard.shardId(), - "failed to clean after replication", - ex - ); - fail(rfe, true); - throw rfe; - } finally { - if (store != null) { - store.decRef(); + assert cancellableThreads.isCancelled() : "Replication target closed but segment replication not cancelled"; + logger.info("Replication target closed", ex); + } catch (Exception ex) { + ReplicationFailedException rfe = new ReplicationFailedException( + indexShard.shardId(), + "failed to clean after replication", + ex + ); + fail(rfe, true); + throw rfe; + } finally { + if (store != null) { + store.decRef(); + } } - } - return null; - }); + return null; + }); + } } /** diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index c9b8c023e26aa..818a8f4d61856 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -47,7 +47,7 @@ public void setup(boolean primary, int numberOfDocs) throws IOException { indexDocs(1, numberOfDocs); indexShard.refresh("test"); - remoteStoreRefreshListener = new RemoteStoreRefreshListener(indexShard); + remoteStoreRefreshListener = new RemoteStoreRefreshListener(indexShard, RemoteStoreSegmentUploadNotificationPublisher.EMPTY); } private void indexDocs(int startDocId, int numberOfDocs) throws IOException { diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 014a37249612b..9b899b69abda4 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -833,7 +833,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); @@ -903,7 +903,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { Assert.fail("Should not be reached"); @@ -943,7 +943,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { // randomly resolve the listener, indicating the source has resolved. @@ -985,7 +985,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) {} }; diff --git a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 0989bf869f18e..f25ee8c0beb04 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -46,6 +46,7 @@ import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.opensearch.indices.recovery.RecoveryState; @@ -153,7 +154,8 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem newRouting, s -> {}, RetentionLeaseSyncer.EMPTY, - SegmentReplicationCheckpointPublisher.EMPTY + SegmentReplicationCheckpointPublisher.EMPTY, + RemoteStoreSegmentUploadNotificationPublisher.EMPTY ); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); diff --git a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 0619e3e3f62a2..132bf8c2dffdf 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -51,6 +51,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.shard.PrimaryReplicaSyncer.ResyncTask; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; import org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndex; @@ -252,18 +253,18 @@ public MockIndexService indexService(Index index) { } @Override - public MockIndexShard createShard( - final ShardRouting shardRouting, - final SegmentReplicationCheckpointPublisher checkpointPublisher, - final PeerRecoveryTargetService recoveryTargetService, - final RecoveryListener recoveryListener, - final RepositoriesService repositoriesService, - final Consumer onShardFailure, - final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer, - final DiscoveryNode targetNode, - final DiscoveryNode sourceNode - ) throws IOException { + public MockIndexShard createShard(ShardRouting shardRouting, + SegmentReplicationCheckpointPublisher checkpointPublisher, + PeerRecoveryTargetService recoveryTargetService, + RecoveryListener recoveryListener, + RepositoriesService repositoriesService, + Consumer onShardFailure, + Consumer globalCheckpointSyncer, + RetentionLeaseSyncer retentionLeaseSyncer, + DiscoveryNode targetNode, + DiscoveryNode sourceNode, + RemoteStoreSegmentUploadNotificationPublisher remoteSegmentNotificationPublisher) + throws IOException { failRandomly(); RecoveryState recoveryState = new RecoveryState(shardRouting, targetNode, sourceNode); MockIndexService indexService = indexService(recoveryState.getShardId().getIndex()); diff --git a/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 22481b5a7b99f..e43ad291c2d76 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -64,6 +64,7 @@ import org.opensearch.index.Index; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.PrimaryReplicaSyncer; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.replication.SegmentReplicationSourceService; @@ -583,7 +584,8 @@ private IndicesClusterStateService createIndicesClusterStateService( null, primaryReplicaSyncer, s -> {}, - RetentionLeaseSyncer.EMPTY + RetentionLeaseSyncer.EMPTY, + RemoteStoreSegmentUploadNotificationPublisher.EMPTY ); } diff --git a/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java index d925956bd95ef..3b06a4ffa0f69 100644 --- a/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java @@ -110,7 +110,7 @@ public void testGetSegmentFiles() { REPLICATION_ID, checkpoint, Arrays.asList(testMetadata), - mock(Store.class), + mock(IndexShard.class), mock(ActionListener.class) ); CapturingTransport.CapturedRequest[] requestList = transport.getCapturedRequestsAndClear(); @@ -132,7 +132,7 @@ public void testTransportTimeoutForGetSegmentFilesAction() { REPLICATION_ID, checkpoint, Arrays.asList(testMetadata), - mock(Store.class), + mock(IndexShard.class), mock(ActionListener.class) ); CapturingTransport.CapturedRequest[] requestList = transport.getCapturedRequestsAndClear(); @@ -151,7 +151,7 @@ public void testGetSegmentFiles_CancelWhileRequestOpen() throws InterruptedExcep REPLICATION_ID, checkpoint, Arrays.asList(testMetadata), - mock(Store.class), + mock(IndexShard.class), new ActionListener<>() { @Override public void onResponse(GetSegmentFilesResponse getSegmentFilesResponse) { diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java index bae0afb5bcc3b..671e067aa32b1 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -130,7 +130,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { Assert.fail("Should not be called"); diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java index 599e73b548ddb..ffb29cbb5d4c3 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java @@ -127,7 +127,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { assertEquals(1, filesToFetch.size()); @@ -178,7 +178,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); @@ -221,7 +221,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onFailure(exception); @@ -264,7 +264,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); @@ -309,7 +309,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); @@ -353,7 +353,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); @@ -404,7 +404,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 6c4b636e3c002..ab9a957b2d722 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -175,6 +175,7 @@ import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.PrimaryReplicaSyncer; +import org.opensearch.index.shard.RemoteStoreSegmentUploadNotificationPublisher; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.remote.filecache.FileCacheCleaner; import org.opensearch.indices.IndicesModule; @@ -1940,7 +1941,7 @@ public void onFailure(final Exception e) { actionFilters ), RetentionLeaseSyncer.EMPTY, - SegmentReplicationCheckpointPublisher.EMPTY + RemoteStoreSegmentUploadNotificationPublisher.EMPTY ); Map actions = new HashMap<>(); final SystemIndices systemIndices = new SystemIndices(emptyMap()); diff --git a/test/framework/src/main/java/org/opensearch/index/replication/TestReplicationSource.java b/test/framework/src/main/java/org/opensearch/index/replication/TestReplicationSource.java index a3adedcbdef86..f6c0331d2056a 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/TestReplicationSource.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/TestReplicationSource.java @@ -9,6 +9,7 @@ package org.opensearch.index.replication; import org.opensearch.action.ActionListener; +import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.CheckpointInfoResponse; @@ -35,7 +36,7 @@ public abstract void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index ab0cf38f77c7d..c8c5ff1f58f5e 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -600,7 +600,8 @@ protected IndexShard newShard( breakerService, translogFactorySupplier, checkpointPublisher, - remoteStore + remoteStore, + new RemoteStoreSegmentUploadNotificationPublisher(null) ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); success = true; @@ -1315,7 +1316,7 @@ public void getSegmentFiles( long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, - Store store, + IndexShard indexShard, ActionListener listener ) { try ( From 42d61360b671991323e9f85af425d8f6309ef08a Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Mon, 17 Apr 2023 13:40:46 +0530 Subject: [PATCH 2/2] Full cluster replication POC --- .../org/opensearch/action/ActionModule.java | 20 ++ .../cluster/metadata/IndexMetadata.java | 10 + .../common/settings/IndexScopedSettings.java | 1 + .../opensearch/common/util/FeatureFlags.java | 7 + .../org/opensearch/index/IndexSettings.java | 16 ++ .../opensearch/index/engine/EngineConfig.java | 5 + .../index/engine/InternalEngine.java | 1 + .../engine/NRTReplicationEngineFactory.java | 3 +- .../opensearch/index/shard/IndexShard.java | 19 +- ...oreSegmentUploadNotificationPublisher.java | 12 +- .../opensearch/index/shard/StoreRecovery.java | 2 +- .../RemoteSegmentStoreDirectoryFactory.java | 7 +- .../RemoteSnapshotDirectoryFactory.java | 16 +- .../index/translog/TranslogHeader.java | 7 +- .../recovery/PeerRecoveryTargetService.java | 2 +- .../RemoteStoreReplicationSource.java | 24 ++- .../main/java/org/opensearch/node/Node.java | 53 ++++- .../followers/StartCCRFollowerTaskAction.java | 23 +++ .../StartCCRFollowerTaskRequest.java | 46 +++++ .../StartCCRFollowerTaskResponse.java | 48 +++++ .../TransportStartCCRFollowerTaskAction.java | 71 +++++++ .../index/StartCCRIndexTaskAction.java | 23 +++ .../index/StartCCRIndexTaskRequest.java | 53 +++++ .../index/StartCCRIndexTaskResponse.java | 48 +++++ .../TransportStartCCRIndexTaskAction.java | 66 ++++++ .../NotifyCCRFollowersAction.java | 79 ++++++++ .../NotifyCCRFollowersRequest.java | 25 +++ .../NotifyCCRFollowersResponse.java | 25 +++ .../actions/start/RestStartCCRAction.java | 49 +++++ .../actions/start/StartCCRAction.java | 22 ++ .../actions/start/StartCCRRequest.java | 59 ++++++ .../actions/start/StartCCRResponse.java | 48 +++++ .../start/TransportStartCCRAction.java | 61 ++++++ .../syncsegments/SyncFromLeaderAction.java | 23 +++ .../syncsegments/SyncFromLeaderRequest.java | 73 +++++++ .../syncsegments/SyncFromLeaderResponse.java | 39 ++++ .../SyncShardFromLeaderRequest.java | 39 ++++ .../SyncShardFromLeaderResponse.java | 25 +++ .../TransportSyncFromLeaderAction.java | 67 +++++++ .../services/XReplicationFollowerService.java | 98 +++++++++ .../services/XReplicationLeaderService.java | 188 ++++++++++++++++++ .../follower/FollowerReplicationExecutor.java | 60 ++++++ .../follower/FollowerReplicationParams.java | 57 ++++++ .../follower/FollowerReplicationState.java | 45 +++++ .../follower/FollowerReplicationTask.java | 31 +++ .../task/index/IndexReplicationExecutor.java | 62 ++++++ .../task/index/IndexReplicationParams.java | 63 ++++++ .../task/index/IndexReplicationState.java | 46 +++++ .../task/index/IndexReplicationTask.java | 126 ++++++++++++ .../SupervisorReplicationExecutor.java | 62 ++++++ .../SupervisorReplicationParams.java | 56 ++++++ .../SupervisorReplicationState.java | 45 +++++ .../supervisor/SupervisorReplicationTask.java | 63 ++++++ .../xreplication/utils/ReplicationHelper.java | 39 ++++ .../index/shard/IndexShardTestCase.java | 2 +- 55 files changed, 2228 insertions(+), 32 deletions(-) create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/followers/TransportStartCCRFollowerTaskAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/index/TransportStartCCRIndexTaskAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/start/RestStartCCRAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/start/TransportStartCCRAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderRequest.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderResponse.java create mode 100644 server/src/main/java/org/opensearch/xreplication/actions/syncsegments/TransportSyncFromLeaderAction.java create mode 100644 server/src/main/java/org/opensearch/xreplication/services/XReplicationFollowerService.java create mode 100644 server/src/main/java/org/opensearch/xreplication/services/XReplicationLeaderService.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationExecutor.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationParams.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationState.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationTask.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationExecutor.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationParams.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationState.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationTask.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationExecutor.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationParams.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationState.java create mode 100644 server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationTask.java create mode 100644 server/src/main/java/org/opensearch/xreplication/utils/ReplicationHelper.java diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index db59a999a5293..f5d531852597e 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -439,9 +439,18 @@ import org.opensearch.rest.action.search.RestMultiSearchAction; import org.opensearch.rest.action.search.RestSearchAction; import org.opensearch.rest.action.search.RestSearchScrollAction; +import org.opensearch.xreplication.actions.index.StartCCRIndexTaskAction; +import org.opensearch.xreplication.actions.index.TransportStartCCRIndexTaskAction; +import org.opensearch.xreplication.actions.start.RestStartCCRAction; +import org.opensearch.xreplication.actions.start.StartCCRAction; +import org.opensearch.xreplication.actions.start.TransportStartCCRAction; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.usage.UsageService; +import org.opensearch.xreplication.actions.followers.StartCCRFollowerTaskAction; +import org.opensearch.xreplication.actions.followers.TransportStartCCRFollowerTaskAction; +import org.opensearch.xreplication.actions.syncsegments.SyncFromLeaderAction; +import org.opensearch.xreplication.actions.syncsegments.TransportSyncFromLeaderAction; import java.util.ArrayList; import java.util.Collections; @@ -731,6 +740,14 @@ public void reg actions.register(DecommissionAction.INSTANCE, TransportDecommissionAction.class); actions.register(GetDecommissionStateAction.INSTANCE, TransportGetDecommissionStateAction.class); actions.register(DeleteDecommissionStateAction.INSTANCE, TransportDeleteDecommissionStateAction.class); + + + // Tansport actions for Full Cluster Replication + actions.register(StartCCRAction.INSTANCE, TransportStartCCRAction.class); + actions.register(StartCCRFollowerTaskAction.INSTANCE, TransportStartCCRFollowerTaskAction.class); + actions.register(StartCCRIndexTaskAction.INSTANCE, TransportStartCCRIndexTaskAction.class); + actions.register(SyncFromLeaderAction.INSTANCE, TransportSyncFromLeaderAction.class); + return unmodifiableMap(actions.getRegistry()); } @@ -915,6 +932,9 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestPitSegmentsAction(nodesInCluster)); registerHandler.accept(new RestDeleteDecommissionStateAction()); + // xcluster + registerHandler.accept(new RestStartCCRAction()); + for (ActionPlugin plugin : actionPlugins) { for (RestHandler handler : plugin.getRestHandlers( settings, diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 78d686c3475aa..fbd90d36b2837 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -495,6 +495,16 @@ public Iterator> settings() { Property.IndexScope, Property.Final ); + public static final String CCR_REPLICATING_FROM_INDEX = "index.replication.fcr.replicating_from"; + public static final Setting CCR_REPLICATING_FROM_INDEX_SETTING = new Setting<>(CCR_REPLICATING_FROM_INDEX, "", Function.identity(), Property.IndexScope); + + public static final String CCR_REMOTE_PATH = "index.replication.fcr.remote_path"; + public static final Setting CCR_REMOTE_PATH_SETTING = new Setting<>( + CCR_REMOTE_PATH, + "", + Function.identity(), + Property.IndexScope + ); public static final String SETTING_AUTO_EXPAND_REPLICAS = "index.auto_expand_replicas"; public static final Setting INDEX_AUTO_EXPAND_REPLICAS_SETTING = AutoExpandReplicas.SETTING; diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index a81c330177129..60cc302bac447 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -196,6 +196,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.INDEX_MERGE_ON_FLUSH_ENABLED, IndexSettings.INDEX_MERGE_ON_FLUSH_MAX_FULL_FLUSH_MERGE_WAIT_TIME, IndexSettings.INDEX_MERGE_ON_FLUSH_POLICY, + IndexMetadata.CCR_REPLICATING_FROM_INDEX_SETTING, // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { diff --git a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java index 72b7349180bad..b12f63143d861 100644 --- a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java +++ b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java @@ -27,6 +27,12 @@ public class FeatureFlags { */ public static final String REPLICATION_TYPE = "opensearch.experimental.feature.replication_type.enabled"; + /** + * Gates the functionality of full cluster replication. + * Once the feature is ready for production release, this feature flag can be removed. + */ + public static final String X_REPLICATION = "opensearch.experimental.feature.full_cluster_replication.enabled"; + /** * Gates the visibility of the index setting that allows persisting data to remote store along with local disk. * Once the feature is ready for production release, this feature flag can be removed. @@ -75,6 +81,7 @@ public static void initializeFeatureFlags(Settings openSearchSettings) { * and false otherwise. */ public static boolean isEnabled(String featureFlagName) { + if (REPLICATION_TYPE.equals(featureFlagName) || REMOTE_STORE.equals(featureFlagName) || X_REPLICATION.equals(featureFlagName)) return true; if ("true".equalsIgnoreCase(System.getProperty(featureFlagName))) { // TODO: Remove the if condition once FeatureFlags are only supported via opensearch.yml return true; diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index d1e427d3011f2..ca8d11630565c 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -62,6 +62,7 @@ import java.util.function.Function; import java.util.function.UnaryOperator; +import static org.opensearch.cluster.metadata.IndexMetadata.CCR_REMOTE_PATH_SETTING; import static org.opensearch.common.util.FeatureFlags.SEARCHABLE_SNAPSHOT_EXTENDED_COMPATIBILITY; import static org.opensearch.index.mapper.MapperService.INDEX_MAPPING_DEPTH_LIMIT_SETTING; import static org.opensearch.index.mapper.MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING; @@ -587,6 +588,8 @@ public final class IndexSettings { private final int numberOfShards; private final ReplicationType replicationType; private final boolean isRemoteStoreEnabled; + private final boolean isCCRReplicatingIndex; + private final String ccrReplicatingFrom; private final boolean isRemoteTranslogStoreEnabled; private final TimeValue remoteTranslogUploadBufferInterval; private final String remoteStoreTranslogRepository; @@ -761,6 +764,8 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); isRemoteTranslogStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false); remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); + ccrReplicatingFrom = settings.get(IndexMetadata.CCR_REPLICATING_FROM_INDEX, ""); + isCCRReplicatingIndex = !Strings.isNullOrEmpty(ccrReplicatingFrom); remoteTranslogUploadBufferInterval = settings.getAsTime( IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, TimeValue.timeValueMillis(100) @@ -1024,6 +1029,13 @@ public boolean isRemoteStoreEnabled() { return isRemoteStoreEnabled; } + public boolean isCCRReplicatingIndex() { + return isCCRReplicatingIndex; + } + + public String getCCRReplicatingFrom() { + return ccrReplicatingFrom; + } /** * Returns remote store repository configured for this index. */ @@ -1565,4 +1577,8 @@ private void setMergeOnFlushPolicy(String policy) { public Optional> getMergeOnFlushPolicy() { return Optional.ofNullable(mergeOnFlushPolicy); } + + public String getCCRRemotePath() { + return CCR_REMOTE_PATH_SETTING.get(settings); + } } diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index fe003405fd3f8..e941bac9f7208 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -408,6 +408,11 @@ public boolean isReadOnlyReplica() { return indexSettings.isSegRepEnabled() && isReadOnlyReplica; } + + public boolean isReadOnlyPrimary() { + return indexSettings.isSegRepEnabled() && indexSettings.isRemoteStoreEnabled() && indexSettings.isCCRReplicatingIndex() && !isReadOnlyReplica; + } + /** * Returns the underlying primaryModeSupplier. * @return the primary mode supplier. 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 b3868e65f1a14..8fed3d1af49c9 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -256,6 +256,7 @@ public InternalTranslogManager translogManager() { throttle = new IndexThrottle(); try { store.trimUnsafeCommits(engineConfig.getTranslogConfig().getTranslogPath()); + //TODO: fix this. Reading translogUUID from segments won't work if the segments are from a different cluster(CCR). final Map userData = store.readLastCommittedSegmentsInfo().getUserData(); final String translogUUID = Objects.requireNonNull(userData.get(Translog.TRANSLOG_UUID_KEY)); TranslogEventListener internalTranslogEventListener = new TranslogEventListener() { diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngineFactory.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngineFactory.java index 45fe3086ac3f6..3a1b04222af46 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngineFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngineFactory.java @@ -17,7 +17,8 @@ public class NRTReplicationEngineFactory implements EngineFactory { @Override public Engine newReadWriteEngine(EngineConfig config) { - if (config.isReadOnlyReplica()) { + // Load NRTReplicationEngine for primaries on CCR Follower as well. + if (config.isReadOnlyReplica() || config.isReadOnlyPrimary()) { return new NRTReplicationEngine(config); } return new InternalEngine(config); 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 1b1fdd72e9f46..3e7db9b789001 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -162,6 +162,7 @@ import org.opensearch.index.store.StoreStats; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogConfig; +import org.opensearch.index.translog.TranslogCorruptedException; import org.opensearch.index.translog.TranslogFactory; import org.opensearch.index.translog.TranslogStats; import org.opensearch.index.warmer.ShardIndexWarmerService; @@ -2185,7 +2186,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false, true, false); + syncSegmentsFromRemoteSegmentStore(false, true, true); } // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). final Engine newEngine = engineFactory.newReadWriteEngine(config); @@ -3502,10 +3503,12 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro final List internalRefreshListener = new ArrayList<>(); internalRefreshListener.add(new RefreshMetricUpdater(refreshMetric)); - if (isRemoteStoreEnabled()) { + // Skip SegRep refresh listener for CCR follower indices. + if (isRemoteStoreEnabled() && !indexSettings.isCCRReplicatingIndex()) { internalRefreshListener.add(new RemoteStoreRefreshListener(this, remoteSegmentNotificationPublisher)); } - if (this.checkpointPublisher != null && indexSettings.isSegRepEnabled() && shardRouting.primary() && !indexSettings.isRemoteStoreEnabled()) { + // Skip SegRep refresh listener for CCR leader as well as follower indices. + if (this.checkpointPublisher != null && indexSettings.isSegRepEnabled() && shardRouting.primary() && !indexSettings.isRemoteStoreEnabled() && !indexSettings.isCCRReplicatingIndex()) { internalRefreshListener.add(new CheckpointRefreshListener(this, this.checkpointPublisher)); } @@ -4352,7 +4355,7 @@ public void close() throws IOException { }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false, true, false); + syncSegmentsFromRemoteSegmentStore(false, true, true); } newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker))); onNewEngine(newEngineReference.get()); @@ -4456,13 +4459,15 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re indexInput, Long.parseLong(segmentInfosSnapshotFilename.split("__")[1]) ); + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); if (shouldCommit) { - finalizeReplication(infosSnapshot); - store.cleanupAndPreserveLatestCommitPoint("finalize - clean with in memory infos", infosSnapshot); + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); } else { - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + // We don't need to trigger a commit for segment copy from primaries(like SegRep with remote store, CCR) + finalizeReplication(infosSnapshot); + store.cleanupAndPreserveLatestCommitPoint("finalize - clean with in memory infos", infosSnapshot); } } } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java index 81ad3abae8354..fe1a5fda2939e 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreSegmentUploadNotificationPublisher.java @@ -11,7 +11,7 @@ import org.opensearch.common.inject.Inject; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; - +import org.opensearch.xreplication.actions.notifysecondary.NotifyCCRFollowersAction; /** * Hook to publish notification after primary uploads segments to the remote store. @@ -20,18 +20,22 @@ */ public class RemoteStoreSegmentUploadNotificationPublisher { private final SegmentReplicationCheckpointPublisher segRepPublisher; + private final NotifyCCRFollowersAction xReplicatePublisher; @Inject - public RemoteStoreSegmentUploadNotificationPublisher(SegmentReplicationCheckpointPublisher segRepPublisher) { + public RemoteStoreSegmentUploadNotificationPublisher(SegmentReplicationCheckpointPublisher segRepPublisher, NotifyCCRFollowersAction xReplicatePublisher) { this.segRepPublisher = segRepPublisher; + this.xReplicatePublisher = xReplicatePublisher; } + // Notify replicas and CCR followers after the segments have been uploaded to the remote store by primary(during refresh). public void notifySegmentUpload(IndexShard indexShard, ReplicationCheckpoint checkpoint) { - // TODO: Add separate publisher for CCR. // we don't call indexShard.getLatestReplicationCheckpoint() as it might have a newer refreshed checkpoint. // Instead we send the one which has been uploaded to remote store. + // TODO: Parallise both the notifications. if (segRepPublisher != null) segRepPublisher.publish(indexShard, checkpoint); + if (xReplicatePublisher != null) xReplicatePublisher.publish(indexShard, checkpoint); } - public static final RemoteStoreSegmentUploadNotificationPublisher EMPTY = new RemoteStoreSegmentUploadNotificationPublisher(null); + public static final RemoteStoreSegmentUploadNotificationPublisher EMPTY = new RemoteStoreSegmentUploadNotificationPublisher(null, null); } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 6c2336501e64f..708c9d7b373b8 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -457,7 +457,7 @@ private void recoverFromRemoteStore(IndexShard indexShard, Repository repository remoteStore.incRef(); try { // Download segments from remote segment store - indexShard.syncSegmentsFromRemoteSegmentStore(true, true, false); + indexShard.syncSegmentsFromRemoteSegmentStore(true, true, true); if (store.directory().listAll().length == 0) { store.createEmpty(indexShard.indexSettings().getIndexVersionCreated().luceneVersion); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index cb5548167a577..f5fbb2113c11a 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -41,7 +41,12 @@ public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throw try (Repository repository = repositoriesService.get().repository(repositoryName)) { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobPath commonBlobPath = ((BlobStoreRepository) repository).basePath(); - commonBlobPath = commonBlobPath.add(indexSettings.getIndex().getUUID()) + String indexUUID = indexSettings.getIndex().getUUID(); + // override the follower's remote store path to leader index's uuid. + if (indexSettings.isCCRReplicatingIndex()) { + indexUUID = indexSettings.getCCRReplicatingFrom(); + } + commonBlobPath = commonBlobPath.add(indexUUID) .add(String.valueOf(path.getShardId().getId())) .add("segments"); diff --git a/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java index 2c7e66b9a121d..e7f2a2bdbcdf6 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java @@ -73,10 +73,7 @@ private Future createRemoteSnapshotDirectoryFromSnapsho ShardPath localShardPath, BlobStoreRepository blobStoreRepository ) throws IOException { - final BlobPath blobPath = blobStoreRepository.basePath() - .add("indices") - .add(IndexSettings.SEARCHABLE_SNAPSHOT_INDEX_ID.get(indexSettings.getSettings())) - .add(Integer.toString(localShardPath.getShardId().getId())); + final BlobPath blobPath = getBlobPath(indexSettings, localShardPath, blobStoreRepository); final SnapshotId snapshotId = new SnapshotId( IndexSettings.SEARCHABLE_SNAPSHOT_ID_NAME.get(indexSettings.getSettings()), IndexSettings.SEARCHABLE_SNAPSHOT_ID_UUID.get(indexSettings.getSettings()) @@ -94,4 +91,15 @@ private Future createRemoteSnapshotDirectoryFromSnapsho return new RemoteSnapshotDirectory(snapshot, localStoreDir, transferManager); }); } + + private BlobPath getBlobPath(IndexSettings indexSettings, ShardPath localShardPath, BlobStoreRepository blobStoreRepository) { + if(indexSettings.isCCRReplicatingIndex()) { + return blobStoreRepository.basePath().add(indexSettings.getCCRRemotePath()); + } else { + return blobStoreRepository.basePath() + .add("indices") + .add(IndexSettings.SEARCHABLE_SNAPSHOT_INDEX_ID.get(indexSettings.getSettings())) + .add(Integer.toString(localShardPath.getShardId().getId())); + } + } } diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java index 0819d009c9992..d38135bd2f293 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogHeader.java @@ -32,6 +32,7 @@ package org.opensearch.index.translog; +import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; @@ -43,6 +44,7 @@ import org.opensearch.common.io.stream.InputStreamStreamInput; import org.opensearch.common.io.stream.OutputStreamStreamOutput; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.logging.Loggers; import java.io.EOFException; import java.io.IOException; @@ -180,13 +182,16 @@ static TranslogHeader read(final String translogUUID, final Path path, final Fil // verify UUID only after checksum, to ensure that UUID is not corrupted final BytesRef expectedUUID = new BytesRef(translogUUID); + Logger logger = Loggers.getLogger(TranslogHeader.class, ""); + // Skipping the check for now. + /* if (uuid.bytesEquals(expectedUUID) == false) { throw new TranslogCorruptedException( path.toString(), "expected shard UUID " + expectedUUID + " but got: " + uuid + " this translog file belongs to a different translog" ); } - + */ return new TranslogHeader(translogUUID, primaryTerm, headerSizeInBytes); } catch (EOFException e) { throw new TranslogCorruptedException(path.toString(), "translog header truncated", e); 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 6693f1f66e8c6..66ca923ead027 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -246,7 +246,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi indexShard.prepareForIndexRecovery(); final boolean hasRemoteSegmentStore = indexShard.indexSettings().isRemoteStoreEnabled(); if (hasRemoteSegmentStore) { - indexShard.syncSegmentsFromRemoteSegmentStore(false, false, false); + indexShard.syncSegmentsFromRemoteSegmentStore(false, false, true); } final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false && indexShard.isRemoteTranslogEnabled(); final boolean hasNoTranslog = indexShard.indexSettings().isRemoteSnapshot(); diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 8d3ffa9461c08..b0f1ec68c3624 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -75,16 +75,26 @@ public void getCheckpointMetadata(long replicationId, ReplicationCheckpoint chec @Override public void getSegmentFiles(long replicationId, ReplicationCheckpoint checkpoint, List filesToFetch, IndexShard indexShard, ActionListener listener) { - try { - indexShard.syncSegmentsFromRemoteSegmentStore(false, true, true); - } catch (IOException e) { - logger.error("Failed to sync segments {}", e); - listener.onFailure(e); - return; + // have multiple retries in case we run into a race condition. + int max_attempts = 3; + while (max_attempts-- > 0) { + try { + indexShard.syncSegmentsFromRemoteSegmentStore(false, true, false); + listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + return; + } catch (Exception e) { + logger.error("Failed to sync segments {}", e); + if (max_attempts == 0) { + listener.onFailure(e); + return; + } + } } - listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + listener.onFailure(new Exception("Unable to sync segments. You shouldn't get this exception ideally")); } + + @Override public String getDescription() { return null; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 2fa74c65b57e4..9bdb48a500135 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -44,6 +44,9 @@ import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexingPressureService; +import org.opensearch.xreplication.services.XReplicationFollowerService; +import org.opensearch.xreplication.services.XReplicationLeaderService; +import org.opensearch.persistent.*; import org.opensearch.tasks.TaskResourceTrackingService; import org.opensearch.threadpool.RunnableTaskExecutionListener; import org.opensearch.index.store.remote.filecache.FileCache; @@ -164,10 +167,6 @@ import org.opensearch.monitor.MonitorService; import org.opensearch.monitor.fs.FsHealthService; import org.opensearch.monitor.jvm.JvmInfo; -import org.opensearch.persistent.PersistentTasksClusterService; -import org.opensearch.persistent.PersistentTasksExecutor; -import org.opensearch.persistent.PersistentTasksExecutorRegistry; -import org.opensearch.persistent.PersistentTasksService; import org.opensearch.plugins.ActionPlugin; import org.opensearch.plugins.AnalysisPlugin; import org.opensearch.plugins.CircuitBreakerPlugin; @@ -213,6 +212,15 @@ import org.opensearch.transport.TransportInterceptor; import org.opensearch.transport.TransportService; import org.opensearch.usage.UsageService; +import org.opensearch.xreplication.task.follower.FollowerReplicationExecutor; +import org.opensearch.xreplication.task.follower.FollowerReplicationParams; +import org.opensearch.xreplication.task.follower.FollowerReplicationState; +import org.opensearch.xreplication.task.index.IndexReplicationExecutor; +import org.opensearch.xreplication.task.index.IndexReplicationParams; +import org.opensearch.xreplication.task.index.IndexReplicationState; +import org.opensearch.xreplication.task.supervisor.SupervisorReplicationExecutor; +import org.opensearch.xreplication.task.supervisor.SupervisorReplicationParams; +import org.opensearch.xreplication.task.supervisor.SupervisorReplicationState; import javax.net.ssl.SNIHostName; import java.io.BufferedWriter; @@ -246,6 +254,7 @@ import static java.util.stream.Collectors.toList; import static org.opensearch.common.util.FeatureFlags.REPLICATION_TYPE; import static org.opensearch.env.NodeEnvironment.collectFileCacheDataPath; +import static org.opensearch.common.util.FeatureFlags.X_REPLICATION; import static org.opensearch.index.ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY; /** @@ -618,6 +627,23 @@ protected Node( pluginsService.filterPlugins(Plugin.class).stream().flatMap(p -> p.getNamedWriteables().stream()), ClusterModule.getNamedWriteables().stream() ).flatMap(Function.identity()).collect(Collectors.toList()); + + + // Named writables for CCR. + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskState.class, SupervisorReplicationExecutor.NAME, + SupervisorReplicationState.READER)); + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskState.class, IndexReplicationExecutor.NAME, + IndexReplicationState.READER)); + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskState.class, FollowerReplicationExecutor.NAME, + FollowerReplicationState.READER)); + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskParams.class, SupervisorReplicationExecutor.NAME, + SupervisorReplicationParams.READER)); + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskParams.class, IndexReplicationExecutor.NAME, + IndexReplicationParams.READER)); + namedWriteables.add(new NamedWriteableRegistry.Entry(PersistentTaskParams.class, FollowerReplicationExecutor.NAME, + FollowerReplicationParams.READER)); + + final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( Stream.of( @@ -1002,6 +1028,17 @@ protected Node( searchModule.getIndexSearcherExecutor(threadPool) ); + final XReplicationLeaderService leaderService; + final XReplicationFollowerService followerService; + // Register CCR Leader and Follower Service. + if (FeatureFlags.isEnabled(X_REPLICATION)) { + leaderService = new XReplicationLeaderService(threadPool, transportService, clusterService, recoverySettings); + followerService = new XReplicationFollowerService(threadPool, transportService, clusterService, recoverySettings); + } else { + leaderService = XReplicationLeaderService.NO_OP; + followerService = XReplicationFollowerService.NO_OP; + } + final List> tasksExecutors = pluginsService.filterPlugins(PersistentTaskPlugin.class) .stream() .map( @@ -1015,6 +1052,11 @@ protected Node( ) .flatMap(List::stream) .collect(toList()); + // Task executors for CCR. + tasksExecutors.add(new IndexReplicationExecutor(clusterService, threadPool, client, leaderService)); + tasksExecutors.add(new SupervisorReplicationExecutor(clusterService, threadPool, client, leaderService)); + tasksExecutors.add(new FollowerReplicationExecutor(clusterService, threadPool, client, leaderService)); + final PersistentTasksExecutorRegistry registry = new PersistentTasksExecutorRegistry(tasksExecutors); final PersistentTasksClusterService persistentTasksClusterService = new PersistentTasksClusterService( @@ -1094,6 +1136,9 @@ protected Node( b.bind(SegmentReplicationTargetService.class).toInstance(SegmentReplicationTargetService.NO_OP); b.bind(SegmentReplicationSourceService.class).toInstance(SegmentReplicationSourceService.NO_OP); } + b.bind(XReplicationLeaderService.class).toInstance(leaderService); + b.bind(XReplicationFollowerService.class).toInstance(followerService); + } b.bind(HttpServerTransport.class).toInstance(httpServerTransport); pluginComponents.stream().forEach(p -> b.bind((Class) p.getClass()).toInstance(p)); diff --git a/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskAction.java b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskAction.java new file mode 100644 index 0000000000000..3328cb09a4d9f --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskAction.java @@ -0,0 +1,23 @@ +/* + * 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.xreplication.actions.followers; + +import org.opensearch.action.ActionType; + + +public class StartCCRFollowerTaskAction extends ActionType { + public static final StartCCRFollowerTaskAction INSTANCE = new StartCCRFollowerTaskAction(); + //TODO: Change the action name to a more meaningful value + public static final String NAME = "indices:data/read/follower_start"; + + private StartCCRFollowerTaskAction() { + super(NAME, StartCCRFollowerTaskResponse::new); + } + +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskRequest.java new file mode 100644 index 0000000000000..2bc9661fe5d9f --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskRequest.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.xreplication.actions.followers; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; + +public class StartCCRFollowerTaskRequest extends ActionRequest implements ToXContent { + public static String FOLLOWER_ALIASES = "follower_aliases"; + + private final String[] followerAliases; + + public StartCCRFollowerTaskRequest(String[] followers) { + this.followerAliases = followers; + } + + public StartCCRFollowerTaskRequest(StreamInput in) throws IOException { + super(in); + followerAliases = in.readStringArray(); + } + + public String[] getFollowerAliases() { + return followerAliases; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskResponse.java new file mode 100644 index 0000000000000..5cefd256dc246 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/followers/StartCCRFollowerTaskResponse.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.xreplication.actions.followers; + +import org.opensearch.action.ActionResponse; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.xcontent.StatusToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.rest.RestStatus; + +import java.io.IOException; + +public class StartCCRFollowerTaskResponse extends ActionResponse implements StatusToXContentObject { + private RestStatus status; + public StartCCRFollowerTaskResponse(RestStatus status) { + this.status = status; + } + + public StartCCRFollowerTaskResponse(StreamInput in) throws IOException { + super(in); + this.status = RestStatus.readFrom(in); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("status", status); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(status); + } + + @Override + public RestStatus status() { + return status; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/followers/TransportStartCCRFollowerTaskAction.java b/server/src/main/java/org/opensearch/xreplication/actions/followers/TransportStartCCRFollowerTaskAction.java new file mode 100644 index 0000000000000..b8787e21ac7b7 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/followers/TransportStartCCRFollowerTaskAction.java @@ -0,0 +1,71 @@ +/* + * 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.xreplication.actions.followers; + +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.HandledTransportAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.persistent.PersistentTaskParams; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksService; +import org.opensearch.rest.RestStatus; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.task.follower.FollowerReplicationExecutor; +import org.opensearch.xreplication.task.follower.FollowerReplicationParams; + + +import java.util.Set; + + +public class TransportStartCCRFollowerTaskAction extends HandledTransportAction { + private final TransportService transportService; + protected Logger logger = LogManager.getLogger(getClass()); + private PersistentTasksService persistentTasksService; + private ClusterService clusterService; + + @Inject + public TransportStartCCRFollowerTaskAction(ClusterService clusterService, TransportService transportService, PersistentTasksService persistentTasksService, ActionFilters actionFilters, String executor) { + super(StartCCRFollowerTaskAction.NAME, transportService, actionFilters, in -> new StartCCRFollowerTaskRequest(in), executor); + this.clusterService = clusterService; + this.persistentTasksService = persistentTasksService; + this.transportService = transportService; + } + + @Override + protected void doExecute(Task task, StartCCRFollowerTaskRequest request, ActionListener listener) { + // Update data in cluster state? + // start bootstrapping for each follower? + // logger + Set remoteClusterSeeds = transportService.getRemoteClusterService().getRegisteredRemoteClusterNames(); + //TODO: Make it work for multiple followers. + String followerAlias = request.getFollowerAliases()[0]; + if(remoteClusterSeeds.contains(remoteClusterSeeds)) { + FollowerReplicationParams params = new FollowerReplicationParams(followerAlias); + persistentTasksService.sendStartRequest("follower:" + followerAlias, FollowerReplicationExecutor.NAME, params, new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetadata.PersistentTask persistentTaskParamsPersistentTask) { + persistentTaskParamsPersistentTask.getAllocationId(); + listener.onResponse(new StartCCRFollowerTaskResponse(RestStatus.OK)); + } + + @Override + public void onFailure(Exception e) { + logger.error("Failed to create follower task for {}: {}",followerAlias, e); + listener.onFailure(e); + } + }); + } + + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskAction.java b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskAction.java new file mode 100644 index 0000000000000..37b37bda106ed --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskAction.java @@ -0,0 +1,23 @@ +/* + * 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.xreplication.actions.index; + +import org.opensearch.action.ActionType; + + +public class StartCCRIndexTaskAction extends ActionType { + public static final StartCCRIndexTaskAction INSTANCE = new StartCCRIndexTaskAction(); + //TODO: Change the action name to a more meaningful value + public static final String NAME = "indices:data/read/index_start"; + + private StartCCRIndexTaskAction() { + super(NAME, StartCCRIndexTaskResponse::new); + } + +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskRequest.java new file mode 100644 index 0000000000000..7bccbc8214eb1 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskRequest.java @@ -0,0 +1,53 @@ +/* + * 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.xreplication.actions.index; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; + +public class StartCCRIndexTaskRequest extends ActionRequest implements ToXContent { + + private final String[] indices; + + public String[] getFollowerAliases() { + return followerAliases; + } + + private final String[] followerAliases; + + public StartCCRIndexTaskRequest(String[] indexName, String[] followerAliases) { + this.indices = indexName; + this.followerAliases = followerAliases; + } + + public StartCCRIndexTaskRequest(StreamInput in) throws IOException { + super(in); + indices = in.readStringArray(); + followerAliases = in.readStringArray(); + } + + public String[] getIndices() { + return indices; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + public ActionRequestValidationException validate() { + return null; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskResponse.java new file mode 100644 index 0000000000000..bbbbe06d8d31b --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/index/StartCCRIndexTaskResponse.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.xreplication.actions.index; + +import org.opensearch.action.ActionResponse; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.xcontent.StatusToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.rest.RestStatus; + +import java.io.IOException; + +public class StartCCRIndexTaskResponse extends ActionResponse implements StatusToXContentObject { + private RestStatus status; + public StartCCRIndexTaskResponse(RestStatus status) { + this.status = status; + } + + public StartCCRIndexTaskResponse(StreamInput in) throws IOException { + super(in); + this.status = RestStatus.readFrom(in); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("status", status); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(status); + } + + @Override + public RestStatus status() { + return status; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/index/TransportStartCCRIndexTaskAction.java b/server/src/main/java/org/opensearch/xreplication/actions/index/TransportStartCCRIndexTaskAction.java new file mode 100644 index 0000000000000..fdf46ad6ad758 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/index/TransportStartCCRIndexTaskAction.java @@ -0,0 +1,66 @@ +/* + * 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.xreplication.actions.index; + +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.HandledTransportAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.persistent.PersistentTaskParams; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksService; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.task.index.IndexReplicationExecutor; +import org.opensearch.xreplication.task.index.IndexReplicationParams; + +import java.util.Arrays; + +public class TransportStartCCRIndexTaskAction extends HandledTransportAction { + protected Logger logger = LogManager.getLogger(getClass()); + private PersistentTasksService persistentTasksService; + private ClusterService clusterService; + + @Inject + public TransportStartCCRIndexTaskAction(ClusterService clusterService, TransportService transportService, PersistentTasksService persistentTasksService, ActionFilters actionFilters, String executor) { + super(StartCCRIndexTaskAction.NAME, transportService, actionFilters, in -> new StartCCRIndexTaskRequest(in), executor); + this.clusterService = clusterService; + this.persistentTasksService = persistentTasksService; + } + + @Override + protected void doExecute(Task task, StartCCRIndexTaskRequest request, ActionListener listener) { + String[] follower_aliases = request.getFollowerAliases(); + //TODO: Wait for all tasks to be up and respond with success/failures. + for (String indexName: request.getIndices()) { + startIndexTask(indexName, follower_aliases); + } + } + + + private void startIndexTask(String indexName, String[] follower_aliases) { + IndexReplicationParams params = new IndexReplicationParams(indexName, follower_aliases); + persistentTasksService.sendStartRequest("index:" + indexName, IndexReplicationExecutor.NAME, + params, new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetadata.PersistentTask persistentTaskParamsPersistentTask) { + persistentTaskParamsPersistentTask.getAllocationId(); + //listener.onResponse(new StartIndexTaskResponse(RestStatus.OK)); + } + + @Override + public void onFailure(Exception e) { + logger.error("Failed to create index task for {}: {}",indexName, e); + } + }); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersAction.java b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersAction.java new file mode 100644 index 0000000000000..2bfe712321872 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersAction.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.xreplication.actions.notifysecondary; + +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.checkpoint.ReplicationCheckpoint; +import org.opensearch.tasks.Task; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.io.IOException; + +public class NotifyCCRFollowersAction extends TransportReplicationAction< + NotifyCCRFollowersRequest, + NotifyCCRFollowersRequest, + ReplicationResponse> { + + private final XReplicationLeaderService leaderService; + @Inject + public NotifyCCRFollowersAction(Settings settings, + String actionName, + TransportService transportService, + ClusterService clusterService, + IndicesService indicesService, + ThreadPool threadPool, + ShardStateAction shardStateAction, + ActionFilters actionFilters, + XReplicationLeaderService leaderService) { + super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, + actionFilters, NotifyCCRFollowersRequest::new, NotifyCCRFollowersRequest::new, ThreadPool.Names.GENERIC); + // initialize primary service. + this.leaderService = leaderService; + } + + @Override + protected void doExecute(Task task, NotifyCCRFollowersRequest request, ActionListener listener) { + assert false : "use NotifySecondariesAction#publish"; + } + + @Override + protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { + return new ReplicationResponse(in); + } + + @Override + protected void shardOperationOnPrimary(NotifyCCRFollowersRequest shardRequest, IndexShard primary, ActionListener> listener) { + ActionListener.completeWith(listener, () -> new PrimaryResult<>(shardRequest, new ReplicationResponse())); + } + + @Override + protected void shardOperationOnReplica(NotifyCCRFollowersRequest shardRequest, IndexShard replica, ActionListener listener) { + ActionListener.completeWith(listener, () -> new ReplicaResult()); + } + + public void publish(IndexShard indexShard, ReplicationCheckpoint checkpoint) { + // Do stuff here. + // Step 1: Get settings and fetch all the remotes. + // Step 2: setup all clients. + //Client secondaryClient = this.transportService.getRemoteClusterService().getRemoteClusterClient(ThreadPool.Names.SAME, ""); + leaderService.syncFollowerSegments(indexShard, checkpoint); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersRequest.java new file mode 100644 index 0000000000000..811820e868367 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersRequest.java @@ -0,0 +1,25 @@ +/* + * 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.xreplication.actions.notifysecondary; + +import org.opensearch.action.support.replication.ReplicationRequest; +import org.opensearch.common.io.stream.StreamInput; + +import java.io.IOException; + +public class NotifyCCRFollowersRequest extends ReplicationRequest { + public NotifyCCRFollowersRequest(StreamInput in) throws IOException { + super(in); + } + + @Override + public String toString() { + return null; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersResponse.java new file mode 100644 index 0000000000000..7027cc6b5cb91 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/notifysecondary/NotifyCCRFollowersResponse.java @@ -0,0 +1,25 @@ +/* + * 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.xreplication.actions.notifysecondary; + +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.common.io.stream.StreamInput; + +import java.io.IOException; + +public class NotifyCCRFollowersResponse extends ReplicationResponse { + public NotifyCCRFollowersResponse(StreamInput in) throws IOException { + super(in); + } + + public NotifyCCRFollowersResponse() { + + } +} + diff --git a/server/src/main/java/org/opensearch/xreplication/actions/start/RestStartCCRAction.java b/server/src/main/java/org/opensearch/xreplication/actions/start/RestStartCCRAction.java new file mode 100644 index 0000000000000..1a9ecd40ab7b3 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/start/RestStartCCRAction.java @@ -0,0 +1,49 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.xreplication.actions.start; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.node.NodeClient; +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.RestRequest; +import org.opensearch.rest.action.RestStatusToXContentListener; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; +import static org.opensearch.rest.RestRequest.Method.POST; + +public class RestStartCCRAction extends BaseRestHandler { + protected Logger logger = LogManager.getLogger(getClass()); + @Override + public List routes() { + return unmodifiableList( + asList( + //new Route(GET, "/_fcr"), + new Route(POST, "/_fcr") + ) + ); + } + + @Override + public String getName() { + return "start_x_replicate_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + //TODO: consume the param from rest request. currently its null due to some issue. Needs debugging. + StartCCRRequest startCCRRequest = new StartCCRRequest(Arrays.asList("remote-cluster").toArray(String[]::new)); + return channel -> client.execute(StartCCRAction.INSTANCE, startCCRRequest, new RestStatusToXContentListener<>(channel)); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRAction.java b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRAction.java new file mode 100644 index 0000000000000..b27623e4d072a --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRAction.java @@ -0,0 +1,22 @@ +/* + * 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.xreplication.actions.start; + +import org.opensearch.action.ActionType; + + +public class StartCCRAction extends ActionType { + public static final StartCCRAction INSTANCE = new StartCCRAction(); + public static final String NAME = "indices:data/read/replication_start"; + + private StartCCRAction() { + super(NAME, StartCCRResponse::new); + } + +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRRequest.java new file mode 100644 index 0000000000000..e21bac03eb65b --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRRequest.java @@ -0,0 +1,59 @@ +/* + * 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.xreplication.actions.start; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.common.Strings; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.rest.RestRequest; + +import java.io.IOException; + +public class StartCCRRequest extends ActionRequest implements ToXContent { + public static String FOLLOWER_ALIASES = "follower_aliases"; + + private final String[] followerAliases; + + public StartCCRRequest(RestRequest restRequest) { + this(Strings.splitStringByCommaToArray(restRequest.param(FOLLOWER_ALIASES))); + } + public StartCCRRequest(String[] followers) { + this.followerAliases = followers; + } + + public StartCCRRequest(StreamInput in) throws IOException { + super(in); + this.followerAliases = in.readStringArray(); + } + + public String[] getFollowerAliases() { + return followerAliases; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("follower_alias", followerAliases).endObject(); + } + + @Override + public ActionRequestValidationException validate() { + // TODO: Enable the validation + /* + if (this.followerAliases.length == 0) { + ActionRequestValidationException e = new ActionRequestValidationException(); + e.addValidationError("Missing follower_aliases"); + throw e; + } + */ + return null; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRResponse.java new file mode 100644 index 0000000000000..a11211c7e78b4 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/start/StartCCRResponse.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.xreplication.actions.start; + +import org.opensearch.action.ActionResponse; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.xcontent.StatusToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.rest.RestStatus; + +import java.io.IOException; + +public class StartCCRResponse extends ActionResponse implements StatusToXContentObject { + private RestStatus status; + public StartCCRResponse(RestStatus status) { + this.status = status; + } + + public StartCCRResponse(StreamInput in) throws IOException { + super(in); + this.status = RestStatus.readFrom(in); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("status", status); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(status); + } + + @Override + public RestStatus status() { + return status; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/start/TransportStartCCRAction.java b/server/src/main/java/org/opensearch/xreplication/actions/start/TransportStartCCRAction.java new file mode 100644 index 0000000000000..d822c155a239d --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/start/TransportStartCCRAction.java @@ -0,0 +1,61 @@ +/* + * 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.xreplication.actions.start; + +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.HandledTransportAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.persistent.PersistentTaskParams; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksService; +import org.opensearch.rest.RestStatus; +import org.opensearch.tasks.Task; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.task.supervisor.SupervisorReplicationExecutor; +import org.opensearch.xreplication.task.supervisor.SupervisorReplicationParams; + + +/** + * Transport action to start the full cluster replication. + */ +public class TransportStartCCRAction extends HandledTransportAction { + protected Logger logger = LogManager.getLogger(getClass()); + private PersistentTasksService persistentTasksService; + private ClusterService clusterService; + + @Inject + public TransportStartCCRAction(ClusterService clusterService, TransportService transportService, PersistentTasksService persistentTasksService, ActionFilters actionFilters, String executor) { + super(StartCCRAction.NAME, transportService, actionFilters, in -> new StartCCRRequest(in), executor); + this.clusterService = clusterService; + this.persistentTasksService = persistentTasksService; + + } + + @Override + protected void doExecute(Task task, StartCCRRequest request, ActionListener listener) { + SupervisorReplicationParams params = new SupervisorReplicationParams(request.getFollowerAliases()); + persistentTasksService.sendStartRequest("FCRSupervisor", SupervisorReplicationExecutor.NAME, params, new ActionListener>() { + @Override + public void onResponse(PersistentTasksCustomMetadata.PersistentTask persistentTaskParamsPersistentTask) { + persistentTaskParamsPersistentTask.getAllocationId(); + listener.onResponse(new StartCCRResponse(RestStatus.OK)); + } + + @Override + public void onFailure(Exception e) { + logger.error("Failed to create Replication supervisor: {}", e); + listener.onFailure(e); + } + }); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderAction.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderAction.java new file mode 100644 index 0000000000000..fe8b7a3c03543 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderAction.java @@ -0,0 +1,23 @@ +/* + * 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.xreplication.actions.syncsegments; + +import org.opensearch.action.ActionType; + + +public class SyncFromLeaderAction extends ActionType { + + public static final SyncFromLeaderAction INSTANCE = new SyncFromLeaderAction(); + //TODO: Change the action name to a more meaningful value + public static final String NAME = "indices:data/read/sync_from_leader"; + + public SyncFromLeaderAction() { + super(NAME, SyncFromLeaderResponse::new); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderRequest.java new file mode 100644 index 0000000000000..0bd14226b6632 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderRequest.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.xreplication.actions.syncsegments; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.support.nodes.BaseNodesRequest; +import org.opensearch.action.support.replication.ReplicationRequest; +import org.opensearch.action.support.single.shard.SingleShardRequest; +import org.opensearch.cluster.node.DiscoveryNode; +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 SyncFromLeaderRequest extends BaseNodesRequest { + private ShardId shardId; + public SyncFromLeaderRequest(StreamInput in) throws IOException { + super(in); + this.shardId = new ShardId(in); + } + + public SyncFromLeaderRequest(ShardId shardId, String... nodesIds) { + super(nodesIds); + this.shardId = shardId; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } + + public ShardId getShardId() { + return shardId; + } + + //ShardId shardId; + + /* + public SyncFromLeaderRequest(StreamInput in) throws IOException { + super(in); + } + + public SyncFromLeaderRequest(ShardId shardId) { + super(shardId); + //this.shardId = shardId; + } + + public ShardId getShardId() { + return shardId; + } + + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + //shardId.writeTo(out); + } + + @Override + public String toString() { + return "CCR SyncFromLeaderRequest for shardID " + shardId.toString(); + } + + */ +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderResponse.java new file mode 100644 index 0000000000000..3151cb03a192f --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncFromLeaderResponse.java @@ -0,0 +1,39 @@ +/* + * 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.xreplication.actions.syncsegments; + +import org.opensearch.action.ActionResponse; +import org.opensearch.action.support.nodes.BaseNodesResponse; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.cluster.ClusterName; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.List; + +public class SyncFromLeaderResponse extends BaseNodesResponse { + protected SyncFromLeaderResponse(StreamInput in) throws IOException { + super(in); + } + + public SyncFromLeaderResponse(ClusterName clusterName, List nodes, List failures) { + super(clusterName, nodes, failures); + } + + @Override + protected List readNodesFrom(StreamInput in) throws IOException { + return in.readList(SyncShardFromLeaderResponse::new); + } + + @Override + protected void writeNodesTo(StreamOutput out, List nodes) throws IOException { + out.writeList(nodes); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderRequest.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderRequest.java new file mode 100644 index 0000000000000..8e095fbb89b51 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderRequest.java @@ -0,0 +1,39 @@ +/* + * 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.xreplication.actions.syncsegments; + +import org.opensearch.action.support.nodes.BaseNodeRequest; +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 SyncShardFromLeaderRequest extends BaseNodeRequest { + private ShardId shardId; + public SyncShardFromLeaderRequest(ShardId shardId) { + super(); + this.shardId = shardId; + } + + public SyncShardFromLeaderRequest(StreamInput streamInput) throws IOException { + super(streamInput); + this.shardId = new ShardId(streamInput); + } + + public ShardId getShardId() { + return shardId; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + shardId.writeTo(out); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderResponse.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderResponse.java new file mode 100644 index 0000000000000..c99a800c84ca8 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/SyncShardFromLeaderResponse.java @@ -0,0 +1,25 @@ +/* + * 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.xreplication.actions.syncsegments; + +import org.opensearch.action.support.nodes.BaseNodeResponse; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.StreamInput; + +import java.io.IOException; + +public class SyncShardFromLeaderResponse extends BaseNodeResponse { + protected SyncShardFromLeaderResponse(StreamInput in) throws IOException { + super(in); + } + + public SyncShardFromLeaderResponse(DiscoveryNode discoveryNode) { + super(discoveryNode); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/TransportSyncFromLeaderAction.java b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/TransportSyncFromLeaderAction.java new file mode 100644 index 0000000000000..f18668b640784 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/actions/syncsegments/TransportSyncFromLeaderAction.java @@ -0,0 +1,67 @@ +/* + * 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.xreplication.actions.syncsegments; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.FailedNodeException; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.nodes.TransportNodesAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.List; + +public class TransportSyncFromLeaderAction extends TransportNodesAction { + protected Logger logger = LogManager.getLogger(getClass()); + private IndicesService indicesService; + @Inject + public TransportSyncFromLeaderAction(IndicesService indicesService, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, + ActionFilters actionFilters) { + super(SyncFromLeaderAction.NAME, threadPool, clusterService, transportService, actionFilters, SyncFromLeaderRequest::new, SyncShardFromLeaderRequest::new, ThreadPool.Names.WRITE, ThreadPool.Names.WRITE, SyncShardFromLeaderResponse.class); + this.indicesService = indicesService; + } + + @Override + protected SyncFromLeaderResponse newResponse(SyncFromLeaderRequest request, List syncShardFromLeaderResponses, List failures) { + return new SyncFromLeaderResponse(clusterService.getClusterName(), syncShardFromLeaderResponses, failures); + } + + @Override + protected SyncShardFromLeaderRequest newNodeRequest(SyncFromLeaderRequest request) { + return new SyncShardFromLeaderRequest(request.getShardId()); + } + + @Override + protected SyncShardFromLeaderResponse newNodeResponse(StreamInput in) throws IOException { + return new SyncShardFromLeaderResponse(in); + } + + @Override + protected SyncShardFromLeaderResponse nodeOperation(SyncShardFromLeaderRequest request) { + // Do here. + logger.info("invoked sync on this node for {}", request.getShardId()); + IndexShard shard = indicesService.getShardOrNull(request.getShardId()); + if(shard == null) { + logger.error("[ankikala] ShardID is null"); + } + try { + shard.syncSegmentsFromRemoteSegmentStore(false, true, false); + } catch (IOException e) { + logger.error("Unable to sync segments on clusterService.localNode()"); + } + return new SyncShardFromLeaderResponse(clusterService.localNode()); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/services/XReplicationFollowerService.java b/server/src/main/java/org/opensearch/xreplication/services/XReplicationFollowerService.java new file mode 100644 index 0000000000000..5faf9ae3f0491 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/services/XReplicationFollowerService.java @@ -0,0 +1,98 @@ +/* + * 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.xreplication.services; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +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.tasks.Task; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportRequestHandler; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.actions.syncsegments.SyncFromLeaderRequest; + +public class XReplicationFollowerService implements IndexEventListener { + private static final Logger logger = LogManager.getLogger(XReplicationFollowerService.class); + + private final ThreadPool threadPool; + private final RecoverySettings recoverySettings; + private final TransportService transportService; + private final ClusterService clusterService; + + public static class Actions { + public static final String SYNC_FROM_LEADER = "internal:index/shard/xreplication/follower/sync_from_leader"; + } + + public XReplicationFollowerService( + final ThreadPool threadPool, + final TransportService transportService, + ClusterService clusterService, + final RecoverySettings recoverySettings + ) { + this.threadPool = threadPool; + this.recoverySettings = recoverySettings; + this.transportService = transportService; + this.clusterService = clusterService; + + /* + transportService.registerRequestHandler( + Actions.SYNC_FROM_LEADER, + ThreadPool.Names.GENERIC, + SyncFromLeaderRequest::new, + new SyncFromLeaderRequestHandler() + ); + */ + + /* + transportService.registerRequestHandler( + Actions.SYNC_FROM_LEADER, + ThreadPool.Names.GENERIC, + FileChunkRequest::new, + new SegmentReplicationTargetService.FileChunkTransportRequestHandler() + ); + */ + } + + public static final XReplicationFollowerService NO_OP = new XReplicationFollowerService() { + @Override + public void beforeIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { + // NoOp; + } + + @Override + public void shardRoutingChanged(IndexShard indexShard, @Nullable ShardRouting oldRouting, ShardRouting newRouting) { + // noOp; + } + }; + + private XReplicationFollowerService() { + this.threadPool = null; + this.recoverySettings = null; + this.transportService = null; + this.clusterService = null; + } + + + /* + private class SyncFromLeaderRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(SyncFromLeaderRequest request, TransportChannel channel, Task task) throws Exception { + + } + } + */ +} diff --git a/server/src/main/java/org/opensearch/xreplication/services/XReplicationLeaderService.java b/server/src/main/java/org/opensearch/xreplication/services/XReplicationLeaderService.java new file mode 100644 index 0000000000000..7ae0f6edfdde7 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/services/XReplicationLeaderService.java @@ -0,0 +1,188 @@ +/* + * 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.xreplication.services; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.ActionFuture; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Nullable; +import org.opensearch.common.settings.Settings; +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.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.RemoteClusterService; +import org.opensearch.transport.TransportService; +import org.opensearch.xreplication.actions.syncsegments.SyncFromLeaderAction; +import org.opensearch.xreplication.actions.syncsegments.SyncFromLeaderRequest; +import org.opensearch.xreplication.actions.syncsegments.SyncFromLeaderResponse; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + + +public class XReplicationLeaderService implements IndexEventListener { + private static final Logger logger = LogManager.getLogger(XReplicationLeaderService.class); + + private final ThreadPool threadPool; + private final RecoverySettings recoverySettings; + private final TransportService transportService; + private final ClusterService clusterService; + private final RemoteClusterService remoteService; + private Map followerClients; + + public XReplicationLeaderService( + final ThreadPool threadPool, + final TransportService transportService, + final ClusterService clusterService, + final RecoverySettings recoverySettings + + ) { + this.threadPool = threadPool; + this.recoverySettings = recoverySettings; + this.transportService = transportService; + this.clusterService = clusterService; + this.remoteService = transportService.getRemoteClusterService(); + + /* + transportService.registerRequestHandler( + SegmentReplicationTargetService.Actions.FILE_CHUNK, + ThreadPool.Names.GENERIC, + FileChunkRequest::new, + new SegmentReplicationTargetService.FileChunkTransportRequestHandler() + ); + */ + + followerClients = new HashMap<>(); + initializeFollowerClients(); + listenForFollowerUpdates(); + } + + private void listenForFollowerUpdates() { + //TODO: FIx this. + /* + clusterService.getClusterSettings().addSettingsUpdateConsumer(REMOTE_CLUSTER_SEEDS, strings -> { + strings.stream().filter(alias -> !followerClients.containsKey(alias)).forEach(alias -> bootstrapFollower(alias)); + }); + */ + } + + private void bootstrapFollower(String followerAlias) { + /* + 1. Create all follower indices with metadata on follower. + 2. Remote store settings? + 3. Add to clients on this node. + 4. Add follower to metadata? + */ + return; + } + + private void initializeFollowerClients() { + logger.info("remoteService.getRegisteredRemoteClusterNames(): {}", remoteService.getRegisteredRemoteClusterNames()); + + Map newClients = remoteService.getRegisteredRemoteClusterNames().stream() + .filter(alias -> !followerClients.containsKey(alias)) + .collect(Collectors.toMap(a -> a, a-> remoteService.getRemoteClusterClient(threadPool, a))); + logger.info("new clients are {}", newClients); + followerClients.putAll(newClients); + /* + REMOTE_CLUSTER_SEEDS.getNamespaces(clusterService.getSettings()).stream() + .filter(alias -> !followerClients.containsKey(alias)) + .map(alias -> + followerClients.put(alias, remoteService.getRemoteClusterClient(threadPool, alias))); + */ + } + + + public void syncFollowerSegments(IndexShard indexShard, ReplicationCheckpoint checkpoint) { + initializeFollowerClients(); + List> responseFutures = new ArrayList<>(); + logger.info("Notifying followers to sync segments"); + for (Map.Entry follower : followerClients.entrySet()) { + //SyncFromLeaderRequest request = createSyncRequest(follower.getValue(), indexShard); + logger.info("Notifying follower {}", follower.getKey()); + // invoke replication. + responseFutures.add(follower.getValue().execute(SyncFromLeaderAction.INSTANCE, createSyncRequest(follower.getValue(), indexShard))); + ///WORKING HERE. + } + List responses = + responseFutures.stream().map(ActionFuture::actionGet).collect(Collectors.toList()); + logger.info("[ankikala] all responses {}", responses); + } + + private SyncFromLeaderRequest createSyncRequest(Client followerClient, IndexShard indexShard) { + String indexName = indexShard.indexSettings().getIndex().getName(); + int shardID = indexShard.shardId().getId(); + ClusterState followerClusterState = getFollowerClusterState(followerClient, indexName, true, true); + String followerIndexUUID = followerClusterState.getMetadata().index(indexName).getIndexUUID(); + ShardId followerShardID = new ShardId(indexName, followerIndexUUID, shardID); + logger.info("ankikala routing info: {}", followerClusterState.getRoutingNodes()); + String[] nodes = followerClusterState.getRoutingNodes().shards(sr -> sr.shardId().getId() == shardID).stream().map(sr -> sr.currentNodeId()).toArray(String[] ::new); + logger.info("nodes: {} {}", nodes[0], nodes[1]); + return new SyncFromLeaderRequest(followerShardID, nodes); + } + + private ClusterState getFollowerClusterState(Client followerClient, String indexName, Boolean includeNodes, Boolean includeRoutingTable) { + ClusterStateRequest clusterStateRequest = followerClient.admin().cluster().prepareState() + .clear() + .setIndices(indexName) + .setMetadata(true) + .setNodes(includeNodes) + .setRoutingTable(includeRoutingTable) + .setIndicesOptions(IndicesOptions.strictSingleIndexNoExpandForbidClosed()) + .request(); + return followerClient.admin().cluster().state(clusterStateRequest).actionGet().getState(); + } + + public void createFollowerIndex(CreateIndexRequest request, List followerAliases) { + for(String followerAlias: followerAliases) { + logger.info("[ankikala] Creating the index {} on {}", request.index(), followerAlias); + getRemoteClient(followerAlias).admin().indices().create(request).actionGet(); + logger.info("[ankikala] Created the index {} on {}", request.index(), followerAlias); + } + } + + private Client getRemoteClient(String followerAlias) { + if(!followerClients.containsKey(followerAlias)) { + followerClients.put(followerAlias, remoteService.getRemoteClusterClient(threadPool, followerAlias)); + } + return followerClients.get(followerAlias); + } + + public static final XReplicationLeaderService NO_OP = new XReplicationLeaderService() { + @Override + public void beforeIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { + // NoOp; + } + + @Override + public void shardRoutingChanged(IndexShard indexShard, @Nullable ShardRouting oldRouting, ShardRouting newRouting) { + // noOp; + } + }; + private XReplicationLeaderService() { + this.threadPool = null; + this.recoverySettings = null; + this.transportService = null; + this.clusterService = null; + this.remoteService = null; + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationExecutor.java b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationExecutor.java new file mode 100644 index 0000000000000..84f8e087a1e12 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationExecutor.java @@ -0,0 +1,60 @@ +/* + * 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.xreplication.task.follower; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTaskState; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksExecutor; +import org.opensearch.tasks.TaskId; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.Map; + +public class FollowerReplicationExecutor extends PersistentTasksExecutor { + public static final String NAME = "cluster:indices/admin/replication_follower"; + private final Logger logger = LogManager.getLogger(FollowerReplicationExecutor.class); + private ClusterService clusterService; + private ThreadPool threadPool; + private Client client; + private XReplicationLeaderService leaderService; + public FollowerReplicationExecutor(ClusterService clusterService, ThreadPool threadPool, Client client, XReplicationLeaderService leaderService) { + // TODO: Use a dedicated threadpool for FCR + super(NAME, ThreadPool.Names.WRITE); + this.clusterService = clusterService; + this.threadPool = threadPool; + this.client = client; + this.leaderService = leaderService; + } + + @Override + public void validate(FollowerReplicationParams params, ClusterState clusterState) { + super.validate(params, clusterState); + } + + @Override + protected void nodeOperation(AllocatedPersistentTask task, FollowerReplicationParams params, PersistentTaskState state) { + logger.info("Executing"); + if (task instanceof FollowerReplicationTask) ((FollowerReplicationTask) task).execute(); + else { + task.markAsFailed(new IllegalArgumentException(String.format("Unknown task class %s", task.getClass()))); + } + } + + @Override + protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId, PersistentTasksCustomMetadata.PersistentTask taskInProgress, Map headers) { + return new FollowerReplicationTask(id, type, action, parentTaskId, taskInProgress, headers, leaderService); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationParams.java b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationParams.java new file mode 100644 index 0000000000000..3930b5b68b0aa --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationParams.java @@ -0,0 +1,57 @@ +/* + * 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.xreplication.task.follower; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskParams; +import org.opensearch.xreplication.task.index.IndexReplicationParams; + +import java.io.IOException; + +public class FollowerReplicationParams implements PersistentTaskParams { + private String follower_alias; + + public String getFollower_alias() { + return follower_alias; + } + + public FollowerReplicationParams(String follower_alias) { + this.follower_alias = follower_alias; + } + + public static Writeable.Reader READER = in -> new FollowerReplicationParams(in); + + public FollowerReplicationParams(StreamInput in) throws IOException { + this(in.readString()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("follower_alias", follower_alias).endObject(); + } + + @Override + public String getWriteableName() { + return FollowerReplicationExecutor.NAME; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.V_2_0_0; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(follower_alias); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationState.java b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationState.java new file mode 100644 index 0000000000000..fce1a7e7a36e6 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationState.java @@ -0,0 +1,45 @@ +/* + * 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.xreplication.task.follower; + +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskState; + +import java.io.IOException; + +public class FollowerReplicationState implements PersistentTaskState { + enum State { + NOT_STARTED, BOOTSTRAPPING, SYNCING, PAUSED, FAILED + } + private State state; + + public static Writeable.Reader READER = + in -> new FollowerReplicationState(in.readEnum(State.class)); + + public FollowerReplicationState(State state) { + this.state = state; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("state", state).endObject(); + } + + @Override + public String getWriteableName() { + return FollowerReplicationExecutor.NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(state); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationTask.java b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationTask.java new file mode 100644 index 0000000000000..c8686b6ae83e7 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/follower/FollowerReplicationTask.java @@ -0,0 +1,31 @@ +/* + * 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.xreplication.task.follower; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.tasks.TaskId; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.Map; + +public class FollowerReplicationTask extends AllocatedPersistentTask { + private static final Logger logger = LogManager.getLogger(FollowerReplicationTask.class); + + public FollowerReplicationTask(long id, String type, String action, TaskId parentTaskId, + PersistentTasksCustomMetadata.PersistentTask taskInProgress, Map headers, XReplicationLeaderService leaderService) { + super(id, type, action, String.format("FCR:IndexReplicationTask:{}", taskInProgress.getParams()), parentTaskId, headers); + } + + public void execute() { + logger.info("[ankikala] Follower task is up"); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationExecutor.java b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationExecutor.java new file mode 100644 index 0000000000000..b13100650fbf0 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationExecutor.java @@ -0,0 +1,62 @@ +/* + * 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.xreplication.task.index; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTaskState; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksExecutor; +import org.opensearch.tasks.TaskId; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.Map; + +public class IndexReplicationExecutor extends PersistentTasksExecutor { + public static final String NAME = "cluster:indices/admin/index_task"; + private final Logger logger = LogManager.getLogger(IndexReplicationExecutor.class); + private ClusterService clusterService; + private ThreadPool threadPool; + private Client client; + private XReplicationLeaderService leaderService; + public IndexReplicationExecutor(ClusterService clusterService, ThreadPool threadPool, Client client, + XReplicationLeaderService leaderService) { + // TODO: Use a dedicated threadpool for FCR + super(NAME, ThreadPool.Names.WRITE); + this.clusterService = clusterService; + this.threadPool = threadPool; + this.client = client; + this.leaderService = leaderService; + } + + @Override + public void validate(IndexReplicationParams params, ClusterState clusterState) { + super.validate(params, clusterState); + } + + @Override + protected void nodeOperation(AllocatedPersistentTask task, IndexReplicationParams params, PersistentTaskState state) { + logger.info("Executing"); + if (task instanceof IndexReplicationTask) { + ((IndexReplicationTask)task).execute(); + } else { + task.markAsFailed(new IllegalArgumentException(String.format("Unknown task class %s", task.getClass()))); + } + } + + @Override + protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId, PersistentTasksCustomMetadata.PersistentTask taskInProgress, Map headers) { + return new IndexReplicationTask(id, type, action, parentTaskId, taskInProgress, headers, client, clusterService, leaderService); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationParams.java b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationParams.java new file mode 100644 index 0000000000000..853b8231d826f --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationParams.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.xreplication.task.index; + + +import org.opensearch.Version; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskParams; + +import java.io.IOException; + +public class IndexReplicationParams implements PersistentTaskParams { + private String indexName; + + public String[] getFollowers() { + return followers; + } + + private String[] followers; + + public String getIndexName() { + return indexName; + } + public static Writeable.Reader READER = in -> new IndexReplicationParams(in); + public IndexReplicationParams(String indexName, String[] followers) { + this.indexName = indexName; + this.followers = followers; + } + + public IndexReplicationParams(StreamInput in) throws IOException { + this(in.readString(), in.readStringArray()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("indexName", indexName).field("followers", followers).endObject(); + } + + @Override + public String getWriteableName() { + return IndexReplicationExecutor.NAME; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.V_2_0_0; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(indexName); + out.writeStringArray(followers); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationState.java b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationState.java new file mode 100644 index 0000000000000..0b2a0db6167a4 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationState.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.xreplication.task.index; + +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskState; + +import java.io.IOException; + +public class IndexReplicationState implements PersistentTaskState { + enum State { + NOT_STARTED, STARTING, SYNCING, SKIPPED, FAILED + } + private State state; + + public static Writeable.Reader READER = + in -> new IndexReplicationState(in.readEnum(State.class)); + + public IndexReplicationState(State state) { + this.state = state; + } + + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("state", state).endObject(); + } + + @Override + public String getWriteableName() { + return IndexReplicationExecutor.NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(state); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationTask.java b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationTask.java new file mode 100644 index 0000000000000..111b294b6b51e --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/index/IndexReplicationTask.java @@ -0,0 +1,126 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.xreplication.task.index; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.action.admin.indices.alias.Alias; +import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; +import org.opensearch.action.admin.indices.create.CreateIndexRequest; +import org.opensearch.action.admin.indices.mapping.get.GetMappingsRequest; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.AliasMetadata; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.MappingMetadata; +import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.tasks.TaskId; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.*; + +public class IndexReplicationTask extends AllocatedPersistentTask { + private String indexName; + private Client client; + private ClusterService clusterService; + private XReplicationLeaderService leaderService; + private String[] followers; + private final Logger logger = LogManager.getLogger(IndexReplicationTask.class); + + private static final List blockedSettings = Arrays.asList( + IndexMetadata.INDEX_READ_ONLY_SETTING.getKey(), + IndexMetadata.INDEX_BLOCKS_READ_SETTING.getKey(), + IndexMetadata.INDEX_BLOCKS_WRITE_SETTING.getKey(), + IndexMetadata.INDEX_BLOCKS_METADATA_SETTING.getKey(), + IndexMetadata.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING.getKey(), + EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), + EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.getKey(), + IndexMetadata.SETTING_CREATION_DATE, + IndexMetadata.SETTING_INDEX_PROVIDED_NAME, + IndexMetadata.SETTING_INDEX_UUID, + IndexMetadata.SETTING_CREATION_DATE, + IndexMetadata.SETTING_VERSION_CREATED + ); + + public IndexReplicationTask(long id, String type, String action, TaskId parentTaskId, + PersistentTasksCustomMetadata.PersistentTask taskInProgress, + Map headers, Client client, ClusterService clusterService, + XReplicationLeaderService leaderService) { + super(id, type, action, String.format("FCR:IndexReplicationTask:{}", taskInProgress.getParams().getIndexName()), parentTaskId, headers); + this.indexName = taskInProgress.getParams().getIndexName(); + this.leaderService = leaderService; + this.clusterService = clusterService; + this.client = client; + this.followers = taskInProgress.getParams().getFollowers(); + } + + public void execute() { + //List followerAliases = ReplicationHelper.getAllClusterAliases(clusterService.getSettings()); + logger.info("[ankikala] Creating index {} on these followers: {}", indexName, Arrays.asList(followers)); + createFollowerIndex(); + } + + private void createFollowerIndex() { + ClusterState state = getClusterState(indexName); + Settings leaderIndexSettings = state.metadata().index(indexName).getSettings(); + logger.info("Leader index settings: {}", leaderIndexSettings); + //TODO: Add FCR specific settings if required + Settings.Builder followerSettingsBuilder = Settings.builder().put(leaderIndexSettings); + blockedSettings.forEach(followerSettingsBuilder::remove); + followerSettingsBuilder.put(IndexMetadata.CCR_REPLICATING_FROM_INDEX_SETTING.getKey(), leaderIndexSettings.get(IndexMetadata.SETTING_INDEX_UUID)); + CreateIndexRequest request = new CreateIndexRequest(indexName, followerSettingsBuilder.build()); + logger.info("follower index settings: {}", request.settings()); + + // mappings + request.mapping(getMappings().source().string()); + + //aliases + getAliases().stream().map( + aliasMetadata -> new Alias(aliasMetadata.alias()) + .searchRouting(aliasMetadata.searchRouting()) + .indexRouting(aliasMetadata.indexRouting()) + .writeIndex(aliasMetadata.writeIndex()) + .isHidden(aliasMetadata.isHidden())) + .forEach(request::alias); + + leaderService.createFollowerIndex(request, List.of(followers)); + return; + } + + private List getAliases() { + GetAliasesRequest request = new GetAliasesRequest().indices(indexName); + return client.admin().indices().getAliases(request).actionGet().getAliases().get(indexName); + } + + private MappingMetadata getMappings() { + GetMappingsRequest getMappingsRequest = new GetMappingsRequest(); + getMappingsRequest.indices(indexName); + return client.admin().indices().getMappings(getMappingsRequest).actionGet().getMappings().get(indexName); + + } + + + private ClusterState getClusterState(String... indices) { + ClusterStateRequest request = client.admin().cluster().prepareState().clear() + .setIndices(indices) + .setMetadata(true) + .setNodes(false) // TODO: Check if required + .setRoutingTable(false) // TODO: Check if required + .setIndicesOptions(IndicesOptions.strictSingleIndexNoExpandForbidClosed()).request(); + return client.admin().cluster().state(request).actionGet().getState(); // TODO: add timeouts to the actionGet + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationExecutor.java b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationExecutor.java new file mode 100644 index 0000000000000..a88f98dadac52 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationExecutor.java @@ -0,0 +1,62 @@ +/* + * 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.xreplication.task.supervisor; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTaskState; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.persistent.PersistentTasksExecutor; +import org.opensearch.tasks.TaskId; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.Map; + +public class SupervisorReplicationExecutor extends PersistentTasksExecutor { + public static final String NAME = "cluster:indices/admin/fcr_supervisor"; + private final Logger logger = LogManager.getLogger(SupervisorReplicationExecutor.class); + private ClusterService clusterService; + private ThreadPool threadPool; + private XReplicationLeaderService leaderService; + private Client client; + public SupervisorReplicationExecutor(ClusterService clusterService, ThreadPool threadPool, Client client, XReplicationLeaderService leaderService) { + // TODO: Use a dedicated threadpool for FCR + super(NAME, ThreadPool.Names.WRITE); + this.clusterService = clusterService; + this.threadPool = threadPool; + this.client = client; + this.leaderService = leaderService; + } + + @Override + public void validate(SupervisorReplicationParams params, ClusterState clusterState) { + super.validate(params, clusterState); + } + + @Override + protected void nodeOperation(AllocatedPersistentTask task, SupervisorReplicationParams params, PersistentTaskState state) { + logger.info("Executing"); + if (task instanceof SupervisorReplicationTask) { + ((SupervisorReplicationTask)task).execute(); + } else { + task.markAsFailed(new IllegalArgumentException(String.format("Unknown task class %s", task.getClass()))); + } + } + + @Override + protected AllocatedPersistentTask createTask(long id, String type, String action, TaskId parentTaskId, PersistentTasksCustomMetadata.PersistentTask taskInProgress, Map headers) { + return new SupervisorReplicationTask(id, type, action, parentTaskId, taskInProgress, headers, clusterService, client, leaderService); + } + +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationParams.java b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationParams.java new file mode 100644 index 0000000000000..c3b611c53389f --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationParams.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.xreplication.task.supervisor; + +import org.opensearch.Version; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskParams; + +import java.io.IOException; + +public class SupervisorReplicationParams implements PersistentTaskParams { + private String[] followerAliases; + + public String[] getFollowerAliases() { + return followerAliases; + } + + public SupervisorReplicationParams(String[] followerAliases) { + this.followerAliases = followerAliases; + } + + public static Writeable.Reader READER = in -> new SupervisorReplicationParams(in); + + public SupervisorReplicationParams(StreamInput in) throws IOException { + this(in.readStringArray()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("followerAliases", followerAliases).endObject(); + } + + @Override + public String getWriteableName() { + return SupervisorReplicationExecutor.NAME; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.V_2_0_0; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeStringArray(followerAliases); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationState.java b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationState.java new file mode 100644 index 0000000000000..2ccef4394b464 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationState.java @@ -0,0 +1,45 @@ +/* + * 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.xreplication.task.supervisor; + +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.persistent.PersistentTaskState; + +import java.io.IOException; + +public class SupervisorReplicationState implements PersistentTaskState { + enum State { + NOT_STARTED, SYNCING, PAUSED, FAILED + } + private State state; + + public static Writeable.Reader READER = + in -> new SupervisorReplicationState(in.readEnum(State.class)); + + public SupervisorReplicationState(State state) { + this.state = state; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject().field("state", state).endObject(); + } + + @Override + public String getWriteableName() { + return SupervisorReplicationExecutor.NAME; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeEnum(state); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationTask.java b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationTask.java new file mode 100644 index 0000000000000..8109f34a995a0 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/task/supervisor/SupervisorReplicationTask.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.xreplication.task.supervisor; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.client.Client; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.persistent.AllocatedPersistentTask; +import org.opensearch.persistent.PersistentTasksCustomMetadata; +import org.opensearch.tasks.TaskId; +import org.opensearch.xreplication.actions.followers.StartCCRFollowerTaskAction; +import org.opensearch.xreplication.actions.followers.StartCCRFollowerTaskRequest; +import org.opensearch.xreplication.actions.index.StartCCRIndexTaskAction; +import org.opensearch.xreplication.actions.index.StartCCRIndexTaskRequest; +import org.opensearch.xreplication.services.XReplicationLeaderService; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public class SupervisorReplicationTask extends AllocatedPersistentTask { + private ClusterService clusterService; + private Client client; + private final Logger logger = LogManager.getLogger(SupervisorReplicationTask.class); + String[] followers; + public SupervisorReplicationTask(long id, String type, String action, TaskId parentTaskId, + PersistentTasksCustomMetadata.PersistentTask taskInProgress, + Map headers, ClusterService clusterService, Client client, XReplicationLeaderService leaderService) { + super(id, type, action, String.format("FCR:IndexReplicationTask: %s", taskInProgress.getParams().getFollowerAliases().toString()), parentTaskId, headers); + this.client = client; + this.clusterService = clusterService; + this.followers = taskInProgress.getParams().getFollowerAliases(); + } + + public void execute() { + logger.info("[ankikala] Supervisor task is up, followers: {}", Arrays.toString(followers)); + createFollowerTasks(followers); + + String[] indices = Arrays.stream(client.admin().indices().getIndex(new GetIndexRequest()).actionGet().indices()) + .filter(i -> !i.startsWith(".")).collect(Collectors.toList()).toArray(String[]::new); + createIndexTasks(indices); + } + + private void createIndexTasks(String[] indices) { + logger.info("creating index tasks {}: {}", Arrays.toString(indices), Arrays.toString(followers)); + StartCCRIndexTaskRequest startCCRIndexTaskRequest = new StartCCRIndexTaskRequest(indices, followers); + client.execute(StartCCRIndexTaskAction.INSTANCE, startCCRIndexTaskRequest); + } + + private void createFollowerTasks(String[] followers) { + logger.info("creating followers {}", Arrays.toString(followers)); + StartCCRFollowerTaskRequest startCCRFollowerTaskRequest = new StartCCRFollowerTaskRequest(followers); + client.execute(StartCCRFollowerTaskAction.INSTANCE, startCCRFollowerTaskRequest); + } +} diff --git a/server/src/main/java/org/opensearch/xreplication/utils/ReplicationHelper.java b/server/src/main/java/org/opensearch/xreplication/utils/ReplicationHelper.java new file mode 100644 index 0000000000000..672e32fd1e424 --- /dev/null +++ b/server/src/main/java/org/opensearch/xreplication/utils/ReplicationHelper.java @@ -0,0 +1,39 @@ +/* + * 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.xreplication.utils; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.settings.Settings; + +import java.util.List; +import java.util.stream.Collectors; + +import static org.opensearch.transport.SniffConnectionStrategy.REMOTE_CLUSTER_SEEDS; + +public class ReplicationHelper { + private static final Logger logger = LogManager.getLogger(ReplicationHelper.class); + public static List getAllClusterAliases(Settings settings) { + List aliases = REMOTE_CLUSTER_SEEDS.getAllConcreteSettings(settings).map(s -> s.getKey()).collect(Collectors.toList()); + logger.info("[ankikala]: New Follower aliases {}", aliases.toString()); + getAllClusterAliases3(settings); + return aliases; + } + + private static void getAllClusterAliases3(Settings settings) { + logger.info("[ankikala] manual stream", settings.keySet().stream().filter(s -> s.startsWith("cluster.remote")).collect(Collectors.toList())); + logger.info("settings.keySet(): {}", settings.keySet().toString()); + } + + public static List getAllClusterAliases2(Settings settings) { + List aliases = REMOTE_CLUSTER_SEEDS.getNamespaces(settings).stream().collect(Collectors.toList()); + logger.info("[ankikala]: Follower aliases {}", aliases.toString()); + return aliases; + } +} diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index c8c5ff1f58f5e..4667023d9dff8 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -601,7 +601,7 @@ protected IndexShard newShard( translogFactorySupplier, checkpointPublisher, remoteStore, - new RemoteStoreSegmentUploadNotificationPublisher(null) + RemoteStoreSegmentUploadNotificationPublisher.EMPTY ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); success = true;