From aa13d96591b77c998c7e21228b9a7c19e1023ea9 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Mon, 12 May 2025 10:42:05 +0800 Subject: [PATCH 01/23] support local merged segment warmer Signed-off-by: guojialiang --- .../replication/MergedSegmentWarmerIT.java | 199 ++++++++++++++ .../common/settings/ClusterSettings.java | 2 + .../engine/LocalMergedSegmentWarmer.java | 80 +++++- .../engine/MergedSegmentWarmerFactory.java | 2 +- .../opensearch/index/shard/IndexShard.java | 32 +++ .../org/opensearch/index/store/Store.java | 35 +++ .../indices/recovery/RecoverySettings.java | 73 +++++ .../MergedSegmentReplicationTarget.java | 99 +++++++ .../PrimaryShardReplicationSource.java | 34 +++ .../replication/SegmentReplicationSource.java | 19 ++ .../SegmentReplicationSourceService.java | 103 ++++++++ .../replication/SegmentReplicationTarget.java | 10 +- .../SegmentReplicationTargetService.java | 171 ++++++++++++ .../replication/SegmentReplicator.java | 84 +++++- .../PublishMergedSegmentRequest.java | 91 +++++++ .../ReplicationSegmentCheckpoint.java | 125 +++++++++ .../common/ReplicationCollection.java | 10 + .../index/shard/RemoteIndexShardTests.java | 10 + .../SegmentReplicationIndexShardTests.java | 57 ++++ ...licationWithNodeToNodeIndexShardTests.java | 1 + .../RecoverySettingsDynamicUpdateTests.java | 42 +++ .../MergedSegmentReplicationTargetTests.java | 250 ++++++++++++++++++ .../PrimaryShardReplicationSourceTests.java | 58 ++++ .../SegmentReplicationSourceServiceTests.java | 148 ++++++++++- .../SegmentReplicationTargetServiceTests.java | 84 ++++++ .../recovery/ReplicationCollectionTests.java | 1 + ...enSearchIndexLevelReplicationTestCase.java | 2 + .../index/shard/IndexShardTestCase.java | 136 +++++++++- 28 files changed, 1940 insertions(+), 18 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/indices/replication/MergedSegmentWarmerIT.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java create mode 100644 server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/MergedSegmentWarmerIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/MergedSegmentWarmerIT.java new file mode 100644 index 0000000000000..6e0e4cbe86926 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/MergedSegmentWarmerIT.java @@ -0,0 +1,199 @@ +/* + * 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.Logger; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; +import org.opensearch.action.admin.indices.segments.IndexShardSegments; +import org.opensearch.action.admin.indices.segments.IndicesSegmentResponse; +import org.opensearch.action.admin.indices.segments.ShardSegments; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.util.set.Sets; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.TieredMergePolicyProvider; +import org.opensearch.index.engine.Segment; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.transport.TransportService; + +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +/** + * This class runs Segment Replication Integ test suite with merged segment warmer enabled. + */ +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class MergedSegmentWarmerIT extends SegmentReplicationIT { + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).build(); + } + + @Override + protected Settings featureFlagSettings() { + Settings.Builder featureSettings = Settings.builder(); + featureSettings.put(FeatureFlags.MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG, true); + return featureSettings.build(); + } + + public void testMergeSegmentWarmer() throws Exception { + final String primaryNode = internalCluster().startDataOnlyNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); + createIndex(INDEX_NAME); + ensureGreen(INDEX_NAME); + + for (int i = 0; i < 30; i++) { + client().prepareIndex(INDEX_NAME) + .setId(String.valueOf(i)) + .setSource("foo" + i, "bar" + i) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + } + + waitForSearchableDocs(30, primaryNode, replicaNode); + + MockTransportService primaryTransportService = ((MockTransportService) internalCluster().getInstance( + TransportService.class, + primaryNode + )); + + primaryTransportService.addRequestHandlingBehavior( + SegmentReplicationSourceService.Actions.GET_SEGMENT_FILES, + (handler, request, channel, task) -> { + logger.info( + "replicationId {}, get segment files {}", + ((GetSegmentFilesRequest) request).getReplicationId(), + ((GetSegmentFilesRequest) request).getFilesToFetch().stream().map(StoreFileMetadata::name).collect(Collectors.toList()) + ); + // After the pre-copy merged segment is complete, the merged segment files is to reuse, so the files to fetch is empty. + assertEquals(0, ((GetSegmentFilesRequest) request).getFilesToFetch().size()); + handler.messageReceived(request, channel, task); + } + ); + + client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(2)); + + waitForSegmentCount(INDEX_NAME, 2, logger); + primaryTransportService.clearAllRules(); + } + + public void testConcurrentMergeSegmentWarmer() throws Exception { + final String primaryNode = internalCluster().startDataOnlyNode(); + createIndex( + INDEX_NAME, + Settings.builder() + .put(indexSettings()) + .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), 5) + .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), 5) + .put(IndexSettings.INDEX_MERGE_ON_FLUSH_ENABLED.getKey(), false) + .build() + ); + ensureYellowAndNoInitializingShards(INDEX_NAME); + final String replicaNode = internalCluster().startDataOnlyNode(); + ensureGreen(INDEX_NAME); + + // ensure pre-copy merge segment concurrent execution + AtomicInteger getMergeSegmentFilesActionCount = new AtomicInteger(0); + MockTransportService primaryTransportService = ((MockTransportService) internalCluster().getInstance( + TransportService.class, + primaryNode + )); + + CountDownLatch blockFileCopy = new CountDownLatch(1); + primaryTransportService.addRequestHandlingBehavior( + SegmentReplicationSourceService.Actions.GET_MERGED_SEGMENT_FILES, + (handler, request, channel, task) -> { + logger.info( + "replicationId {}, get merge segment files {}", + ((GetSegmentFilesRequest) request).getReplicationId(), + ((GetSegmentFilesRequest) request).getFilesToFetch().stream().map(StoreFileMetadata::name).collect(Collectors.toList()) + ); + getMergeSegmentFilesActionCount.incrementAndGet(); + if (getMergeSegmentFilesActionCount.get() > 2) { + blockFileCopy.countDown(); + } + handler.messageReceived(request, channel, task); + } + ); + + primaryTransportService.addSendBehavior( + internalCluster().getInstance(TransportService.class, replicaNode), + (connection, requestId, action, request, options) -> { + if (action.equals(SegmentReplicationTargetService.Actions.MERGED_SEGMENT_FILE_CHUNK)) { + try { + blockFileCopy.await(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + connection.sendRequest(requestId, action, request, options); + } + ); + + for (int i = 0; i < 30; i++) { + client().prepareIndex(INDEX_NAME) + .setId(String.valueOf(i)) + .setSource("foo" + i, "bar" + i) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + } + + client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(2)); + + waitForSegmentCount(INDEX_NAME, 2, logger); + primaryTransportService.clearAllRules(); + } + + public void testMergeSegmentWarmerWithInactiveReplica() throws Exception { + internalCluster().startDataOnlyNode(); + createIndex(INDEX_NAME); + ensureYellowAndNoInitializingShards(INDEX_NAME); + + for (int i = 0; i < 30; i++) { + client().prepareIndex(INDEX_NAME) + .setId(String.valueOf(i)) + .setSource("foo" + i, "bar" + i) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + } + + client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(1)).get(); + final IndicesSegmentResponse response = client().admin().indices().prepareSegments(INDEX_NAME).get(); + assertEquals(1, response.getIndices().get(INDEX_NAME).getShards().values().size()); + } + + public static void waitForSegmentCount(String indexName, int segmentCount, Logger logger) throws Exception { + assertBusy(() -> { + Set primarySegments = Sets.newHashSet(); + Set replicaSegments = Sets.newHashSet(); + final IndicesSegmentResponse response = client().admin().indices().prepareSegments(indexName).get(); + for (IndexShardSegments indexShardSegments : response.getIndices().get(indexName).getShards().values()) { + for (ShardSegments shardSegment : indexShardSegments.getShards()) { + for (Segment segment : shardSegment.getSegments()) { + if (shardSegment.getShardRouting().primary()) { + primarySegments.add(segment.getName()); + } else { + replicaSegments.add(segment.getName()); + } + } + } + } + logger.info("primary segments: {}, replica segments: {}", primarySegments, replicaSegments); + assertEquals(segmentCount, primarySegments.size()); + assertEquals(segmentCount, replicaSegments.size()); + }, 1, TimeUnit.MINUTES); + } +} diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 20a7a907c7511..25523bab7d0a7 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -314,6 +314,8 @@ public void apply(Settings value, Settings current, Settings previous) { ShardLimitValidator.SETTING_CLUSTER_IGNORE_DOT_INDEXES, RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, RecoverySettings.INDICES_REPLICATION_MAX_BYTES_PER_SEC_SETTING, + RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING, + RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, diff --git a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java index 4eabd9399eadf..b64d178f4d667 100644 --- a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java +++ b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java @@ -32,13 +32,33 @@ package org.opensearch.index.engine; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentReader; +import org.opensearch.action.ActionListenerResponseHandler; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.transport.TransportResponse; +import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; +import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; +import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; /** * Implementation of a {@link IndexWriter.IndexReaderWarmer} when local on-disk segment replication is enabled. @@ -46,18 +66,74 @@ * @opensearch.internal */ public class LocalMergedSegmentWarmer implements IndexWriter.IndexReaderWarmer { + private static final Logger logger = LogManager.getLogger(LocalMergedSegmentWarmer.class); private final TransportService transportService; private final RecoverySettings recoverySettings; private final ClusterService clusterService; + private final IndexShard indexShard; - public LocalMergedSegmentWarmer(TransportService transportService, RecoverySettings recoverySettings, ClusterService clusterService) { + public LocalMergedSegmentWarmer( + TransportService transportService, + RecoverySettings recoverySettings, + ClusterService clusterService, + IndexShard indexShard + ) { this.transportService = transportService; this.recoverySettings = recoverySettings; this.clusterService = clusterService; + this.indexShard = indexShard; } @Override public void warm(LeafReader leafReader) throws IOException { - // TODO: node-node segment replication merged segment warmer + SegmentCommitInfo segmentCommitInfo = ((SegmentReader) leafReader).getSegmentInfo(); + ReplicationSegmentCheckpoint mergedSegment = indexShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); + PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(mergedSegment); + + DiscoveryNodes nodes = clusterService.state().nodes(); + List replicaShards = indexShard.getReplicationGroup().getRoutingTable().replicaShards(); + List activeReplicaNodes = replicaShards.stream() + .filter(ShardRouting::active) + .map(s -> nodes.get(s.currentNodeId())) + .toList(); + + if (activeReplicaNodes.isEmpty()) { + logger.trace("There are no active replicas, skip pre copy merged segment [{}]", segmentCommitInfo.info.name); + return; + } + + CountDownLatch countDownLatch = new CountDownLatch(activeReplicaNodes.size()); + AtomicInteger successfulCount = new AtomicInteger(0); + AtomicInteger failureCount = new AtomicInteger(0); + for (DiscoveryNode replicaNode : activeReplicaNodes) { + ActionListener listener = ActionListener.wrap(r -> { + successfulCount.incrementAndGet(); + countDownLatch.countDown(); + }, e -> { + failureCount.incrementAndGet(); + countDownLatch.countDown(); + }); + transportService.sendRequest( + replicaNode, + SegmentReplicationTargetService.Actions.PUBLISH_MERGED_SEGMENT, + request, + new ActionListenerResponseHandler<>(listener, (in) -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC) + ); + } + try { + countDownLatch.await(recoverySettings.getMergedSegmentReplicationTimeout().seconds(), TimeUnit.SECONDS); + logger.trace( + "pre copy merged segment [{}] to [{}] active replicas, [{}] successful, [{}] failed", + segmentCommitInfo.info.name, + activeReplicaNodes.size(), + successfulCount, + failureCount + ); + } catch (InterruptedException e) { + logger.warn( + () -> new ParameterizedMessage("Interrupted while waiting for pre copy merged segment [{}]", segmentCommitInfo.info.name), + e + ); + } } } diff --git a/server/src/main/java/org/opensearch/index/engine/MergedSegmentWarmerFactory.java b/server/src/main/java/org/opensearch/index/engine/MergedSegmentWarmerFactory.java index fd6a9851167fc..eda93d22d9c3b 100644 --- a/server/src/main/java/org/opensearch/index/engine/MergedSegmentWarmerFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/MergedSegmentWarmerFactory.java @@ -61,7 +61,7 @@ public IndexWriter.IndexReaderWarmer get(IndexShard shard) { if (shard.indexSettings().isAssignedOnRemoteNode()) { return new RemoteStoreMergedSegmentWarmer(transportService, recoverySettings, clusterService); } else if (shard.indexSettings().isSegRepLocalEnabled()) { - return new LocalMergedSegmentWarmer(transportService, recoverySettings, clusterService); + return new LocalMergedSegmentWarmer(transportService, recoverySettings, clusterService, shard); } else if (shard.indexSettings().isDocumentReplication()) { // MergedSegmentWarmerFactory#get is called when IndexShard is initialized. In scenario document replication, // IndexWriter.IndexReaderWarmer should be null. 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 52eabcfb9a486..1e0d1bad0e6ce 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -41,6 +41,7 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.Term; import org.apache.lucene.search.Query; @@ -198,6 +199,7 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationTimer; import org.opensearch.repositories.RepositoriesService; @@ -1845,6 +1847,26 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th return checkpoint; } + /** + * Compute {@link ReplicationSegmentCheckpoint} from a SegmentCommitInfo. + * This function fetches a metadata snapshot from the store that comes with an IO cost. + * + * @param segmentCommitInfo {@link SegmentCommitInfo} segmentCommitInfo to use to compute. + * @return {@link ReplicationSegmentCheckpoint} Checkpoint computed from the segmentCommitInfo. + * @throws IOException When there is an error computing segment metadata from the store. + */ + public ReplicationSegmentCheckpoint computeReplicationSegmentCheckpoint(SegmentCommitInfo segmentCommitInfo) throws IOException { + Map segmentMetadataMap = store.getSegmentMetadataMap(segmentCommitInfo); + return new ReplicationSegmentCheckpoint( + shardId, + getOperationPrimaryTerm(), + segmentMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), + getEngine().config().getCodec().getName(), + segmentMetadataMap, + segmentCommitInfo.info.name + ); + } + /** * Checks if this target shard should start a round of segment replication. * @return - True if the shard is able to perform segment replication. @@ -1909,6 +1931,16 @@ public final boolean shouldProcessCheckpoint(ReplicationCheckpoint requestCheckp return true; } + /** + * Checks if segment checkpoint should be processed + * + * @param requestCheckpoint received segment checkpoint that is checked for processing + * @return true if segment checkpoint should be processed + */ + public final boolean shouldProcessMergedSegmentCheckpoint(ReplicationCheckpoint requestCheckpoint) { + return isSegmentReplicationAllowed() && requestCheckpoint.getPrimaryTerm() >= getOperationPrimaryTerm(); + } + /** * gets a {@link Store.MetadataSnapshot} for the current directory. This method is safe to call in all lifecycle of the index shard, * without having to worry about the current state of the engine and concurrent flushes. 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 48ed4d9e75dcb..5aec02b517ee1 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -389,6 +389,21 @@ public Map getSegmentMetadataMap(SegmentInfos segment } } + /** + * Segment Replication method - Fetch a map of StoreFileMetadata for segmentCommitInfo. + * @param segmentCommitInfo {@link SegmentCommitInfo} from which to compute metadata. + * @return {@link Map} map file name to {@link StoreFileMetadata}. + */ + public Map getSegmentMetadataMap(SegmentCommitInfo segmentCommitInfo) throws IOException { + failIfCorrupted(); + try { + return loadMetadata(segmentCommitInfo, directory, logger); + } catch (NoSuchFileException | CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + markStoreCorrupted(ex); + throw ex; + } + } + /** * Segment Replication method * Returns a diff between the Maps of StoreFileMetadata that can be used for getting list of files to copy over to a replica for segment replication. The returned diff will hold a list of files that are: @@ -1193,6 +1208,26 @@ static LoadedMetadata loadMetadata(SegmentInfos segmentInfos, Directory director return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserDataBuilder), numDocs); } + static Map loadMetadata(SegmentCommitInfo info, Directory directory, Logger logger) throws IOException { + Map builder = new HashMap<>(); + final Version version = info.info.getVersion(); + if (version == null) { + // version is written since 3.1+: we should have already hit IndexFormatTooOld. + throw new IllegalArgumentException("expected valid version value: " + info.info.toString()); + } + for (String file : info.files()) { + checksumFromLuceneFile( + directory, + file, + builder, + logger, + version, + SEGMENT_INFO_EXTENSION.equals(IndexFileNames.getExtension(file)) + ); + } + return unmodifiableMap(builder); + } + private static void checksumFromLuceneFile( Directory directory, String file, diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java index 7d9f0c0762e46..4c7dd361c8ffb 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java @@ -75,6 +75,27 @@ public class RecoverySettings { Property.NodeScope ); + /** + * Individual speed setting for merged segment replication, default -1B to reuse the setting of recovery. + */ + public static final Setting INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING = Setting.byteSizeSetting( + "indices.merged_segment_replication.max_bytes_per_sec", + new ByteSizeValue(-1), + Property.Dynamic, + Property.NodeScope + ); + + /** + * Control the maximum waiting time for replicate merged segment to the replica + */ + public static final Setting INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING = Setting.timeSetting( + "indices.merged_segment_replication_timeout", + TimeValue.timeValueMinutes(15), + TimeValue.timeValueMinutes(0), + Property.Dynamic, + Property.NodeScope + ); + /** * Controls the maximum number of file chunk requests that can be sent concurrently from the source node to the target node. */ @@ -190,11 +211,13 @@ public class RecoverySettings { private volatile ByteSizeValue recoveryMaxBytesPerSec; private volatile ByteSizeValue replicationMaxBytesPerSec; + private volatile ByteSizeValue mergedSegmentReplicationMaxBytesPerSec; private volatile int maxConcurrentFileChunks; private volatile int maxConcurrentOperations; private volatile int maxConcurrentRemoteStoreStreams; private volatile SimpleRateLimiter recoveryRateLimiter; private volatile SimpleRateLimiter replicationRateLimiter; + private volatile SimpleRateLimiter mergedSegmentReplicationRateLimiter; private volatile TimeValue retryDelayStateSync; private volatile TimeValue retryDelayNetwork; private volatile TimeValue activityTimeout; @@ -204,6 +227,7 @@ public class RecoverySettings { private volatile ByteSizeValue chunkSize; private volatile TimeValue internalRemoteUploadTimeout; + private volatile TimeValue mergedSegmentReplicationTimeout; public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { this.retryDelayStateSync = INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.get(settings); @@ -226,7 +250,10 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { recoveryRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); } this.replicationMaxBytesPerSec = INDICES_REPLICATION_MAX_BYTES_PER_SEC_SETTING.get(settings); + this.mergedSegmentReplicationMaxBytesPerSec = INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING.get(settings); + this.mergedSegmentReplicationTimeout = INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING.get(settings); updateReplicationRateLimiter(); + updateMergedSegmentReplicationRateLimiter(); logger.debug("using recovery max_bytes_per_sec[{}]", recoveryMaxBytesPerSec); this.internalRemoteUploadTimeout = INDICES_INTERNAL_REMOTE_UPLOAD_TIMEOUT.get(settings); @@ -234,6 +261,14 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, this::setRecoveryMaxBytesPerSec); clusterSettings.addSettingsUpdateConsumer(INDICES_REPLICATION_MAX_BYTES_PER_SEC_SETTING, this::setReplicationMaxBytesPerSec); + clusterSettings.addSettingsUpdateConsumer( + INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING, + this::setMergedSegmentReplicationMaxBytesPerSec + ); + clusterSettings.addSettingsUpdateConsumer( + INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING, + this::setMergedSegmentReplicationTimeout + ); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, this::setMaxConcurrentFileChunks); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_CONCURRENT_OPERATIONS_SETTING, this::setMaxConcurrentOperations); clusterSettings.addSettingsUpdateConsumer( @@ -264,6 +299,10 @@ public RateLimiter replicationRateLimiter() { return replicationRateLimiter; } + public SimpleRateLimiter mergedSegmentReplicationRateLimiter() { + return mergedSegmentReplicationRateLimiter; + } + public TimeValue retryDelayNetwork() { return retryDelayNetwork; } @@ -338,6 +377,7 @@ private void setRecoveryMaxBytesPerSec(ByteSizeValue recoveryMaxBytesPerSec) { recoveryRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); } if (replicationMaxBytesPerSec.getBytes() < 0) updateReplicationRateLimiter(); + if (mergedSegmentReplicationMaxBytesPerSec.getBytes() < 0) updateMergedSegmentReplicationRateLimiter(); } private void setReplicationMaxBytesPerSec(ByteSizeValue replicationMaxBytesPerSec) { @@ -365,6 +405,39 @@ private void updateReplicationRateLimiter() { } } + public TimeValue getMergedSegmentReplicationTimeout() { + return mergedSegmentReplicationTimeout; + } + + private void setMergedSegmentReplicationMaxBytesPerSec(ByteSizeValue mergedSegmentReplicationMaxBytesPerSec) { + this.mergedSegmentReplicationMaxBytesPerSec = mergedSegmentReplicationMaxBytesPerSec; + updateMergedSegmentReplicationRateLimiter(); + } + + public void setMergedSegmentReplicationTimeout(TimeValue mergedSegmentReplicationTimeout) { + this.mergedSegmentReplicationTimeout = mergedSegmentReplicationTimeout; + } + + private void updateMergedSegmentReplicationRateLimiter() { + if (mergedSegmentReplicationMaxBytesPerSec.getBytes() >= 0) { + if (mergedSegmentReplicationMaxBytesPerSec.getBytes() == 0) { + mergedSegmentReplicationRateLimiter = null; + } else if (mergedSegmentReplicationRateLimiter != null) { + mergedSegmentReplicationRateLimiter.setMBPerSec(mergedSegmentReplicationMaxBytesPerSec.getMbFrac()); + } else { + mergedSegmentReplicationRateLimiter = new SimpleRateLimiter(mergedSegmentReplicationMaxBytesPerSec.getMbFrac()); + } + } else { // when mergeReplicationMaxBytesPerSec = -1B, use setting of recovery + if (recoveryMaxBytesPerSec.getBytes() <= 0) { + mergedSegmentReplicationRateLimiter = null; + } else if (mergedSegmentReplicationRateLimiter != null) { + mergedSegmentReplicationRateLimiter.setMBPerSec(recoveryMaxBytesPerSec.getMbFrac()); + } else { + mergedSegmentReplicationRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); + } + } + } + public int getMaxConcurrentFileChunks() { return maxConcurrentFileChunks; } diff --git a/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java new file mode 100644 index 0000000000000..08fdd00ec030d --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java @@ -0,0 +1,99 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication; + +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.action.StepListener; +import org.opensearch.common.UUIDs; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationListener; + +import java.util.List; +import java.util.function.BiConsumer; + +/** + * Represents the target of a merged segment replication event. + * + * @opensearch.internal + */ +public class MergedSegmentReplicationTarget extends SegmentReplicationTarget { + public final static String MERGE_REPLICATION_PREFIX = "merge."; + + public MergedSegmentReplicationTarget( + IndexShard indexShard, + ReplicationCheckpoint checkpoint, + SegmentReplicationSource source, + ReplicationListener listener + ) { + super(indexShard, checkpoint, source, listener); + } + + @Override + protected String getPrefix() { + return MERGE_REPLICATION_PREFIX + UUIDs.randomBase64UUID() + "."; + } + + @Override + public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { + state.setStage(SegmentReplicationState.Stage.REPLICATING); + cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { + throw new CancellableThreads.ExecutionCancelledException("merge replication was canceled reason [" + reason + "]"); + }); + + final StepListener getFilesListener = new StepListener<>(); + + logger.trace(new ParameterizedMessage("Starting Merge Replication Target: {}", description())); + + state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); + List filesToFetch; + try { + filesToFetch = getFiles(new CheckpointInfoResponse(checkpoint, checkpoint.getMetadataMap(), null)); + } catch (Exception e) { + listener.onFailure(e); + return; + } + state.setStage(SegmentReplicationState.Stage.GET_FILES); + cancellableThreads.checkForCancel(); + source.getMergedSegmentFiles(getId(), checkpoint, filesToFetch, indexShard, this::updateFileRecoveryBytes, getFilesListener); + getFilesListener.whenComplete(response -> { + state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + cancellableThreads.checkForCancel(); + multiFileWriter.renameAllTempFiles(); + listener.onResponse(null); + }, listener::onFailure); + } +} 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 af37594f88fee..eada720bcb959 100644 --- a/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/PrimaryShardReplicationSource.java @@ -23,6 +23,7 @@ import java.util.function.BiConsumer; import static org.opensearch.indices.replication.SegmentReplicationSourceService.Actions.GET_CHECKPOINT_INFO; +import static org.opensearch.indices.replication.SegmentReplicationSourceService.Actions.GET_MERGED_SEGMENT_FILES; import static org.opensearch.indices.replication.SegmentReplicationSourceService.Actions.GET_SEGMENT_FILES; /** @@ -101,6 +102,39 @@ public void getSegmentFiles( ); } + @Override + public void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + replicationId, + targetAllocationId, + targetNode, + filesToFetch, + new ReplicationCheckpoint( + checkpoint.getShardId(), + checkpoint.getPrimaryTerm(), + checkpoint.getSegmentsGen(), + checkpoint.getSegmentInfosVersion(), + checkpoint.getLength(), + checkpoint.getCodec(), + checkpoint.getMetadataMap() + ) + ); + transportService.sendRequest( + sourceNode, + GET_MERGED_SEGMENT_FILES, + request, + TransportRequestOptions.builder().withTimeout(recoverySettings.getMergedSegmentReplicationTimeout()).build(), + new ActionListenerResponseHandler<>(listener, GetSegmentFilesResponse::new, ThreadPool.Names.GENERIC) + ); + } + @Override public String getDescription() { return sourceNode.getName(); 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 5341f9507bef4..fa8968030db96 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSource.java @@ -57,6 +57,25 @@ void getSegmentFiles( ActionListener listener ); + /** + * Fetch the merged segment files. Passes a listener that completes when files are stored locally. + * + * @param replicationId 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 indexShard {@link IndexShard} Reference to the IndexShard. + * @param fileProgressTracker {@link BiConsumer} A consumer that updates the replication progress for shard files. + * @param listener {@link ActionListener} Listener that completes with the list of files copied. + */ + default void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) {}; + /** * Get the source description */ diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index 21fd066b8be2f..db033fa3b4870 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -21,12 +21,20 @@ import org.opensearch.common.Nullable; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; +import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.IndexShardNotStartedException; +import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.MultiChunkTransfer; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RetryableTransportClient; import org.opensearch.indices.replication.common.ReplicationTimer; @@ -64,6 +72,7 @@ public static class Actions { public static final String GET_CHECKPOINT_INFO = "internal:index/shard/replication/get_checkpoint_info"; public static final String GET_SEGMENT_FILES = "internal:index/shard/replication/get_segment_files"; public static final String UPDATE_VISIBLE_CHECKPOINT = "internal:index/shard/replication/update_visible_checkpoint"; + public static final String GET_MERGED_SEGMENT_FILES = "internal:index/shard/replication/get_merged_segment_files"; } private final OngoingSegmentReplications ongoingSegmentReplications; @@ -96,6 +105,12 @@ protected SegmentReplicationSourceService( UpdateVisibleCheckpointRequest::new, new UpdateVisibleCheckpointRequestHandler() ); + transportService.registerRequestHandler( + Actions.GET_MERGED_SEGMENT_FILES, + ThreadPool.Names.GENERIC, + GetSegmentFilesRequest::new, + new GetMergedSegmentFilesRequestHandler() + ); } public SegmentReplicationSourceService( @@ -165,6 +180,94 @@ public void messageReceived(UpdateVisibleCheckpointRequest request, TransportCha } } + private class GetMergedSegmentFilesRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(GetSegmentFilesRequest request, TransportChannel channel, Task task) throws Exception { + ActionListener listener = new ChannelActionListener<>( + channel, + Actions.GET_MERGED_SEGMENT_FILES, + request + ); + final ShardId shardId = request.getCheckpoint().getShardId(); + IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); + if (indexService == null) { + listener.onFailure(new IndexNotFoundException(shardId.getIndexName())); + return; + } + IndexShard indexShard = indexService.getShard(shardId.id()); + if (indexShard == null || indexShard.state().equals(IndexShardState.CLOSED)) { + listener.onFailure(new IndexShardClosedException(shardId)); + return; + } + if (indexShard.state().equals(IndexShardState.STARTED) == false) { + listener.onFailure(new IndexShardNotStartedException(shardId, indexShard.state())); + return; + } + if (indexShard.routingEntry().primary() == false || indexShard.isPrimaryMode() == false) { + listener.onFailure(new IllegalArgumentException(String.format("%s is not primary", shardId))); + return; + } + if (indexShard.getOperationPrimaryTerm() > request.getCheckpoint().getPrimaryTerm()) { + listener.onFailure( + new IllegalArgumentException( + String.format( + "request primary term %d is lower than %d", + request.getCheckpoint().getPrimaryTerm(), + indexShard.getOperationPrimaryTerm() + ) + ) + ); + return; + } + + RemoteSegmentFileChunkWriter mergedSegmentFileChunkWriter = new RemoteSegmentFileChunkWriter( + request.getReplicationId(), + indexShard.getRecoverySettings(), + new RetryableTransportClient( + transportService, + request.getTargetNode(), + indexShard.getRecoverySettings().getMergedSegmentReplicationTimeout(), + logger + ), + request.getCheckpoint().getShardId(), + SegmentReplicationTargetService.Actions.MERGED_SEGMENT_FILE_CHUNK, + new AtomicLong(0), + (throttleTime) -> {}, + indexShard.getRecoverySettings()::mergedSegmentReplicationRateLimiter + ); + + SegmentFileTransferHandler mergedSegmentFileTransferHandler = new SegmentFileTransferHandler( + indexShard, + request.getTargetNode(), + mergedSegmentFileChunkWriter, + logger, + indexShard.getThreadPool(), + new CancellableThreads(), + Math.toIntExact(indexShard.getRecoverySettings().getChunkSize().getBytes()), + indexShard.getRecoverySettings().getMaxConcurrentFileChunks() + ); + + final MultiChunkTransfer transfer = mergedSegmentFileTransferHandler + .createTransfer( + indexShard.store(), + request.getCheckpoint().getMetadataMap().values().toArray(new StoreFileMetadata[0]), + () -> 0, + new ActionListener<>() { + @Override + public void onResponse(Void unused) { + listener.onResponse(new GetSegmentFilesResponse(request.getFilesToFetch())); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + } + ); + transfer.start(); + } + } + @Override public void clusterChanged(ClusterChangedEvent event) { if (event.nodesRemoved()) { 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 550be9fb12965..a75623393f5ca 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -50,9 +50,9 @@ */ public class SegmentReplicationTarget extends ReplicationTarget { - private final ReplicationCheckpoint checkpoint; - private final SegmentReplicationSource source; - private final SegmentReplicationState state; + protected final ReplicationCheckpoint checkpoint; + protected final SegmentReplicationSource source; + protected final SegmentReplicationState state; protected final MultiFileWriter multiFileWriter; public final static String REPLICATION_PREFIX = "replication."; @@ -200,7 +200,7 @@ public void startReplication(ActionListener listener, BiConsumer getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { + protected List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FILE_DIFF); @@ -282,7 +282,7 @@ private boolean validateLocalChecksum(StoreFileMetadata file) { * @param fileName Name of the file being downloaded * @param bytesRecovered Number of bytes recovered */ - private void updateFileRecoveryBytes(String fileName, long bytesRecovered) { + protected void updateFileRecoveryBytes(String fileName, long bytesRecovered) { ReplicationLuceneIndex index = state.getIndex(); if (index != null) { index.addRecoveredBytesToFile(fileName, bytesRecovered); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index d57f35a5079fc..ccadd763fc675 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -28,12 +28,15 @@ import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.IndexShardNotStartedException; import org.opensearch.index.shard.IndexShardState; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.FileChunkRequest; import org.opensearch.indices.recovery.ForceSyncRequest; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RetryableTransportClient; +import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationCollection; import org.opensearch.indices.replication.common.ReplicationCollection.ReplicationRef; @@ -48,6 +51,7 @@ import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -80,6 +84,8 @@ public class SegmentReplicationTargetService extends AbstractLifecycleComponent public static class Actions { public static final String FILE_CHUNK = "internal:index/shard/replication/file_chunk"; public static final String FORCE_SYNC = "internal:index/shard/replication/segments_sync"; + public static final String PUBLISH_MERGED_SEGMENT = "internal:index/shard/replication/publish_merged_segment"; + public static final String MERGED_SEGMENT_FILE_CHUNK = "internal:index/shard/replication/merged_segment_file_chunk"; } @Deprecated @@ -152,6 +158,18 @@ public SegmentReplicationTargetService( ForceSyncRequest::new, new ForceSyncTransportRequestHandler() ); + transportService.registerRequestHandler( + Actions.PUBLISH_MERGED_SEGMENT, + ThreadPool.Names.GENERIC, + PublishMergedSegmentRequest::new, + new PublishMergedSegmentHandler() + ); + transportService.registerRequestHandler( + Actions.MERGED_SEGMENT_FILE_CHUNK, + ThreadPool.Names.GENERIC, + FileChunkRequest::new, + new MergedSegmentFileChunkTransportRequestHandler() + ); replicator.setSourceFactory(sourceFactory); } @@ -268,6 +286,10 @@ public ReplicationRef get(long replicationId) { return replicator.get(replicationId); } + public ReplicationRef getMergedSegmentReplicationRef(long replicationId) { + return replicator.getMergeReplicationRef(replicationId); + } + public SegmentReplicationTarget get(ShardId shardId) { return replicator.get(shardId); } @@ -627,4 +649,153 @@ private void failShard(ReplicationFailedException e, IndexShard indexShard) { } } + /** + * start a round of fetch merged segment files + */ + public class PublishMergedSegmentHandler implements TransportRequestHandler { + public PublishMergedSegmentHandler() {} + + @Override + public void messageReceived(PublishMergedSegmentRequest request, TransportChannel channel, Task task) throws Exception { + onNewMergedSegmentCheckpoint( + request.getMergedSegment(), + indicesService.getShardOrNull(request.getMergedSegment().getShardId()), + channel + ); + } + } + + public synchronized void onNewMergedSegmentCheckpoint( + final ReplicationCheckpoint receivedCheckpoint, + final IndexShard replicaShard, + TransportChannel channel + ) throws IOException { + logger.debug( + () -> new ParameterizedMessage("Replica received new merged segment checkpoint [{}] from primary", receivedCheckpoint) + ); + // if the shard is in any state + if (replicaShard.state().equals(IndexShardState.CLOSED)) { + // ignore if shard is closed + logger.trace(() -> "Ignoring merged segment checkpoint, Shard is closed"); + channel.sendResponse(new IndexShardClosedException(replicaShard.shardId())); + return; + } + + if (replicaShard.state().equals(IndexShardState.STARTED) == true) { + // Checks if received checkpoint is already present + List ongoingReplicationTargetList = getMergedSegmentReplicationTarget(replicaShard.shardId()); + if (ongoingReplicationTargetList != null) { + for (MergedSegmentReplicationTarget ongoingReplicationTarget : ongoingReplicationTargetList) { + if (ongoingReplicationTarget.getCheckpoint().getPrimaryTerm() < receivedCheckpoint.getPrimaryTerm()) { + logger.debug( + () -> new ParameterizedMessage( + "Cancelling ongoing merge replication {} from old primary with primary term {}", + ongoingReplicationTarget.description(), + ongoingReplicationTarget.getCheckpoint().getPrimaryTerm() + ) + ); + ongoingReplicationTarget.cancel("Cancelling stuck merged segment target after new primary"); + } else if (ongoingReplicationTarget.checkpoint.equals(receivedCheckpoint)) { + logger.debug( + () -> new ParameterizedMessage( + "Ignoring new merge replication checkpoint - shard is currently replicating to checkpoint {}", + ongoingReplicationTarget.getCheckpoint() + ) + ); + channel.sendResponse( + new IllegalArgumentException(String.format("merged segment %s already exist", receivedCheckpoint)) + ); + return; + } + } + } + if (replicaShard.shouldProcessMergedSegmentCheckpoint(receivedCheckpoint)) { + startMergedSegmentReplication(replicaShard, receivedCheckpoint, new SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + logger.debug( + () -> new ParameterizedMessage( + "[shardId {}] [replication id {}] Merge Replication complete to {}, timing data: {}", + replicaShard.shardId().getId(), + state.getReplicationId(), + receivedCheckpoint, + state.getTimingData() + ) + ); + try { + channel.sendResponse(TransportResponse.Empty.INSTANCE); + } catch (Exception e) { + logger.warn("merge pre copy is successful, but fail to send merge pre copy response", e); + } + } + + @Override + public void onReplicationFailure( + SegmentReplicationState state, + ReplicationFailedException e, + boolean sendShardFailure + ) { + logReplicationFailure(state, e, replicaShard); + if (sendShardFailure == true) { + failShard(e, replicaShard); + } + try { + channel.sendResponse(e); + } catch (Exception t) { + t.addSuppressed(e); + logger.warn("merge pre copy is failure, fail to send merge pre copy response", t); + } + } + }); + } + } else { + logger.trace( + () -> new ParameterizedMessage( + "Ignoring merged segment checkpoint, shard not started {} {}", + receivedCheckpoint, + replicaShard.state() + ) + ); + channel.sendResponse(new IndexShardNotStartedException(replicaShard.shardId(), replicaShard.state())); + } + } + + List getMergedSegmentReplicationTarget(ShardId shardId) { + return replicator.getMergedSegmentReplicationTarget(shardId); + } + + public void startMergedSegmentReplication( + final IndexShard indexShard, + final ReplicationCheckpoint checkpoint, + final SegmentReplicationListener listener + ) { + replicator.startMergedSegmentReplication(indexShard, checkpoint, sourceFactory.get(indexShard), listener); + } + + private class MergedSegmentFileChunkTransportRequestHandler implements TransportRequestHandler { + + // How many bytes we've copied since we last called RateLimiter.pause + final AtomicLong bytesSinceLastPause = new AtomicLong(); + + @Override + public void messageReceived(final FileChunkRequest request, TransportChannel channel, Task task) throws Exception { + try ( + ReplicationRef ref = replicator.getMergeReplicationRef( + request.recoveryId(), + request.shardId() + ) + ) { + final MergedSegmentReplicationTarget target = ref.get(); + final ActionListener listener = target.createOrFinishListener(channel, Actions.MERGED_SEGMENT_FILE_CHUNK, request); + target.handleFileChunk( + request, + target, + bytesSinceLastPause, + recoverySettings.mergedSegmentReplicationRateLimiter(), + listener + ); + } + } + } + } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java index c35898cfffe1e..24b96122f9d3a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -13,6 +13,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.opensearch.OpenSearchCorruptionException; +import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; @@ -30,12 +31,15 @@ import org.opensearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.TimeUnit; +import reactor.util.annotation.NonNull; + /** * This class is responsible for managing segment replication events on replicas. * It uses a {@link ReplicationCollection} to track ongoing replication events and @@ -48,6 +52,7 @@ public class SegmentReplicator { private static final Logger logger = LogManager.getLogger(SegmentReplicator.class); private final ReplicationCollection onGoingReplications; + private final ReplicationCollection onGoingMergedSegmentReplications; private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); @@ -58,6 +63,7 @@ public class SegmentReplicator { public SegmentReplicator(ThreadPool threadPool) { this.onGoingReplications = new ReplicationCollection<>(logger, threadPool); + this.onGoingMergedSegmentReplications = new ReplicationCollection<>(logger, threadPool); this.threadPool = threadPool; this.sourceFactory = new SetOnce<>(); } @@ -111,6 +117,37 @@ SegmentReplicationTarget startReplication( return target; } + /** + * Start a round of replication merged segment. + * @param indexShard - {@link IndexShard} replica shard + * @param checkpoint - {@link ReplicationCheckpoint} merged segment to replicate + * @param listener - {@link ReplicationListener} + */ + public void startMergedSegmentReplication( + final IndexShard indexShard, + final ReplicationCheckpoint checkpoint, + final SegmentReplicationSource source, + final SegmentReplicationTargetService.SegmentReplicationListener listener + ) { + final MergedSegmentReplicationTarget target = new MergedSegmentReplicationTarget(indexShard, checkpoint, source, listener); + startMergedSegmentReplication(target, indexShard.getRecoverySettings().activityTimeout()); + } + + void startMergedSegmentReplication(final MergedSegmentReplicationTarget target, TimeValue timeout) { + final long replicationId; + try { + replicationId = onGoingMergedSegmentReplications.start(target, timeout); + } catch (ReplicationFailedException e) { + // replication already running for shard. + target.fail(e, false); + return; + } + logger.info(() -> new ParameterizedMessage("Added new merged segment replication to collection {}", target.description())); + // Currently, we have not counted the completion information of the pre-copy merged segment, so the completedReplications parameter + // is null. + threadPool.generic().execute(new ReplicationRunner(replicationId, onGoingMergedSegmentReplications, null)); + } + /** * Retrieves segment replication statistics for a specific shard. * Its computed based on the last and first entry in the replicationCheckpointStats map. @@ -243,12 +280,20 @@ public long getTimestamp() { /** * Runnable implementation to trigger a replication event. */ - private class ReplicationRunner extends AbstractRunnable { + private class ReplicationRunner extends AbstractRunnable { final long replicationId; - - public ReplicationRunner(long replicationId) { + final ReplicationCollection onGoingReplications; + final Map completedReplications; + + public ReplicationRunner( + long replicationId, + @NonNull ReplicationCollection onGoingReplications, + @Nullable Map completedReplications + ) { this.replicationId = replicationId; + this.onGoingReplications = onGoingReplications; + this.completedReplications = completedReplications; } @Override @@ -258,16 +303,24 @@ public void onFailure(Exception e) { @Override public void doRun() { - start(replicationId); + start(replicationId, onGoingReplications, completedReplications); } } - private void start(final long replicationId) { + private void start( + final long replicationId, + ReplicationCollection onGoingReplications, + Map completedReplications + ) { final SegmentReplicationTarget target; - try (ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.get(replicationId)) { + logger.info("onGoingReplications size {}, id {}", onGoingReplications.size(), replicationId); + try ( + ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.get(replicationId) + ) { // This check is for handling edge cases where the reference is removed before the ReplicationRunner is started by the // threadpool. if (replicationRef == null) { + logger.info("replicationRef is null"); return; } target = replicationRef.get(); @@ -278,7 +331,9 @@ public void onResponse(Void o) { logger.debug(() -> new ParameterizedMessage("Finished replicating {} marking as done.", target.description())); pruneCheckpointsUpToLastSync(target.indexShard()); onGoingReplications.markAsDone(replicationId); - if (target.state().getIndex().recoveredFileCount() != 0 && target.state().getIndex().recoveredBytes() != 0) { + if (target.state().getIndex().recoveredFileCount() != 0 + && target.state().getIndex().recoveredBytes() != 0 + && null != completedReplications) { completedReplications.put(target.shardId(), target.state()); } } @@ -306,7 +361,7 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) return; } logger.trace(() -> new ParameterizedMessage("Added new replication to collection {}", target.description())); - threadPool.generic().execute(new ReplicationRunner(replicationId)); + threadPool.generic().execute(new ReplicationRunner(replicationId, onGoingReplications, completedReplications)); } private boolean isStoreCorrupt(SegmentReplicationTarget target) { @@ -336,6 +391,7 @@ int size() { void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); + onGoingMergedSegmentReplications.cancelForShard(shardId, reason); replicationCheckpointStats.remove(shardId); primaryCheckpoint.remove(shardId); } @@ -344,6 +400,10 @@ SegmentReplicationTarget get(ShardId shardId) { return onGoingReplications.getOngoingReplicationTarget(shardId); } + List getMergedSegmentReplicationTarget(ShardId shardId) { + return onGoingMergedSegmentReplications.getOngoingReplicationTargetList(shardId); + } + ReplicationCheckpoint getPrimaryCheckpoint(ShardId shardId) { return primaryCheckpoint.get(shardId); } @@ -359,4 +419,12 @@ SegmentReplicationState getCompleted(ShardId shardId) { ReplicationCollection.ReplicationRef get(long id, ShardId shardId) { return onGoingReplications.getSafe(id, shardId); } + + ReplicationCollection.ReplicationRef getMergeReplicationRef(long id) { + return onGoingMergedSegmentReplications.get(id); + } + + ReplicationCollection.ReplicationRef getMergeReplicationRef(long id, ShardId shardId) { + return onGoingMergedSegmentReplications.getSafe(id, shardId); + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java new file mode 100644 index 0000000000000..a7a1123b0c4f4 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java @@ -0,0 +1,91 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.opensearch.action.ActionRequest; +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +/** + * Request object for publish merged segment + * + * @opensearch.internal + */ +public class PublishMergedSegmentRequest extends ActionRequest { + private final ReplicationSegmentCheckpoint mergedSegment; + + public PublishMergedSegmentRequest(ReplicationSegmentCheckpoint mergedSegment) { + this.mergedSegment = mergedSegment; + } + + public PublishMergedSegmentRequest(StreamInput in) throws IOException { + super(in); + this.mergedSegment = new ReplicationSegmentCheckpoint(in); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + mergedSegment.writeTo(out); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof PublishMergedSegmentRequest that)) return false; + return Objects.equals(mergedSegment, that.mergedSegment); + } + + @Override + public int hashCode() { + return Objects.hash(mergedSegment); + } + + @Override + public String toString() { + return "PublishMergedSegmentRequest{" + "mergedSegment=" + mergedSegment + '}'; + } + + public ReplicationSegmentCheckpoint getMergedSegment() { + return mergedSegment; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java new file mode 100644 index 0000000000000..7cecd1720950d --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java @@ -0,0 +1,125 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.StoreFileMetadata; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + +/** + * Represents a Pre-copy merged segment which is sent to a replica shard. + * Inherit {@link ReplicationCheckpoint}, but the segmentsGen and segmentInfosVersion will not be used. + * + * @opensearch.internal + */ +@ExperimentalApi +public class ReplicationSegmentCheckpoint extends ReplicationCheckpoint { + private final String segmentName; + + public ReplicationSegmentCheckpoint( + ShardId shardId, + long primaryTerm, + long length, + String codec, + Map metadataMap, + String segmentName + ) { + super(shardId, primaryTerm, SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED, length, codec, metadataMap); + this.segmentName = segmentName; + } + + public ReplicationSegmentCheckpoint(StreamInput in) throws IOException { + super(in); + segmentName = in.readString(); + } + + /** + * The segmentName + * + * @return the segmentCommitInfo name + */ + public String getSegmentName() { + return segmentName; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(segmentName); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReplicationSegmentCheckpoint that = (ReplicationSegmentCheckpoint) o; + return getPrimaryTerm() == that.getPrimaryTerm() + && segmentName.equals(that.segmentName) + && Objects.equals(getShardId(), that.getShardId()) + && getCodec().equals(that.getCodec()); + } + + @Override + public int hashCode() { + return Objects.hash(getShardId(), getPrimaryTerm(), segmentName); + } + + @Override + public String toString() { + return "ReplicationCheckpoint{" + + "shardId=" + + getShardId() + + ", primaryTerm=" + + getPrimaryTerm() + + ", segmentsGen=" + + getSegmentsGen() + + ", version=" + + getSegmentInfosVersion() + + ", size=" + + getLength() + + ", codec=" + + getCodec() + + ", timestamp=" + + getCreatedTimeStamp() + + ", segmentName=" + + segmentName + + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationCollection.java index 4950b5da2915f..df255a9dcfcda 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationCollection.java @@ -285,6 +285,16 @@ public T getOngoingReplicationTarget(ShardId shardId) { return replicationTargetList.size() > 0 ? replicationTargetList.get(0) : null; } + /** + * Get targets for shard + * + * @param shardId shardId + * @return ReplicationTarget list for input shardId + */ + public List getOngoingReplicationTargetList(ShardId shardId) { + return onGoingTargetEvents.values().stream().filter(t -> t.indexShard.shardId().equals(shardId)).collect(Collectors.toList()); + } + /** * a reference to {@link ReplicationTarget}, which implements {@link AutoCloseable}. closing the reference * causes {@link ReplicationTarget#decRef()} to be called. This makes sure that the underlying resources diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index 4d85a3c491af8..b62d4a9aa291a 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -45,6 +45,7 @@ import org.opensearch.test.junit.annotations.TestLogging; import org.hamcrest.MatcherAssert; import org.junit.Assert; +import org.junit.Ignore; import java.io.IOException; import java.nio.channels.FileChannel; @@ -60,6 +61,7 @@ import java.util.function.BiConsumer; import java.util.stream.Collectors; +import static org.opensearch.common.util.FeatureFlags.MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG; import static org.opensearch.index.engine.EngineTestCase.assertAtMostOneLuceneDocumentPerSequenceNumber; import static org.opensearch.index.shard.RemoteStoreRefreshListener.EXCLUDE_FILES; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -628,6 +630,14 @@ public void testShallowCopySnapshotForClosedIndexSuccessful() throws Exception { } } + @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) + @Override + @Ignore + public void testMergedSegmentReplication() throws Exception { + // TODO: wait for remote store to support merged segment warmer + super.testMergedSegmentReplication(); + } + private RemoteStoreReplicationSource getRemoteStoreReplicationSource(IndexShard shard, Runnable postGetFilesRunnable) { return new RemoteStoreReplicationSource(shard) { @Override 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 90ae37f014708..8fa72432e52b5 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -9,6 +9,7 @@ package org.opensearch.index.shard; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; import org.opensearch.ExceptionsHelper; @@ -26,6 +27,7 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lucene.Lucene; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -61,6 +63,7 @@ import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.CopyState; import org.opensearch.indices.replication.common.ReplicationFailedException; @@ -95,6 +98,7 @@ import java.util.function.BiConsumer; import java.util.function.Function; +import static org.opensearch.common.util.FeatureFlags.MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG; import static org.opensearch.index.engine.EngineTestCase.assertAtMostOneLuceneDocumentPerSequenceNumber; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasToString; @@ -153,6 +157,35 @@ public void testReplication() throws Exception { } } + @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) + public void testMergedSegmentReplication() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + final IndexShard replicaShard = shards.getReplicas().get(0); + + // index and replicate segments to replica. + int numDocs = randomIntBetween(10, 20); + shards.indexDocs(numDocs); + primaryShard.refresh("test"); + flushShard(primaryShard); + + shards.indexDocs(numDocs); + primaryShard.refresh("test"); + flushShard(primaryShard); + replicateSegments(primaryShard, List.of(replicaShard)); + shards.assertAllEqual(2 * numDocs); + + primaryShard.forceMerge(new ForceMergeRequest("test").maxNumSegments(1)); + replicateMergedSegments(primaryShard, List.of(replicaShard)); + primaryShard.refresh("test"); + assertEquals(1, primaryShard.segments(false).size()); + // After the pre-copy merged segment is completed, the merged segment is not visible in the replica, and the number of segments + // in the replica shard is still 2. + assertEquals(2, replicaShard.segments(false).size()); + } + } + /** * Test that latestReplicationCheckpoint returns null only for docrep enabled indices */ @@ -965,6 +998,30 @@ public void testReuseReplicationCheckpointWhenLatestInfosIsUnChanged() throws Ex } } + public void testComputeReplicationSegmentCheckpoint() throws Exception { + try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { + final IndexShard primaryShard = shards.getPrimary(); + shards.startAll(); + shards.indexDocs(10); + shards.refresh("test"); + shards.flush(); + SegmentInfos segmentInfos = Lucene.readSegmentInfos(primaryShard.store().directory()); + for (SegmentCommitInfo segmentCommitInfo : segmentInfos) { + Map segmentMetadataMap = primaryShard.store().getSegmentMetadataMap(segmentCommitInfo); + ReplicationSegmentCheckpoint expectedCheckpoint = new ReplicationSegmentCheckpoint( + primaryShard.shardId, + primaryShard.getOperationPrimaryTerm(), + segmentMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), + primaryShard.getDefaultCodecName(), + segmentMetadataMap, + segmentCommitInfo.info.name + ); + ReplicationSegmentCheckpoint checkpoint = primaryShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); + assertEquals(expectedCheckpoint, checkpoint); + } + } + } + public void testComputeReplicationCheckpointNullInfosReturnsEmptyCheckpoint() throws Exception { try (ReplicationGroup shards = createGroup(1, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { final IndexShard primaryShard = shards.getPrimary(); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index e541e988f3920..ccb7f9a8b5f4a 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -421,6 +421,7 @@ public void testTemporaryFilesNotCleanup() throws Exception { SegmentReplicationSource segmentReplicationSource = getSegmentReplicationSource( primaryShard, (repId) -> targetService.get(repId), + (repId) -> targetService.getMergedSegmentReplicationRef(repId), runnablePostGetFiles ); when(sourceFactory.get(any())).thenReturn(segmentReplicationSource); diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoverySettingsDynamicUpdateTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoverySettingsDynamicUpdateTests.java index 650db6a2baf5a..794e14ab19943 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoverySettingsDynamicUpdateTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoverySettingsDynamicUpdateTests.java @@ -54,6 +54,10 @@ public void testZeroBytesPerSecondIsNoRateLimit() { Settings.builder().put(RecoverySettings.INDICES_REPLICATION_MAX_BYTES_PER_SEC_SETTING.getKey(), 0).build() ); assertNull(recoverySettings.replicationRateLimiter()); + clusterSettings.applySettings( + Settings.builder().put(RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING.getKey(), 0).build() + ); + assertNull(recoverySettings.mergedSegmentReplicationRateLimiter()); } public void testSetReplicationMaxBytesPerSec() { @@ -72,6 +76,44 @@ public void testSetReplicationMaxBytesPerSec() { assertEquals(80, (int) recoverySettings.replicationRateLimiter().getMBPerSec()); } + public void testSetMergedSegmentReplicationMaxBytesPerSec() { + assertEquals(40, (int) recoverySettings.mergedSegmentReplicationRateLimiter().getMBPerSec()); + clusterSettings.applySettings( + Settings.builder() + .put( + RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING.getKey(), + new ByteSizeValue(60, ByteSizeUnit.MB) + ) + .build() + ); + assertEquals(60, (int) recoverySettings.mergedSegmentReplicationRateLimiter().getMBPerSec()); + clusterSettings.applySettings( + Settings.builder() + .put( + RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING.getKey(), + new ByteSizeValue(80, ByteSizeUnit.MB) + ) + .build() + ); + assertEquals(80, (int) recoverySettings.mergedSegmentReplicationRateLimiter().getMBPerSec()); + } + + public void testMergedSegmentReplicationTimeout() { + assertEquals(15, (int) recoverySettings.getMergedSegmentReplicationTimeout().minutes()); + clusterSettings.applySettings( + Settings.builder() + .put(RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING.getKey(), TimeValue.timeValueMinutes(5)) + .build() + ); + assertEquals(5, (int) recoverySettings.getMergedSegmentReplicationTimeout().minutes()); + clusterSettings.applySettings( + Settings.builder() + .put(RecoverySettings.INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING.getKey(), TimeValue.timeValueMinutes(25)) + .build() + ); + assertEquals(25, (int) recoverySettings.getMergedSegmentReplicationTimeout().minutes()); + } + public void testRetryDelayStateSync() { long duration = between(1, 1000); TimeUnit timeUnit = randomFrom(TimeUnit.MILLISECONDS, TimeUnit.SECONDS, TimeUnit.MINUTES, TimeUnit.HOURS); diff --git a/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java new file mode 100644 index 0000000000000..489f67cb9fa61 --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java @@ -0,0 +1,250 @@ +/* + * 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.lucene.index.IndexFileNames; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.store.ByteBuffersIndexOutput; +import org.apache.lucene.util.Version; +import org.opensearch.OpenSearchCorruptionException; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.replication.TestReplicationSource; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardTestCase; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; +import org.opensearch.indices.replication.common.ReplicationFailedException; +import org.opensearch.indices.replication.common.ReplicationType; +import org.junit.Assert; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +public class MergedSegmentReplicationTargetTests extends IndexShardTestCase { + + private MergedSegmentReplicationTarget mergedSegmentReplicationTarget; + private IndexShard indexShard, spyIndexShard; + private ReplicationSegmentCheckpoint mergedSegment; + private ByteBuffersDataOutput buffer; + + private static final String SEGMENT_NAME = "_0.si"; + private static final StoreFileMetadata SEGMENT_FILE = new StoreFileMetadata(SEGMENT_NAME, 1L, "0", Version.LATEST); + private static final StoreFileMetadata SEGMENT_FILE_DIFF = new StoreFileMetadata(SEGMENT_NAME, 5L, "different", Version.LATEST); + + private static final Map SI_SNAPSHOT = Map.of(SEGMENT_FILE.name(), SEGMENT_FILE); + + private static final Map SI_SNAPSHOT_DIFFERENT = Map.of(SEGMENT_FILE_DIFF.name(), SEGMENT_FILE_DIFF); + + private SegmentInfos testSegmentInfos; + + @Override + public void setUp() throws Exception { + + super.setUp(); + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .build(); + + indexShard = newStartedShard(false, indexSettings, new NRTReplicationEngineFactory()); + spyIndexShard = spy(indexShard); + + testSegmentInfos = spyIndexShard.store().readLastCommittedSegmentsInfo(); + buffer = new ByteBuffersDataOutput(); + try (ByteBuffersIndexOutput indexOutput = new ByteBuffersIndexOutput(buffer, "", null)) { + testSegmentInfos.write(indexOutput); + } + mergedSegment = new ReplicationSegmentCheckpoint( + spyIndexShard.shardId(), + spyIndexShard.getPendingPrimaryTerm(), + 1, + indexShard.getLatestReplicationCheckpoint().getCodec(), + SI_SNAPSHOT, + IndexFileNames.parseSegmentName(SEGMENT_NAME) + ); + } + + public void testSuccessfulResponse_startReplication() { + + SegmentReplicationSource segrepSource = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) {} + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) {} + + @Override + public void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + assertEquals(1, filesToFetch.size()); + assert (filesToFetch.contains(SEGMENT_FILE)); + listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); + } + }; + + SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( + SegmentReplicationTargetService.SegmentReplicationListener.class + ); + mergedSegmentReplicationTarget = new MergedSegmentReplicationTarget(spyIndexShard, mergedSegment, segrepSource, segRepListener); + + mergedSegmentReplicationTarget.startReplication(new ActionListener() { + @Override + public void onResponse(Void replicationResponse) { + mergedSegmentReplicationTarget.markAsDone(); + } + + @Override + public void onFailure(Exception e) { + logger.error("Unexpected onFailure", e); + Assert.fail(); + } + }, (ReplicationCheckpoint checkpoint, IndexShard indexShard) -> { + assertEquals(mergedSegment, checkpoint); + assertEquals(indexShard, spyIndexShard); + }); + } + + public void testFailureResponse_getMergedSegmentFiles() { + + Exception exception = new Exception("dummy failure"); + SegmentReplicationSource segrepSource = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) {} + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) {} + + @Override + public void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + listener.onFailure(exception); + } + }; + SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( + SegmentReplicationTargetService.SegmentReplicationListener.class + ); + mergedSegmentReplicationTarget = new MergedSegmentReplicationTarget(spyIndexShard, mergedSegment, segrepSource, segRepListener); + + mergedSegmentReplicationTarget.startReplication(new ActionListener() { + @Override + public void onResponse(Void replicationResponse) { + Assert.fail(); + } + + @Override + public void onFailure(Exception e) { + assertEquals(exception, e.getCause().getCause()); + mergedSegmentReplicationTarget.fail(new ReplicationFailedException(e), false); + } + }, mock(BiConsumer.class)); + } + + public void testFailure_differentSegmentFiles() throws IOException { + + SegmentReplicationSource segrepSource = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) {} + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) {} + + @Override + public void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); + } + }; + SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( + SegmentReplicationTargetService.SegmentReplicationListener.class + ); + mergedSegmentReplicationTarget = new MergedSegmentReplicationTarget(spyIndexShard, mergedSegment, segrepSource, segRepListener); + when(spyIndexShard.getSegmentMetadataMap()).thenReturn(SI_SNAPSHOT_DIFFERENT); + mergedSegmentReplicationTarget.startReplication(new ActionListener() { + @Override + public void onResponse(Void replicationResponse) { + Assert.fail(); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof OpenSearchCorruptionException); + assertTrue(e.getMessage().contains("has local copies of segments that differ from the primary")); + mergedSegmentReplicationTarget.fail(new ReplicationFailedException(e), false); + } + }, mock(BiConsumer.class)); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + closeShards(spyIndexShard, indexShard); + } +} 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 2531790ede4af..132088561cdae 100644 --- a/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java @@ -22,6 +22,7 @@ import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.transport.CapturingTransport; @@ -29,6 +30,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.Map; import static org.mockito.Mockito.mock; @@ -132,6 +134,32 @@ public void testGetSegmentFiles() { assertTrue(capturedRequest.request instanceof GetSegmentFilesRequest); } + public void testGetMergedSegmentFiles() { + StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", 1L, "checksum", Version.LATEST); + final ReplicationCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + PRIMARY_TERM, + 1, + Codec.getDefault().getName(), + Map.of("testFile", testMetadata), + "_0" + ); + replicationSource.getMergedSegmentFiles( + REPLICATION_ID, + checkpoint, + Arrays.asList(testMetadata), + mock(IndexShard.class), + (fileName, bytesRecovered) -> {}, + mock(ActionListener.class) + ); + CapturingTransport.CapturedRequest[] requestList = transport.getCapturedRequestsAndClear(); + assertEquals(1, requestList.length); + CapturingTransport.CapturedRequest capturedRequest = requestList[0]; + assertEquals(SegmentReplicationSourceService.Actions.GET_MERGED_SEGMENT_FILES, capturedRequest.action); + assertEquals(sourceNode, capturedRequest.node); + assertTrue(capturedRequest.request instanceof GetSegmentFilesRequest); + } + /** * This test verifies the transport request timeout value for fetching the segment files. */ @@ -161,6 +189,36 @@ public void testTransportTimeoutForGetSegmentFilesAction() { assertEquals(recoverySettings.internalActionLongTimeout(), capturedRequest.options.timeout()); } + /** + * This test verifies the transport request timeout value for fetching the merged segment files. + */ + public void testTransportTimeoutForGetMergedSegmentFilesAction() { + long fileSize = (long) (Math.pow(10, 9)); + StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", fileSize, "checksum", Version.LATEST); + final ReplicationCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + PRIMARY_TERM, + 1, + Codec.getDefault().getName(), + Map.of("testFile", testMetadata), + "_0" + ); + replicationSource.getMergedSegmentFiles( + REPLICATION_ID, + checkpoint, + Arrays.asList(testMetadata), + mock(IndexShard.class), + (fileName, bytesRecovered) -> {}, + mock(ActionListener.class) + ); + CapturingTransport.CapturedRequest[] requestList = transport.getCapturedRequestsAndClear(); + assertEquals(1, requestList.length); + CapturingTransport.CapturedRequest capturedRequest = requestList[0]; + assertEquals(SegmentReplicationSourceService.Actions.GET_MERGED_SEGMENT_FILES, capturedRequest.action); + assertEquals(sourceNode, capturedRequest.node); + assertEquals(recoverySettings.getMergedSegmentReplicationTimeout(), capturedRequest.options.timeout()); + } + private DiscoveryNode newDiscoveryNode(String nodeName) { return new DiscoveryNode( nodeName, diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java index 8f84053f2618e..7de90b34e8fc2 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java @@ -20,9 +20,12 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; +import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.shard.ReplicationGroup; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; @@ -38,6 +41,7 @@ import org.opensearch.transport.TransportException; import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; +import org.junit.Assert; import java.io.IOException; import java.util.Collections; @@ -61,6 +65,7 @@ public class SegmentReplicationSourceServiceTests extends OpenSearchTestCase { private SegmentReplicationSourceService segmentReplicationSourceService; private OngoingSegmentReplications ongoingSegmentReplications; private IndexShard mockIndexShard; + private IndicesService mockIndicesService; @Override public void setUp() throws Exception { @@ -68,7 +73,7 @@ public void setUp() throws Exception { // setup mocks mockIndexShard = CopyStateTests.createMockIndexShard(); ShardId testShardId = mockIndexShard.shardId(); - IndicesService mockIndicesService = mock(IndicesService.class); + mockIndicesService = mock(IndicesService.class); IndexService mockIndexService = mock(IndexService.class); when(mockIndicesService.iterator()).thenReturn(List.of(mockIndexService).iterator()); when(mockIndicesService.indexServiceSafe(testShardId.getIndex())).thenReturn(mockIndexService); @@ -85,6 +90,7 @@ public void setUp() throws Exception { when(mockIndexShard.isPrimaryMode()).thenReturn(true); final ReplicationGroup replicationGroup = mock(ReplicationGroup.class); when(mockIndexShard.getReplicationGroup()).thenReturn(replicationGroup); + when(mockIndexShard.state()).thenReturn(IndexShardState.STARTED); when(replicationGroup.getInSyncAllocationIds()).thenReturn(Collections.emptySet()); // This mirrors the creation of the ReplicationCheckpoint inside CopyState testCheckpoint = new ReplicationCheckpoint( @@ -112,6 +118,8 @@ public void setUp() throws Exception { final Settings settings = Settings.builder().put("node.name", SegmentReplicationTargetServiceTests.class.getSimpleName()).build(); final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); final RecoverySettings recoverySettings = new RecoverySettings(settings, clusterSettings); + when(mockIndexShard.getRecoverySettings()).thenReturn(recoverySettings); + when(mockIndexShard.getThreadPool()).thenReturn(testThreadPool); ongoingSegmentReplications = spy(new OngoingSegmentReplications(mockIndicesService, recoverySettings)); segmentReplicationSourceService = new SegmentReplicationSourceService( @@ -150,6 +158,115 @@ public void onFailure(Exception e) { }); } + public void testGetMergedSegmentFiles() { + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + 1, + "allocationId", + localNode, + Collections.emptyList(), + testCheckpoint + ); + executeGetMergedSegmentFiles(request, new ActionListener<>() { + @Override + public void onResponse(GetSegmentFilesResponse response) { + assertEquals(0, response.files.size()); + } + + @Override + public void onFailure(Exception e) { + fail("unexpected exception: " + e); + } + }); + } + + public void testGetMergedSegmentFiles_indexNotFound() { + when(mockIndicesService.indexServiceSafe(mockIndexShard.shardId().getIndex())).thenReturn(null); + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + 1, + "allocationId", + localNode, + Collections.emptyList(), + testCheckpoint + ); + executeGetMergedSegmentFiles(request, new ActionListener<>() { + @Override + public void onResponse(GetSegmentFilesResponse response) { + Assert.fail("Test should succeed"); + } + + @Override + public void onFailure(Exception e) { + assert e.getCause() instanceof IndexNotFoundException; + } + }); + } + + public void testGetMergedSegmentFiles_shardClosed() { + when(mockIndexShard.state()).thenReturn(IndexShardState.CLOSED); + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + 1, + "allocationId", + localNode, + Collections.emptyList(), + testCheckpoint + ); + executeGetMergedSegmentFiles(request, new ActionListener<>() { + @Override + public void onResponse(GetSegmentFilesResponse response) { + Assert.fail("Test should succeed"); + } + + @Override + public void onFailure(Exception e) { + assert e.getCause() instanceof IndexShardClosedException; + } + }); + } + + public void testGetMergedSegmentFiles_shardNonPrimary() { + when(mockIndexShard.isPrimaryMode()).thenReturn(false); + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + 1, + "allocationId", + localNode, + Collections.emptyList(), + testCheckpoint + ); + executeGetMergedSegmentFiles(request, new ActionListener<>() { + @Override + public void onResponse(GetSegmentFilesResponse response) { + Assert.fail("Test should succeed"); + } + + @Override + public void onFailure(Exception e) { + assert e.getCause() instanceof IllegalArgumentException; + } + }); + } + + public void testGetMergedSegmentFiles_receiveLowerPrimaryTermRequest() { + final GetSegmentFilesRequest request = new GetSegmentFilesRequest( + 1, + "allocationId", + localNode, + Collections.emptyList(), + testCheckpoint + ); + when(mockIndexShard.getOperationPrimaryTerm()).thenReturn(request.getCheckpoint().getPrimaryTerm() + 1); + executeGetMergedSegmentFiles(request, new ActionListener<>() { + @Override + public void onResponse(GetSegmentFilesResponse response) { + Assert.fail("Test should succeed"); + } + + @Override + public void onFailure(Exception e) { + assert e.getCause() instanceof IllegalArgumentException; + } + }); + } + public void testUpdateVisibleCheckpoint() { UpdateVisibleCheckpointRequest request = new UpdateVisibleCheckpointRequest( 0L, @@ -253,6 +370,35 @@ public GetSegmentFilesResponse read(StreamInput in) throws IOException { ); } + private void executeGetMergedSegmentFiles(GetSegmentFilesRequest request, ActionListener listener) { + transportService.sendRequest( + localNode, + SegmentReplicationSourceService.Actions.GET_MERGED_SEGMENT_FILES, + request, + new TransportResponseHandler() { + @Override + public void handleResponse(GetSegmentFilesResponse response) { + listener.onResponse(response); + } + + @Override + public void handleException(TransportException e) { + listener.onFailure(e); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public GetSegmentFilesResponse read(StreamInput in) throws IOException { + return new GetSegmentFilesResponse(in); + } + } + ); + } + private void executeUpdateVisibleCheckpoint(UpdateVisibleCheckpointRequest request, ActionListener listener) { try (BytesStreamOutput out = new BytesStreamOutput()) { request.writeTo(out); 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 8a47b87b09f30..01790288b337a 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -52,6 +52,7 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.EmptyTransportResponseHandler; +import org.opensearch.transport.TransportChannel; import org.opensearch.transport.TransportRequestOptions; import org.opensearch.transport.TransportService; import org.junit.Assert; @@ -203,6 +204,29 @@ public void onReplicationFailure(SegmentReplicationState state, ReplicationFaile assertEquals(0, latch.getCount()); } + public void testsSuccessfulMergeSegmentReplication_listenerCompletes() throws InterruptedException { + CountDownLatch latch = new CountDownLatch(1); + sut.startMergedSegmentReplication( + replicaShard, + primaryShard.getLatestReplicationCheckpoint(), + new SegmentReplicationTargetService.SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + assertEquals(SegmentReplicationState.Stage.DONE, state.getStage()); + latch.countDown(); + } + + @Override + public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + logger.error("Unexpected error", e); + Assert.fail("Test should succeed"); + } + } + ); + latch.await(2, TimeUnit.SECONDS); + assertEquals(0, latch.getCount()); + } + public void testReplicationFails() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); final OpenSearchException expectedError = new OpenSearchException("Fail"); @@ -445,17 +469,70 @@ public void cancel() { verify(serviceSpy, times(1)).startReplication(eq(replicaShard), any(), any()); } + public void testMergedSegmentReplicating_HigherPrimaryTermReceived() throws IOException { + SegmentReplicationTargetService serviceSpy = spy(sut); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) {} + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + Assert.fail("Unreachable"); + } + }; + final MergedSegmentReplicationTarget targetSpy = spy( + new MergedSegmentReplicationTarget( + replicaShard, + checkpoint, + source, + mock(SegmentReplicationTargetService.SegmentReplicationListener.class) + ) + ); + doReturn(List.of(targetSpy)).when(serviceSpy).getMergedSegmentReplicationTarget(any()); + serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard, mock(TransportChannel.class)); + // ensure the old target is cancelled. and new iteration kicks off. + verify(targetSpy, times(1)).cancel("Cancelling stuck merged segment target after new primary"); + verify(serviceSpy, times(1)).startMergedSegmentReplication(eq(replicaShard), any(), any()); + } + public void testNewCheckpointBehindCurrentCheckpoint() { SegmentReplicationTargetService spy = spy(sut); spy.onNewCheckpoint(checkpoint, replicaShard); verify(spy, times(0)).startReplication(any(), any(), any()); } + public void testNewMergedSegmentCheckpointBehindCurrentCheckpoint() throws IOException { + SegmentReplicationTargetService spy = spy(sut); + ReplicationCheckpoint oldCheckpoint = new ReplicationCheckpoint( + replicaShard.shardId(), + replicaShard.getOperationPrimaryTerm() - 1, + 0L, + 0L, + replicaShard.getLatestReplicationCheckpoint().getCodec() + ); + spy.onNewMergedSegmentCheckpoint(oldCheckpoint, replicaShard, mock(TransportChannel.class)); + verify(spy, times(0)).startReplication(any(), any(), any()); + } + public void testShardNotStarted() throws IOException { SegmentReplicationTargetService spy = spy(sut); IndexShard shard = newShard(false); spy.onNewCheckpoint(checkpoint, shard); verify(spy, times(0)).startReplication(any(), any(), any()); + + spy.onNewMergedSegmentCheckpoint(checkpoint, shard, mock(TransportChannel.class)); + verify(spy, times(0)).startMergedSegmentReplication(any(), any(), any()); closeShards(shard); } @@ -472,6 +549,10 @@ public void testRejectCheckpointOnShardPrimaryMode() throws IOException { // Verify that checkpoint is not processed as shard is in PrimaryMode. verify(spy, times(0)).startReplication(any(), any(), any()); + + spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, spyShard, mock(TransportChannel.class)); + verify(spy, times(0)).startMergedSegmentReplication(any(), any(), any()); + closeShards(primaryShard); } @@ -550,6 +631,9 @@ public void testOnNewCheckpointInvokedOnClosedShardDoesNothing() throws IOExcept SegmentReplicationTargetService spy = spy(sut); spy.onNewCheckpoint(aheadCheckpoint, replicaShard); verify(spy, times(0)).updateLatestReceivedCheckpoint(any(), any()); + + spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, replicaShard, mock(TransportChannel.class)); + verify(spy, times(0)).updateLatestReceivedCheckpoint(any(), any()); } public void testBeforeIndexShardClosed_DoesNothingForDocRepIndex() throws IOException { diff --git a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java index 4ce4e28690697..60c025d025d53 100644 --- a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java @@ -144,6 +144,7 @@ public void testGetReplicationTargetMultiReplicationsForSingleShard() throws Exc final IndexShard shard2 = shards.addReplica(); final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shard1); final long recoveryId2 = startRecovery(collection, shards.getPrimaryNode(), shard2); + assertEquals(2, collection.getOngoingReplicationTargetList(shard1.shardId()).size()); try { collection.getOngoingReplicationTarget(shard1.shardId()); } catch (AssertionError e) { diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index 062ebd2051f6e..bcc4f8d77e72b 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -377,6 +377,8 @@ public synchronized int startReplicas(int numOfReplicasToStart) throws IOExcepti } } } + // Update the status of the replicas in the routing table to IndexShardState.STARTED. + updateAllocationIDsOnPrimary(); return started; } 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 2a158a645691e..205666b494a7b 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 @@ -39,10 +39,12 @@ import org.apache.lucene.store.IndexInput; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; +import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.replication.TransportReplicationAction; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; import org.opensearch.cluster.metadata.RepositoryMetadata; @@ -79,6 +81,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.indices.breaker.CircuitBreakerService; +import org.opensearch.core.transport.TransportResponse; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.env.Environment; @@ -135,11 +138,13 @@ import org.opensearch.indices.recovery.StartRecoveryRequest; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; +import org.opensearch.indices.replication.MergedSegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationSource; import org.opensearch.indices.replication.SegmentReplicationSourceFactory; import org.opensearch.indices.replication.SegmentReplicationState; import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.CopyState; @@ -156,8 +161,10 @@ import org.opensearch.repositories.blobstore.OpenSearchBlobStoreRepositoryIntegTestCase; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.Snapshot; +import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.DummyShardLock; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.CapturingTransport; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -194,7 +201,9 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; /** @@ -693,6 +702,41 @@ protected IndexShard newShard( // This is fine since we are not testing the node stats now Function mockReplicationStatsProvider = mock(Function.class); when(mockReplicationStatsProvider.apply(any())).thenReturn(new ReplicationStats(800, 800, 500)); + + // mock transport service for merged segment warmer + CapturingTransport transport = new CapturingTransport(); + TransportService transportService = transport.createTransportService( + Settings.EMPTY, + mock(ThreadPool.class), + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT), + null, + Collections.emptySet(), + NoopTracer.INSTANCE + ); + transportService.start(); + transportService.acceptIncomingRequests(); + TransportService spyTransportService = spy(transportService); + doAnswer(invocation -> { + ActionListenerResponseHandler handler = invocation.getArgument(3); + handler.handleResponse(TransportResponse.Empty.INSTANCE); + return mock(TransportResponse.class); + }).when(spyTransportService) + .sendRequest( + any(DiscoveryNode.class), + eq(SegmentReplicationTargetService.Actions.PUBLISH_MERGED_SEGMENT), + any(PublishMergedSegmentRequest.class), + any() + ); + + // mock cluster service for merged segment warmer + DiscoveryNodes spyDiscoveryNodes = spy(discoveryNodes); + doAnswer(invocation -> mock(DiscoveryNode.class)).when(spyDiscoveryNodes).get(any()); + ClusterState clusterState = mock(ClusterState.class); + when(clusterState.nodes()).thenReturn(spyDiscoveryNodes); + ClusterService spyClusterService = spy(clusterService); + doAnswer(invocation -> clusterState).when(spyClusterService).state(); + indexShard = new IndexShard( routing, indexSettings, @@ -724,7 +768,7 @@ protected IndexShard newShard( false, discoveryNodes, mockReplicationStatsProvider, - new MergedSegmentWarmerFactory(null, null, null), + new MergedSegmentWarmerFactory(spyTransportService, new RecoverySettings(nodeSettings, clusterSettings), spyClusterService), false, () -> Boolean.FALSE, indexSettings::getRefreshInterval, @@ -1548,6 +1592,7 @@ private SegmentReplicationTargetService prepareForReplication( final SegmentReplicationSource replicationSource = getSegmentReplicationSource( primaryShard, (repId) -> targetService.get(repId), + (repId) -> targetService.getMergedSegmentReplicationRef(repId), postGetFilesRunnable ); when(sourceFactory.get(any())).thenReturn(replicationSource); @@ -1629,10 +1674,48 @@ public void onReplicationFailure(SegmentReplicationState state, ReplicationFaile }; } + /** + * Get listener on started merged segment replication event which verifies replica shard store with primary's after completion + * @param primaryShard - source of segment replication + * @param replicaShard - target of segment replication + * @param primaryMetadata - primary shard metadata before start of segment replication + * @param latch - Latch which allows consumers of this utility to ensure segment replication completed successfully + * @return Returns SegmentReplicationTargetService.SegmentReplicationListener + */ + public SegmentReplicationTargetService.SegmentReplicationListener getMergedSegmentTargetListener( + IndexShard primaryShard, + IndexShard replicaShard, + Map primaryMetadata, + CountDownLatch latch + ) { + return new SegmentReplicationTargetService.SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + try { + // After the pre-copy merged segment is completed, the merged segment is not yet visible in the replica shard, so it is + // necessary to obtain the entire file list through listAll(). + Set replicaFiles = Arrays.stream(replicaShard.store().directory().listAll()).collect(Collectors.toSet()); + assertTrue(replicaFiles.containsAll(primaryMetadata.keySet())); + } catch (Exception e) { + throw ExceptionsHelper.convertToRuntime(e); + } finally { + latch.countDown(); + } + } + + @Override + public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + logger.error("Unexpected replication failure in test", e); + Assert.fail("test replication should not fail: " + e); + } + }; + } + /** * Utility method which creates a segment replication source, which copies files from primary shard to target shard * @param primaryShard Primary IndexShard - source of segment replication * @param getTargetFunc - provides replication target from target service using replication id + * @param getMergedSegmentTargetFunc - provides merged segment replication target from target service using replication id * @param postGetFilesRunnable - Consumer which is executed after file copy operation. This can be used to stub operations * which are desired right after files are copied. e.g. To work with temp files * @return Return SegmentReplicationSource @@ -1640,6 +1723,7 @@ public void onReplicationFailure(SegmentReplicationState state, ReplicationFaile public SegmentReplicationSource getSegmentReplicationSource( IndexShard primaryShard, Function> getTargetFunc, + Function> getMergedSegmentTargetFunc, Consumer postGetFilesRunnable ) { return new TestReplicationSource() { @@ -1678,6 +1762,27 @@ public void getSegmentFiles( postGetFilesRunnable.accept(indexShard); listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); } + + @Override + public void getMergedSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + try ( + final ReplicationCollection.ReplicationRef replicationRef = getMergedSegmentTargetFunc + .apply(replicationId) + ) { + writeFileChunks(replicationRef.get(), primaryShard, filesToFetch.toArray(new StoreFileMetadata[] {})); + } catch (IOException e) { + listener.onFailure(e); + } + postGetFilesRunnable.accept(indexShard); + listener.onResponse(new GetSegmentFilesResponse(filesToFetch)); + } }; } @@ -1714,6 +1819,35 @@ protected final List replicateSegments(IndexShard prim return ids; } + /** + * Segment Replication specific test method - Replicate merged segments to a list of replicas from a given primary. + * This test will use a real {@link SegmentReplicationTarget} for each replica with a mock {@link SegmentReplicationSource} that + * writes all segments directly to the target. + * @param primaryShard - {@link IndexShard} The current primary shard. + * @param replicaShards - Replicas that will be updated. + */ + protected final void replicateMergedSegments(IndexShard primaryShard, List replicaShards) throws IOException, + InterruptedException { + // Latch to block test execution until replica catches up + final CountDownLatch countDownLatch = new CountDownLatch(replicaShards.size()); + // Get primary metadata to verify with replica's, used to ensure replica catches up + Map primaryMetadata; + try (final GatedCloseable segmentInfosSnapshot = primaryShard.getSegmentInfosSnapshot()) { + final SegmentInfos primarySegmentInfos = segmentInfosSnapshot.get(); + primaryMetadata = primaryShard.store().getSegmentMetadataMap(primarySegmentInfos); + } + for (IndexShard replica : replicaShards) { + final SegmentReplicationTargetService targetService = prepareForReplication(primaryShard, replica); + targetService.startMergedSegmentReplication( + replica, + primaryShard.getLatestReplicationCheckpoint(), + getMergedSegmentTargetListener(primaryShard, replica, primaryMetadata, countDownLatch) + ); + } + countDownLatch.await(30, TimeUnit.SECONDS); + assertEquals("Replication merged segment should complete successfully", 0, countDownLatch.getCount()); + } + private void writeFileChunks(SegmentReplicationTarget target, IndexShard primary, StoreFileMetadata[] files) throws IOException { for (StoreFileMetadata md : files) { try (IndexInput in = primary.store().directory().openInput(md.name(), IOContext.READONCE)) { From 1c1f8027355d8092ecf9e35b0e51987d06122852 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Mon, 12 May 2025 15:21:55 +0800 Subject: [PATCH 02/23] update log Signed-off-by: guojialiang --- .../indices/replication/SegmentReplicationSourceService.java | 4 +++- .../indices/replication/SegmentReplicationTargetService.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index db033fa3b4870..4b8fbef18c861 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -46,6 +46,7 @@ import java.io.IOException; import java.util.HashSet; +import java.util.Locale; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -204,13 +205,14 @@ public void messageReceived(GetSegmentFilesRequest request, TransportChannel cha return; } if (indexShard.routingEntry().primary() == false || indexShard.isPrimaryMode() == false) { - listener.onFailure(new IllegalArgumentException(String.format("%s is not primary", shardId))); + listener.onFailure(new IllegalArgumentException(String.format(Locale.ROOT, "%s is not primary", shardId))); return; } if (indexShard.getOperationPrimaryTerm() > request.getCheckpoint().getPrimaryTerm()) { listener.onFailure( new IllegalArgumentException( String.format( + Locale.ROOT, "request primary term %d is lower than %d", request.getCheckpoint().getPrimaryTerm(), indexShard.getOperationPrimaryTerm() diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index ccadd763fc675..0da4003975f2a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -52,6 +52,7 @@ import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -703,7 +704,7 @@ public synchronized void onNewMergedSegmentCheckpoint( ) ); channel.sendResponse( - new IllegalArgumentException(String.format("merged segment %s already exist", receivedCheckpoint)) + new IllegalArgumentException(String.format(Locale.ROOT, "merged segment %s already exist", receivedCheckpoint)) ); return; } From 58528e9019f96226c55ddab465c7fa23b76fd04b Mon Sep 17 00:00:00 2001 From: guojialiang Date: Mon, 12 May 2025 15:45:26 +0800 Subject: [PATCH 03/23] fix test Signed-off-by: guojialiang --- .../java/org/opensearch/index/shard/RemoteIndexShardTests.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index b62d4a9aa291a..f05546de46880 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -45,7 +45,6 @@ import org.opensearch.test.junit.annotations.TestLogging; import org.hamcrest.MatcherAssert; import org.junit.Assert; -import org.junit.Ignore; import java.io.IOException; import java.nio.channels.FileChannel; @@ -632,7 +631,7 @@ public void testShallowCopySnapshotForClosedIndexSuccessful() throws Exception { @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) @Override - @Ignore + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/pull/18255") public void testMergedSegmentReplication() throws Exception { // TODO: wait for remote store to support merged segment warmer super.testMergedSegmentReplication(); From 7c8449e7da1e348a969141fb9ea75b49f1cda3b4 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Tue, 13 May 2025 12:17:51 +0800 Subject: [PATCH 04/23] add tests Signed-off-by: guojialiang --- .../SegmentReplicationTargetServiceTests.java | 40 +++++++++++++ .../PublishMergedSegmentRequestTests.java | 59 +++++++++++++++++++ 2 files changed, 99 insertions(+) create mode 100644 server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java 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 01790288b337a..11113b49c061f 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -506,6 +506,46 @@ public void getSegmentFiles( verify(serviceSpy, times(1)).startMergedSegmentReplication(eq(replicaShard), any(), any()); } + public void testMergedSegmentReplicating_MergedSegmentAlreadyExist() throws IOException { + SegmentReplicationTargetService serviceSpy = spy(sut); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) {} + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + BiConsumer fileProgressTracker, + ActionListener listener + ) { + Assert.fail("Unreachable"); + } + }; + final MergedSegmentReplicationTarget targetSpy = spy( + new MergedSegmentReplicationTarget( + replicaShard, + checkpoint, + source, + mock(SegmentReplicationTargetService.SegmentReplicationListener.class) + ) + ); + doReturn(List.of(targetSpy)).when(serviceSpy).getMergedSegmentReplicationTarget(any()); + // already exist + serviceSpy.onNewMergedSegmentCheckpoint(checkpoint, replicaShard, mock(TransportChannel.class)); + verify(serviceSpy, times(0)).startMergedSegmentReplication(eq(replicaShard), any(), any()); + + // new merged segment + serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard, mock(TransportChannel.class)); + verify(serviceSpy, times(1)).startMergedSegmentReplication(eq(replicaShard), any(), any()); + } + public void testNewCheckpointBehindCurrentCheckpoint() { SegmentReplicationTargetService spy = spy(sut); spy.onNewCheckpoint(checkpoint, replicaShard); diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java new file mode 100644 index 0000000000000..d041a08593aba --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.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.indices.replication.checkpoint; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Collections; +import java.util.Objects; + +import static org.hamcrest.core.IsEqual.equalTo; + +public class PublishMergedSegmentRequestTests extends OpenSearchTestCase { + + public void testPublishMergedSegmentRequest() { + ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + new ShardId(new Index("1", "1"), 0), + 0, + 0, + "", + Collections.emptyMap(), + "_0" + ); + PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); + assertNull(request.validate()); + assertEquals(checkpoint, request.getMergedSegment()); + assertEquals(Objects.hash(checkpoint), request.hashCode()); + } + + public void testSerialize() throws Exception { + ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + new ShardId(new Index("1", "1"), 0), + 0, + 0, + "", + Collections.emptyMap(), + "_0" + ); + PublishMergedSegmentRequest originalRequest = new PublishMergedSegmentRequest(checkpoint); + PublishMergedSegmentRequest cloneRequest; + try (BytesStreamOutput out = new BytesStreamOutput()) { + originalRequest.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + cloneRequest = new PublishMergedSegmentRequest(in); + } + } + assertThat(cloneRequest, equalTo(originalRequest)); + assertThat(cloneRequest.getMergedSegment(), equalTo(originalRequest.getMergedSegment())); + } +} From e3273725b247bd37ecb0689a04e8162fbc129b49 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Tue, 13 May 2025 16:16:07 +0800 Subject: [PATCH 05/23] IndexShard support getActiveReplicaNodes Signed-off-by: guojialiang --- .../engine/LocalMergedSegmentWarmer.java | 10 +-- .../opensearch/index/shard/IndexShard.java | 23 +++++- .../IndexShardReplicationGroupListener.java | 74 +++++++++++++++++++ .../replication/SegmentReplicator.java | 4 +- .../SegmentReplicationIndexShardTests.java | 15 ++++ .../index/shard/IndexShardTestCase.java | 11 +-- 6 files changed, 114 insertions(+), 23 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java diff --git a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java index b64d178f4d667..9f471ab946f85 100644 --- a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java +++ b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java @@ -41,8 +41,6 @@ import org.apache.lucene.index.SegmentReader; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; import org.opensearch.core.action.ActionListener; import org.opensearch.core.transport.TransportResponse; @@ -90,13 +88,7 @@ public void warm(LeafReader leafReader) throws IOException { ReplicationSegmentCheckpoint mergedSegment = indexShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(mergedSegment); - DiscoveryNodes nodes = clusterService.state().nodes(); - List replicaShards = indexShard.getReplicationGroup().getRoutingTable().replicaShards(); - List activeReplicaNodes = replicaShards.stream() - .filter(ShardRouting::active) - .map(s -> nodes.get(s.currentNodeId())) - .toList(); - + List activeReplicaNodes = indexShard.getActiveReplicaNodes(); if (activeReplicaNodes.isEmpty()) { logger.trace("There are no active replicas, skip pre copy merged segment [{}]", segmentCommitInfo.info.name); return; 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 1e0d1bad0e6ce..26edc99ef2734 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -377,6 +377,7 @@ Runnable getGlobalCheckpointSyncer() { private final Supplier refreshInterval; private final Object refreshMutex; private volatile AsyncShardRefreshTask refreshTask; + private List activeReplicaNodes = new ArrayList<>(); public IndexShard( final ShardRouting shardRouting, @@ -475,7 +476,7 @@ public IndexShard( threadPool::absoluteTimeInMillis, (retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, aId, getPendingPrimaryTerm(), retentionLeases, listener), this::getSafeCommitInfo, - pendingReplicationActions, + new CompositeReplicationGroupListener(List.of(pendingReplicationActions, new IndexShardReplicationGroupListener(this))), isShardOnRemoteEnabledNode ); @@ -527,6 +528,14 @@ public boolean shouldCache(Query query) { } } + private record CompositeReplicationGroupListener(List> consumers) implements Consumer { + + @Override + public void accept(ReplicationGroup replicationGroup) { + consumers.forEach(c -> c.accept(replicationGroup)); + } + } + /** * By default, UNASSIGNED_SEQ_NO is used as the initial global checkpoint for new shard initialization. Ingestion * source does not track sequence numbers explicitly and hence defaults to NO_OPS_PERFORMED for compatibility. @@ -5488,6 +5497,18 @@ public AsyncIOProcessor getTranslogSyncProcessor() { return translogSyncProcessor; } + public DiscoveryNodes getDiscoveryNodes() { + return discoveryNodes; + } + + public List getActiveReplicaNodes() { + return activeReplicaNodes; + } + + public void setActiveReplicaNodes(List activeReplicaNodes) { + this.activeReplicaNodes = activeReplicaNodes; + } + enum ShardMigrationState { REMOTE_NON_MIGRATING, REMOTE_MIGRATING_SEEDED, diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java b/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java new file mode 100644 index 0000000000000..5648efba21c35 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java @@ -0,0 +1,74 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.index.shard; + +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.ShardRouting; + +import java.util.List; +import java.util.function.Consumer; + +/** + * When a new {@link ReplicationGroup} is generated, the relevant changes of {@link IndexShard} + * + * @opensearch.internal + */ +public class IndexShardReplicationGroupListener implements Consumer { + private final IndexShard indexShard; + private volatile long replicationGroupVersion = -1; + + public IndexShardReplicationGroupListener(IndexShard indexShard) { + this.indexShard = indexShard; + } + + @Override + public void accept(ReplicationGroup replicationGroup) { + if (isNewerVersion(replicationGroup)) { + synchronized (this) { + if (isNewerVersion(replicationGroup)) { + replicationGroupVersion = replicationGroup.getVersion(); + DiscoveryNodes discoveryNodes = indexShard.getDiscoveryNodes(); + List replicaShards = replicationGroup.getRoutingTable().replicaShards(); + indexShard.setActiveReplicaNodes( + replicaShards.stream().filter(ShardRouting::active).map(s -> discoveryNodes.get(s.currentNodeId())).toList() + ); + } + } + } + } + + private boolean isNewerVersion(ReplicationGroup replicationGroup) { + // Relative comparison to mitigate long overflow + return replicationGroup.getVersion() - replicationGroupVersion > 0; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java index 24b96122f9d3a..b653eda3cc5f4 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -142,7 +142,7 @@ void startMergedSegmentReplication(final MergedSegmentReplicationTarget target, target.fail(e, false); return; } - logger.info(() -> new ParameterizedMessage("Added new merged segment replication to collection {}", target.description())); + logger.trace(() -> new ParameterizedMessage("Added new merged segment replication to collection {}", target.description())); // Currently, we have not counted the completion information of the pre-copy merged segment, so the completedReplications parameter // is null. threadPool.generic().execute(new ReplicationRunner(replicationId, onGoingMergedSegmentReplications, null)); @@ -313,14 +313,12 @@ private void start( Map completedReplications ) { final SegmentReplicationTarget target; - logger.info("onGoingReplications size {}, id {}", onGoingReplications.size(), replicationId); try ( ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.get(replicationId) ) { // This check is for handling edge cases where the reference is removed before the ReplicationRunner is started by the // threadpool. if (replicationRef == null) { - logger.info("replicationRef is null"); return; } target = replicationRef.get(); 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 8fa72432e52b5..1df6921ad2211 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -131,6 +131,21 @@ protected Settings getIndexSettings() { return settings; } + public void testUpdateActiveReplicaNodes() throws Exception { + try (ReplicationGroup shards = createGroup(0, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + assertEquals(0, primaryShard.getActiveReplicaNodes().size()); + IndexShard replicaShard = shards.addReplica(); + assertEquals(0, primaryShard.getActiveReplicaNodes().size()); + shards.startReplicas(0); + assertEquals(1, primaryShard.getActiveReplicaNodes().size()); + shards.removeReplica(replicaShard); + closeShards(replicaShard); + assertEquals(0, primaryShard.getActiveReplicaNodes().size()); + } + } + /** * Validates happy path of segment replication where primary index docs which are replicated to replica shards. Assertions * made on doc count on both primary and replica. 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 205666b494a7b..6d0a87acfeaf1 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 @@ -44,7 +44,6 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.replication.TransportReplicationAction; -import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; import org.opensearch.cluster.metadata.RepositoryMetadata; @@ -729,14 +728,6 @@ protected IndexShard newShard( any() ); - // mock cluster service for merged segment warmer - DiscoveryNodes spyDiscoveryNodes = spy(discoveryNodes); - doAnswer(invocation -> mock(DiscoveryNode.class)).when(spyDiscoveryNodes).get(any()); - ClusterState clusterState = mock(ClusterState.class); - when(clusterState.nodes()).thenReturn(spyDiscoveryNodes); - ClusterService spyClusterService = spy(clusterService); - doAnswer(invocation -> clusterState).when(spyClusterService).state(); - indexShard = new IndexShard( routing, indexSettings, @@ -768,7 +759,7 @@ protected IndexShard newShard( false, discoveryNodes, mockReplicationStatsProvider, - new MergedSegmentWarmerFactory(spyTransportService, new RecoverySettings(nodeSettings, clusterSettings), spyClusterService), + new MergedSegmentWarmerFactory(spyTransportService, new RecoverySettings(nodeSettings, clusterSettings), null), false, () -> Boolean.FALSE, indexSettings::getRefreshInterval, From 290c5b968cf6ce7153e8dc415810fe4d659ba28d Mon Sep 17 00:00:00 2001 From: guojialiang Date: Tue, 13 May 2025 17:54:12 +0800 Subject: [PATCH 06/23] add test Signed-off-by: guojialiang --- .../index/shard/RemoteIndexShardTests.java | 8 +++++++ .../SegmentReplicationIndexShardTests.java | 22 +++++++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index f05546de46880..eb52ee96a157e 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -637,6 +637,14 @@ public void testMergedSegmentReplication() throws Exception { super.testMergedSegmentReplication(); } + @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) + @Override + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/pull/18255") + public void testMergedSegmentReplicationWithZeroReplica() throws Exception { + // TODO: wait for remote store to support merged segment warmer + super.testMergedSegmentReplicationWithZeroReplica(); + } + private RemoteStoreReplicationSource getRemoteStoreReplicationSource(IndexShard shard, Runnable postGetFilesRunnable) { return new RemoteStoreReplicationSource(shard) { @Override 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 1df6921ad2211..8cf77a1bbd240 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -201,6 +201,28 @@ public void testMergedSegmentReplication() throws Exception { } } + @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) + public void testMergedSegmentReplicationWithZeroReplica() throws Exception { + try (ReplicationGroup shards = createGroup(0, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + + int numDocs = randomIntBetween(10, 20); + shards.indexDocs(numDocs); + primaryShard.refresh("test"); + flushShard(primaryShard); + + shards.indexDocs(numDocs); + primaryShard.refresh("test"); + flushShard(primaryShard); + shards.assertAllEqual(2 * numDocs); + + primaryShard.forceMerge(new ForceMergeRequest("test").maxNumSegments(1)); + primaryShard.refresh("test"); + assertEquals(1, primaryShard.segments(false).size()); + } + } + /** * Test that latestReplicationCheckpoint returns null only for docrep enabled indices */ From ddb4258d1b4dfe5266653d1d67aef3e62a81cbd3 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Tue, 3 Jun 2025 10:50:18 +0800 Subject: [PATCH 07/23] add pre-verification. Signed-off-by: guojialiang --- .../org/opensearch/index/engine/LocalMergedSegmentWarmer.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java index 9f471ab946f85..1f94f7d84b009 100644 --- a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java +++ b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java @@ -84,6 +84,9 @@ public LocalMergedSegmentWarmer( @Override public void warm(LeafReader leafReader) throws IOException { + // IndexWriter.IndexReaderWarmer#warm is called by IndexWriter#mergeMiddle. The type of leafReader should be SegmentReader. + assert leafReader instanceof SegmentReader; + SegmentCommitInfo segmentCommitInfo = ((SegmentReader) leafReader).getSegmentInfo(); ReplicationSegmentCheckpoint mergedSegment = indexShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(mergedSegment); From d63a6764b25cb28a4ef7f66e0142b31ad8dec414 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 4 Jun 2025 00:02:50 +0800 Subject: [PATCH 08/23] extend ReplicationAction instead of ActionRequest Signed-off-by: guojialiang --- .../opensearch/index/shard/IndexShardIT.java | 4 +- .../org/opensearch/index/IndexService.java | 7 +- .../engine/LocalMergedSegmentWarmer.java | 56 +---- .../opensearch/index/shard/IndexShard.java | 34 +-- .../IndexShardReplicationGroupListener.java | 74 ------ .../org/opensearch/indices/IndicesModule.java | 2 + .../opensearch/indices/IndicesService.java | 7 +- .../cluster/IndicesClusterStateService.java | 21 +- .../SegmentReplicationTargetService.java | 68 ++--- .../checkpoint/MergedSegmentPublisher.java | 58 +++++ .../PublishMergedSegmentAction.java | 236 ++++++++++++++++++ .../PublishMergedSegmentRequest.java | 37 +-- .../SegmentReplicationIndexShardTests.java | 15 -- ...dicesLifecycleListenerSingleNodeTests.java | 4 +- .../indices/IndicesRequestCacheTests.java | 4 +- ...actIndicesClusterStateServiceTestCase.java | 4 +- ...ClusterStateServiceRandomUpdatesTests.java | 4 +- .../SegmentReplicationTargetServiceTests.java | 15 +- .../snapshots/SnapshotResiliencyTests.java | 4 +- .../index/shard/IndexShardTestCase.java | 40 +-- 20 files changed, 385 insertions(+), 309 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.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 6c6105ac6fa56..f85c422a404c9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -88,6 +88,7 @@ import org.opensearch.indices.DefaultRemoteStoreSettings; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.plugins.Plugin; import org.opensearch.search.builder.SearchSourceBuilder; @@ -729,7 +730,8 @@ public static final IndexShard newIndexShard( OpenSearchTestCase::randomBoolean, () -> indexService.getIndexSettings().getRefreshInterval(), indexService.getRefreshMutex(), - clusterService.getClusterApplierService() + clusterService.getClusterApplierService(), + MergedSegmentPublisher.EMPTY ); } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index e11cc7df6c824..4a2981f735f16 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -104,6 +104,7 @@ import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.plugins.IndexStorePlugin; @@ -596,7 +597,8 @@ public synchronized IndexShard createShard( final DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode, DiscoveryNodes discoveryNodes, - MergedSegmentWarmerFactory mergedSegmentWarmerFactory + MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + MergedSegmentPublisher mergedSegmentPublisher ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); /* @@ -728,7 +730,8 @@ protected void closeInternal() { fixedRefreshIntervalSchedulingEnabled, this::getRefreshInterval, refreshMutex, - clusterService.getClusterApplierService() + clusterService.getClusterApplierService(), + this.indexSettings.isSegRepEnabledOrRemoteNode() ? mergedSegmentPublisher : null ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java index 1f94f7d84b009..7fef9c30fbf97 100644 --- a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java +++ b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java @@ -34,29 +34,16 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentReader; -import org.opensearch.action.ActionListenerResponseHandler; -import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.core.action.ActionListener; -import org.opensearch.core.transport.TransportResponse; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.recovery.RecoverySettings; -import org.opensearch.indices.replication.SegmentReplicationTargetService; -import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; -import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; -import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; /** * Implementation of a {@link IndexWriter.IndexReaderWarmer} when local on-disk segment replication is enabled. @@ -88,47 +75,6 @@ public void warm(LeafReader leafReader) throws IOException { assert leafReader instanceof SegmentReader; SegmentCommitInfo segmentCommitInfo = ((SegmentReader) leafReader).getSegmentInfo(); - ReplicationSegmentCheckpoint mergedSegment = indexShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); - PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(mergedSegment); - - List activeReplicaNodes = indexShard.getActiveReplicaNodes(); - if (activeReplicaNodes.isEmpty()) { - logger.trace("There are no active replicas, skip pre copy merged segment [{}]", segmentCommitInfo.info.name); - return; - } - - CountDownLatch countDownLatch = new CountDownLatch(activeReplicaNodes.size()); - AtomicInteger successfulCount = new AtomicInteger(0); - AtomicInteger failureCount = new AtomicInteger(0); - for (DiscoveryNode replicaNode : activeReplicaNodes) { - ActionListener listener = ActionListener.wrap(r -> { - successfulCount.incrementAndGet(); - countDownLatch.countDown(); - }, e -> { - failureCount.incrementAndGet(); - countDownLatch.countDown(); - }); - transportService.sendRequest( - replicaNode, - SegmentReplicationTargetService.Actions.PUBLISH_MERGED_SEGMENT, - request, - new ActionListenerResponseHandler<>(listener, (in) -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC) - ); - } - try { - countDownLatch.await(recoverySettings.getMergedSegmentReplicationTimeout().seconds(), TimeUnit.SECONDS); - logger.trace( - "pre copy merged segment [{}] to [{}] active replicas, [{}] successful, [{}] failed", - segmentCommitInfo.info.name, - activeReplicaNodes.size(), - successfulCount, - failureCount - ); - } catch (InterruptedException e) { - logger.warn( - () -> new ParameterizedMessage("Interrupted while waiting for pre copy merged segment [{}]", segmentCommitInfo.info.name), - e - ); - } + indexShard.publishMergedSegment(segmentCommitInfo); } } 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 82ff82c4971d6..7ac60bdf81341 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -199,6 +199,7 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; @@ -379,7 +380,7 @@ Runnable getGlobalCheckpointSyncer() { private final Object refreshMutex; private volatile AsyncShardRefreshTask refreshTask; private final ClusterApplierService clusterApplierService; - private List activeReplicaNodes = new ArrayList<>(); + private final MergedSegmentPublisher mergedSegmentPublisher; public IndexShard( final ShardRouting shardRouting, @@ -417,7 +418,8 @@ public IndexShard( final Supplier fixedRefreshIntervalSchedulingEnabled, final Supplier refreshInterval, final Object refreshMutex, - final ClusterApplierService clusterApplierService + final ClusterApplierService clusterApplierService, + @Nullable final MergedSegmentPublisher mergedSegmentPublisher ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -479,7 +481,7 @@ public IndexShard( threadPool::absoluteTimeInMillis, (retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, aId, getPendingPrimaryTerm(), retentionLeases, listener), this::getSafeCommitInfo, - new CompositeReplicationGroupListener(List.of(pendingReplicationActions, new IndexShardReplicationGroupListener(this))), + pendingReplicationActions, isShardOnRemoteEnabledNode ); @@ -525,6 +527,7 @@ public boolean shouldCache(Query query) { this.refreshInterval = refreshInterval; this.refreshMutex = Objects.requireNonNull(refreshMutex); this.clusterApplierService = clusterApplierService; + this.mergedSegmentPublisher = mergedSegmentPublisher; synchronized (this.refreshMutex) { if (shardLevelRefreshEnabled) { startRefreshTask(); @@ -532,14 +535,6 @@ public boolean shouldCache(Query query) { } } - private record CompositeReplicationGroupListener(List> consumers) implements Consumer { - - @Override - public void accept(ReplicationGroup replicationGroup) { - consumers.forEach(c -> c.accept(replicationGroup)); - } - } - /** * By default, UNASSIGNED_SEQ_NO is used as the initial global checkpoint for new shard initialization. Ingestion * source does not track sequence numbers explicitly and hence defaults to NO_OPS_PERFORMED for compatibility. @@ -1860,6 +1855,11 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th return checkpoint; } + public void publishMergedSegment(SegmentCommitInfo segmentCommitInfo) throws IOException { + assert mergedSegmentPublisher != null; + mergedSegmentPublisher.publish(this, computeReplicationSegmentCheckpoint(segmentCommitInfo)); + } + /** * Compute {@link ReplicationSegmentCheckpoint} from a SegmentCommitInfo. * This function fetches a metadata snapshot from the store that comes with an IO cost. @@ -5502,18 +5502,6 @@ public AsyncIOProcessor getTranslogSyncProcessor() { return translogSyncProcessor; } - public DiscoveryNodes getDiscoveryNodes() { - return discoveryNodes; - } - - public List getActiveReplicaNodes() { - return activeReplicaNodes; - } - - public void setActiveReplicaNodes(List activeReplicaNodes) { - this.activeReplicaNodes = activeReplicaNodes; - } - enum ShardMigrationState { REMOTE_NON_MIGRATING, REMOTE_MIGRATING_SEEDED, diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java b/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java deleted file mode 100644 index 5648efba21c35..0000000000000 --- a/server/src/main/java/org/opensearch/index/shard/IndexShardReplicationGroupListener.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -package org.opensearch.index.shard; - -import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.cluster.routing.ShardRouting; - -import java.util.List; -import java.util.function.Consumer; - -/** - * When a new {@link ReplicationGroup} is generated, the relevant changes of {@link IndexShard} - * - * @opensearch.internal - */ -public class IndexShardReplicationGroupListener implements Consumer { - private final IndexShard indexShard; - private volatile long replicationGroupVersion = -1; - - public IndexShardReplicationGroupListener(IndexShard indexShard) { - this.indexShard = indexShard; - } - - @Override - public void accept(ReplicationGroup replicationGroup) { - if (isNewerVersion(replicationGroup)) { - synchronized (this) { - if (isNewerVersion(replicationGroup)) { - replicationGroupVersion = replicationGroup.getVersion(); - DiscoveryNodes discoveryNodes = indexShard.getDiscoveryNodes(); - List replicaShards = replicationGroup.getRoutingTable().replicaShards(); - indexShard.setActiveReplicaNodes( - replicaShards.stream().filter(ShardRouting::active).map(s -> discoveryNodes.get(s.currentNodeId())).toList() - ); - } - } - } - } - - private boolean isNewerVersion(ReplicationGroup replicationGroup) { - // Relative comparison to mitigate long overflow - return replicationGroup.getVersion() - replicationGroupVersion > 0; - } -} diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index f7e52ce9fc1ae..ad2eb834b721e 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -82,6 +82,7 @@ import org.opensearch.index.shard.PrimaryReplicaSyncer; import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.mapper.MapperRegistry; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.store.IndicesStore; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; @@ -298,6 +299,7 @@ protected void configure() { bind(RetentionLeaseBackgroundSyncAction.class).asEagerSingleton(); bind(RetentionLeaseSyncer.class).asEagerSingleton(); bind(SegmentReplicationCheckpointPublisher.class).asEagerSingleton(); + bind(MergedSegmentPublisher.class).asEagerSingleton(); bind(SegmentReplicationPressureService.class).asEagerSingleton(); bind(RemoteStorePressureService.class).asEagerSingleton(); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 8b2c0547ca35e..27b547db37c7b 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -155,6 +155,7 @@ import org.opensearch.indices.recovery.RecoveryListener; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.Node; @@ -1229,7 +1230,8 @@ public IndexShard createShard( final DiscoveryNode sourceNode, final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, final DiscoveryNodes discoveryNodes, - final MergedSegmentWarmerFactory mergedSegmentWarmerFactory + final MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + final MergedSegmentPublisher mergedSegmentPublisher ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); ensureChangesAllowed(); @@ -1246,7 +1248,8 @@ public IndexShard createShard( targetNode, sourceNode, discoveryNodes, - mergedSegmentWarmerFactory + mergedSegmentWarmerFactory, + mergedSegmentPublisher ); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, mapping -> { 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 f7ac53db126b5..bba3031d66bac 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -85,6 +85,7 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.SegmentReplicationSourceService; import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.repositories.RepositoriesService; @@ -154,6 +155,8 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final MergedSegmentWarmerFactory mergedSegmentWarmerFactory; + private final MergedSegmentPublisher mergedSegmentPublisher; + @Inject public IndicesClusterStateService( final Settings settings, @@ -174,7 +177,8 @@ public IndicesClusterStateService( final RetentionLeaseSyncer retentionLeaseSyncer, final SegmentReplicationCheckpointPublisher checkpointPublisher, final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, - final MergedSegmentWarmerFactory mergedSegmentWarmerFactory + final MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + final MergedSegmentPublisher mergedSegmentPublisher ) { this( settings, @@ -195,7 +199,8 @@ public IndicesClusterStateService( globalCheckpointSyncAction::updateGlobalCheckpointForShard, retentionLeaseSyncer, remoteStoreStatsTrackerFactory, - mergedSegmentWarmerFactory + mergedSegmentWarmerFactory, + mergedSegmentPublisher ); } @@ -219,7 +224,8 @@ public IndicesClusterStateService( final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, - final MergedSegmentWarmerFactory mergedSegmentWarmerFactory + final MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + final MergedSegmentPublisher mergedSegmentPublisher ) { this.settings = settings; this.checkpointPublisher = checkpointPublisher; @@ -245,6 +251,7 @@ public IndicesClusterStateService( this.sendRefreshMapping = settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; this.mergedSegmentWarmerFactory = mergedSegmentWarmerFactory; + this.mergedSegmentPublisher = mergedSegmentPublisher; } @Override @@ -689,7 +696,8 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR sourceNode, remoteStoreStatsTrackerFactory, nodes, - mergedSegmentWarmerFactory + mergedSegmentWarmerFactory, + mergedSegmentPublisher ); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); @@ -1055,6 +1063,8 @@ U createIndex(IndexMetadata indexMetadata, List builtInIndex * @param targetNode the node where this shard will be recovered * @param sourceNode the source node to recover this shard from (it might be null) * @param remoteStoreStatsTrackerFactory factory for remote store stats trackers + * @param mergedSegmentWarmerFactory factory for merged segment warmer + * @param mergedSegmentPublisher merged segment publisher * @return a new shard * @throws IOException if an I/O exception occurs when creating the shard */ @@ -1071,7 +1081,8 @@ T createShard( @Nullable DiscoveryNode sourceNode, RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, DiscoveryNodes discoveryNodes, - MergedSegmentWarmerFactory mergedSegmentWarmerFactory + MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + MergedSegmentPublisher mergedSegmentPublisher ) throws IOException; /** diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index 0da4003975f2a..1be0224a77b60 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -28,15 +28,12 @@ import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardClosedException; -import org.opensearch.index.shard.IndexShardNotStartedException; import org.opensearch.index.shard.IndexShardState; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.FileChunkRequest; import org.opensearch.indices.recovery.ForceSyncRequest; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RetryableTransportClient; -import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationCollection; import org.opensearch.indices.replication.common.ReplicationCollection.ReplicationRef; @@ -52,8 +49,8 @@ import java.io.IOException; import java.util.List; -import java.util.Locale; import java.util.Optional; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; import static org.opensearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; @@ -85,7 +82,6 @@ public class SegmentReplicationTargetService extends AbstractLifecycleComponent public static class Actions { public static final String FILE_CHUNK = "internal:index/shard/replication/file_chunk"; public static final String FORCE_SYNC = "internal:index/shard/replication/segments_sync"; - public static final String PUBLISH_MERGED_SEGMENT = "internal:index/shard/replication/publish_merged_segment"; public static final String MERGED_SEGMENT_FILE_CHUNK = "internal:index/shard/replication/merged_segment_file_chunk"; } @@ -159,12 +155,6 @@ public SegmentReplicationTargetService( ForceSyncRequest::new, new ForceSyncTransportRequestHandler() ); - transportService.registerRequestHandler( - Actions.PUBLISH_MERGED_SEGMENT, - ThreadPool.Names.GENERIC, - PublishMergedSegmentRequest::new, - new PublishMergedSegmentHandler() - ); transportService.registerRequestHandler( Actions.MERGED_SEGMENT_FILE_CHUNK, ThreadPool.Names.GENERIC, @@ -650,27 +640,7 @@ private void failShard(ReplicationFailedException e, IndexShard indexShard) { } } - /** - * start a round of fetch merged segment files - */ - public class PublishMergedSegmentHandler implements TransportRequestHandler { - public PublishMergedSegmentHandler() {} - - @Override - public void messageReceived(PublishMergedSegmentRequest request, TransportChannel channel, Task task) throws Exception { - onNewMergedSegmentCheckpoint( - request.getMergedSegment(), - indicesService.getShardOrNull(request.getMergedSegment().getShardId()), - channel - ); - } - } - - public synchronized void onNewMergedSegmentCheckpoint( - final ReplicationCheckpoint receivedCheckpoint, - final IndexShard replicaShard, - TransportChannel channel - ) throws IOException { + public void onNewMergedSegmentCheckpoint(final ReplicationCheckpoint receivedCheckpoint, final IndexShard replicaShard) { logger.debug( () -> new ParameterizedMessage("Replica received new merged segment checkpoint [{}] from primary", receivedCheckpoint) ); @@ -678,7 +648,6 @@ public synchronized void onNewMergedSegmentCheckpoint( if (replicaShard.state().equals(IndexShardState.CLOSED)) { // ignore if shard is closed logger.trace(() -> "Ignoring merged segment checkpoint, Shard is closed"); - channel.sendResponse(new IndexShardClosedException(replicaShard.shardId())); return; } @@ -703,14 +672,12 @@ public synchronized void onNewMergedSegmentCheckpoint( ongoingReplicationTarget.getCheckpoint() ) ); - channel.sendResponse( - new IllegalArgumentException(String.format(Locale.ROOT, "merged segment %s already exist", receivedCheckpoint)) - ); return; } } } if (replicaShard.shouldProcessMergedSegmentCheckpoint(receivedCheckpoint)) { + CountDownLatch latch = new CountDownLatch(1); startMergedSegmentReplication(replicaShard, receivedCheckpoint, new SegmentReplicationListener() { @Override public void onReplicationDone(SegmentReplicationState state) { @@ -723,11 +690,7 @@ public void onReplicationDone(SegmentReplicationState state) { state.getTimingData() ) ); - try { - channel.sendResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - logger.warn("merge pre copy is successful, but fail to send merge pre copy response", e); - } + latch.countDown(); } @Override @@ -736,18 +699,24 @@ public void onReplicationFailure( ReplicationFailedException e, boolean sendShardFailure ) { - logReplicationFailure(state, e, replicaShard); - if (sendShardFailure == true) { - failShard(e, replicaShard); - } try { - channel.sendResponse(e); - } catch (Exception t) { - t.addSuppressed(e); - logger.warn("merge pre copy is failure, fail to send merge pre copy response", t); + logReplicationFailure(state, e, replicaShard); + if (sendShardFailure == true) { + failShard(e, replicaShard); + } + } finally { + latch.countDown(); } } }); + try { + latch.await(); + } catch (InterruptedException e) { + logger.warn( + () -> new ParameterizedMessage("Interrupted while waiting for pre copy merged segment [{}]", receivedCheckpoint), + e + ); + } } } else { logger.trace( @@ -757,7 +726,6 @@ public void onReplicationFailure( replicaShard.state() ) ); - channel.sendResponse(new IndexShardNotStartedException(replicaShard.shardId(), replicaShard.state())); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java new file mode 100644 index 0000000000000..a0c270cf9e3ce --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java @@ -0,0 +1,58 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.inject.Inject; +import org.opensearch.index.shard.IndexShard; + +import java.util.Objects; + +/** + * Publish merged segment. + * + * @opensearch.api + */ +@ExperimentalApi +public class MergedSegmentPublisher { + protected static Logger logger = LogManager.getLogger(MergedSegmentPublisher.class); + + private final PublishAction publishAction; + + // This Component is behind feature flag so we are manually binding this in IndicesModule. + @Inject + public MergedSegmentPublisher(PublishMergedSegmentAction publishAction) { + this(publishAction::publish); + } + + public MergedSegmentPublisher(PublishAction publishAction) { + this.publishAction = Objects.requireNonNull(publishAction); + } + + public void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint) { + publishAction.publish(indexShard, checkpoint); + } + + /** + * Represents an action that is invoked to publish merged segment to replica shard + * + * @opensearch.api + */ + @ExperimentalApi + public interface PublishAction { + void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint); + } + + /** + * NoOp Checkpoint publisher + */ + public static final MergedSegmentPublisher EMPTY = new MergedSegmentPublisher((indexShard, checkpoint) -> {}); +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java new file mode 100644 index 0000000000000..d3fe340d29dbd --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java @@ -0,0 +1,236 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.action.support.replication.ReplicationTask; +import org.opensearch.action.support.replication.TransportReplicationAction; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.util.concurrent.ThreadContextAccess; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.index.IndexNotFoundException; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.ShardNotInPrimaryModeException; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.common.ReplicationTimer; +import org.opensearch.node.NodeClosedException; +import org.opensearch.tasks.Task; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportResponseHandler; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * Replication action responsible for publishing merged segment to a replica shard. + * + * @opensearch.api + */ +@ExperimentalApi +public class PublishMergedSegmentAction extends TransportReplicationAction< + PublishMergedSegmentRequest, + PublishMergedSegmentRequest, + ReplicationResponse> { + + public static final String ACTION_NAME = "indices:admin/publish_merged_segment"; + protected static Logger logger = LogManager.getLogger(PublishMergedSegmentAction.class); + + private final SegmentReplicationTargetService replicationService; + + @Inject + public PublishMergedSegmentAction( + Settings settings, + TransportService transportService, + ClusterService clusterService, + IndicesService indicesService, + ThreadPool threadPool, + ShardStateAction shardStateAction, + ActionFilters actionFilters, + SegmentReplicationTargetService targetService + ) { + super( + settings, + ACTION_NAME, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + actionFilters, + PublishMergedSegmentRequest::new, + PublishMergedSegmentRequest::new, + ThreadPool.Names.GENERIC + ); + this.replicationService = targetService; + } + + @Override + protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { + return new ReplicationResponse(in); + } + + @Override + protected void doExecute(Task task, PublishMergedSegmentRequest request, ActionListener listener) { + assert false : "use PublishMergedSegmentAction#publish"; + } + + @Override + public ReplicationMode getReplicationMode(IndexShard indexShard) { + if (indexShard.indexSettings().isAssignedOnRemoteNode()) { + return ReplicationMode.FULL_REPLICATION; + } + return super.getReplicationMode(indexShard); + } + + /** + * Publish merged segment request to shard + */ + final void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint) { + String primaryAllocationId = indexShard.routingEntry().allocationId().getId(); + long primaryTerm = indexShard.getPendingPrimaryTerm(); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the sync is authorized + ThreadContextAccess.doPrivilegedVoid(threadContext::markAsSystemContext); + PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); + final ReplicationTask task = (ReplicationTask) taskManager.register("transport", "segrep_publish_merged_segment", request); + final ReplicationTimer timer = new ReplicationTimer(); + timer.start(); + CountDownLatch latch = new CountDownLatch(1); + transportService.sendChildRequest( + indexShard.recoveryState().getTargetNode(), + transportPrimaryAction, + new ConcreteShardRequest<>(request, primaryAllocationId, primaryTerm), + task, + transportOptions, + new TransportResponseHandler() { + @Override + public ReplicationResponse read(StreamInput in) throws IOException { + return newResponseInstance(in); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public void handleResponse(ReplicationResponse response) { + try { + timer.stop(); + logger.debug( + () -> new ParameterizedMessage( + "[shardId {}] Completed publishing merged segment [{}], timing: {}", + indexShard.shardId().getId(), + checkpoint, + timer.time() + ) + ); + task.setPhase("finished"); + taskManager.unregister(task); + } finally { + latch.countDown(); + } + } + + @Override + public void handleException(TransportException e) { + try { + timer.stop(); + logger.debug( + "[shardId {}] Failed to publish merged segment [{}], timing: {}", + indexShard.shardId().getId(), + checkpoint, + timer.time() + ); + task.setPhase("finished"); + taskManager.unregister(task); + if (ExceptionsHelper.unwrap( + e, + NodeClosedException.class, + IndexNotFoundException.class, + AlreadyClosedException.class, + IndexShardClosedException.class, + ShardNotInPrimaryModeException.class + ) != null) { + // Node is shutting down or the index was deleted or the shard is closed + return; + } + logger.warn( + new ParameterizedMessage("{} merged segment [{}] publishing failed", indexShard.shardId(), checkpoint), + e + ); + } finally { + latch.countDown(); + } + } + } + ); + logger.trace( + () -> new ParameterizedMessage("[shardId {}] Publishing merged segment [{}]", checkpoint.getShardId().getId(), checkpoint) + ); + try { + latch.await(indexShard.getRecoverySettings().getMergedSegmentReplicationTimeout().seconds(), TimeUnit.SECONDS); + } catch (InterruptedException e) { + logger.warn(() -> new ParameterizedMessage("Interrupted while waiting for pre copy merged segment [{}]", checkpoint), e); + } + } + } + + @Override + protected void shardOperationOnPrimary( + PublishMergedSegmentRequest request, + IndexShard primary, + ActionListener> listener + ) { + ActionListener.completeWith(listener, () -> new PrimaryResult<>(request, new ReplicationResponse())); + } + + @Override + protected void shardOperationOnReplica( + PublishMergedSegmentRequest request, + IndexShard replica, + ActionListener listener + ) { + Objects.requireNonNull(request); + Objects.requireNonNull(replica); + ActionListener.completeWith(listener, () -> { + logger.trace(() -> new ParameterizedMessage("Merged segment {} received on replica {}", request, replica.shardId())); + // Condition for ensuring that we ignore Segrep checkpoints received on Docrep shard copies. + // This case will hit iff the replica hosting node is not remote enabled and replication type != SEGMENT + if (replica.indexSettings().isAssignedOnRemoteNode() == false && replica.indexSettings().isSegRepLocalEnabled() == false) { + logger.trace("Received merged segment on a docrep shard copy during an ongoing remote migration. NoOp."); + return new ReplicaResult(); + } + if (request.getMergedSegment().getShardId().equals(replica.shardId())) { + replicationService.onNewMergedSegmentCheckpoint(request.getMergedSegment(), replica); + } + return new ReplicaResult(); + }); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java index a7a1123b0c4f4..b5f183befef63 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java @@ -6,34 +6,9 @@ * compatible open source license. */ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -/* - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - package org.opensearch.indices.replication.checkpoint; -import org.opensearch.action.ActionRequest; -import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.support.replication.ReplicationRequest; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -41,14 +16,15 @@ import java.util.Objects; /** - * Request object for publish merged segment + * Replication request responsible for publishing merged segment request to a replica shard. * * @opensearch.internal */ -public class PublishMergedSegmentRequest extends ActionRequest { +public class PublishMergedSegmentRequest extends ReplicationRequest { private final ReplicationSegmentCheckpoint mergedSegment; public PublishMergedSegmentRequest(ReplicationSegmentCheckpoint mergedSegment) { + super(mergedSegment.getShardId()); this.mergedSegment = mergedSegment; } @@ -57,11 +33,6 @@ public PublishMergedSegmentRequest(StreamInput in) throws IOException { this.mergedSegment = new ReplicationSegmentCheckpoint(in); } - @Override - public ActionRequestValidationException validate() { - return null; - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); 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 8cf77a1bbd240..8a718b288a771 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -131,21 +131,6 @@ protected Settings getIndexSettings() { return settings; } - public void testUpdateActiveReplicaNodes() throws Exception { - try (ReplicationGroup shards = createGroup(0, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { - shards.startAll(); - final IndexShard primaryShard = shards.getPrimary(); - assertEquals(0, primaryShard.getActiveReplicaNodes().size()); - IndexShard replicaShard = shards.addReplica(); - assertEquals(0, primaryShard.getActiveReplicaNodes().size()); - shards.startReplicas(0); - assertEquals(1, primaryShard.getActiveReplicaNodes().size()); - shards.removeReplica(replicaShard); - closeShards(replicaShard); - assertEquals(0, primaryShard.getActiveReplicaNodes().size()); - } - } - /** * Validates happy path of segment replication where primary index docs which are replicated to replica shards. Assertions * made on doc count on both primary and replica. diff --git a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 0d41230f7a20b..d1b3f6e4ef5b4 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -51,6 +51,7 @@ import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.test.OpenSearchSingleNodeTestCase; @@ -168,7 +169,8 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem localNode, null, DiscoveryNodes.builder().add(localNode).build(), - new MergedSegmentWarmerFactory(null, null, null) + new MergedSegmentWarmerFactory(null, null, null), + MergedSegmentPublisher.EMPTY ); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 0ba97f1d5db09..d454f395283a2 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -88,6 +88,7 @@ import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.shard.ShardNotFoundException; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.node.Node; import org.opensearch.test.ClusterServiceUtils; @@ -1422,7 +1423,8 @@ public void testDeleteAndCreateIndexShardOnSameNodeAndVerifyStats() throws Excep localNode, null, DiscoveryNodes.builder().add(localNode).build(), - new MergedSegmentWarmerFactory(null, null, null) + new MergedSegmentWarmerFactory(null, null, null), + MergedSegmentPublisher.EMPTY ); // Verify that the new shard requestStats entries are empty. 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 9cd8bee123dca..2867673fc87b5 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -61,6 +61,7 @@ import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.recovery.RecoveryListener; import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.repositories.RepositoriesService; import org.opensearch.test.OpenSearchTestCase; @@ -268,7 +269,8 @@ public MockIndexShard createShard( final DiscoveryNode sourceNode, final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, final DiscoveryNodes discoveryNodes, - final MergedSegmentWarmerFactory mergedSegmentWarmerFactory + final MergedSegmentWarmerFactory mergedSegmentWarmerFactory, + final MergedSegmentPublisher mergedSegmentPublisher ) throws IOException { failRandomly(); RecoveryState recoveryState = new RecoveryState(shardRouting, targetNode, sourceNode); 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 7cc59aaaa9347..f5b9fb0f6dea1 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/opensearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -68,6 +68,7 @@ import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.replication.SegmentReplicationSourceService; import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.repositories.RepositoriesService; import org.opensearch.telemetry.tracing.noop.NoopTracer; @@ -585,7 +586,8 @@ private IndicesClusterStateService createIndicesClusterStateService( s -> {}, RetentionLeaseSyncer.EMPTY, null, - null + null, + MergedSegmentPublisher.EMPTY ); } 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 11113b49c061f..f9c9b451d3c77 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -52,7 +52,6 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.EmptyTransportResponseHandler; -import org.opensearch.transport.TransportChannel; import org.opensearch.transport.TransportRequestOptions; import org.opensearch.transport.TransportService; import org.junit.Assert; @@ -500,7 +499,7 @@ public void getSegmentFiles( ) ); doReturn(List.of(targetSpy)).when(serviceSpy).getMergedSegmentReplicationTarget(any()); - serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard, mock(TransportChannel.class)); + serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard); // ensure the old target is cancelled. and new iteration kicks off. verify(targetSpy, times(1)).cancel("Cancelling stuck merged segment target after new primary"); verify(serviceSpy, times(1)).startMergedSegmentReplication(eq(replicaShard), any(), any()); @@ -538,11 +537,11 @@ public void getSegmentFiles( ); doReturn(List.of(targetSpy)).when(serviceSpy).getMergedSegmentReplicationTarget(any()); // already exist - serviceSpy.onNewMergedSegmentCheckpoint(checkpoint, replicaShard, mock(TransportChannel.class)); + serviceSpy.onNewMergedSegmentCheckpoint(checkpoint, replicaShard); verify(serviceSpy, times(0)).startMergedSegmentReplication(eq(replicaShard), any(), any()); // new merged segment - serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard, mock(TransportChannel.class)); + serviceSpy.onNewMergedSegmentCheckpoint(newPrimaryCheckpoint, replicaShard); verify(serviceSpy, times(1)).startMergedSegmentReplication(eq(replicaShard), any(), any()); } @@ -561,7 +560,7 @@ public void testNewMergedSegmentCheckpointBehindCurrentCheckpoint() throws IOExc 0L, replicaShard.getLatestReplicationCheckpoint().getCodec() ); - spy.onNewMergedSegmentCheckpoint(oldCheckpoint, replicaShard, mock(TransportChannel.class)); + spy.onNewMergedSegmentCheckpoint(oldCheckpoint, replicaShard); verify(spy, times(0)).startReplication(any(), any(), any()); } @@ -571,7 +570,7 @@ public void testShardNotStarted() throws IOException { spy.onNewCheckpoint(checkpoint, shard); verify(spy, times(0)).startReplication(any(), any(), any()); - spy.onNewMergedSegmentCheckpoint(checkpoint, shard, mock(TransportChannel.class)); + spy.onNewMergedSegmentCheckpoint(checkpoint, shard); verify(spy, times(0)).startMergedSegmentReplication(any(), any(), any()); closeShards(shard); } @@ -590,7 +589,7 @@ public void testRejectCheckpointOnShardPrimaryMode() throws IOException { // Verify that checkpoint is not processed as shard is in PrimaryMode. verify(spy, times(0)).startReplication(any(), any(), any()); - spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, spyShard, mock(TransportChannel.class)); + spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, spyShard); verify(spy, times(0)).startMergedSegmentReplication(any(), any(), any()); closeShards(primaryShard); @@ -672,7 +671,7 @@ public void testOnNewCheckpointInvokedOnClosedShardDoesNothing() throws IOExcept spy.onNewCheckpoint(aheadCheckpoint, replicaShard); verify(spy, times(0)).updateLatestReceivedCheckpoint(any(), any()); - spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, replicaShard, mock(TransportChannel.class)); + spy.onNewMergedSegmentCheckpoint(aheadCheckpoint, replicaShard); verify(spy, times(0)).updateLatestReceivedCheckpoint(any(), any()); } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index bd869b3a1d161..d56ccd3814cc8 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -213,6 +213,7 @@ import org.opensearch.indices.replication.SegmentReplicationSourceFactory; import org.opensearch.indices.replication.SegmentReplicationSourceService; import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.ingest.IngestService; import org.opensearch.ingest.SystemIngestPipelineCache; @@ -2188,7 +2189,8 @@ public void onFailure(final Exception e) { RetentionLeaseSyncer.EMPTY, SegmentReplicationCheckpointPublisher.EMPTY, mock(RemoteStoreStatsTrackerFactory.class), - new MergedSegmentWarmerFactory(null, null, null) + new MergedSegmentWarmerFactory(null, null, null), + MergedSegmentPublisher.EMPTY ); final SystemIndices systemIndices = new SystemIndices(emptyMap()); 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 fc766dc27293e..c56902f73b0b1 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 @@ -39,7 +39,6 @@ import org.apache.lucene.store.IndexInput; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; -import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; @@ -80,7 +79,6 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.indices.breaker.CircuitBreakerService; -import org.opensearch.core.transport.TransportResponse; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.env.Environment; @@ -143,7 +141,7 @@ import org.opensearch.indices.replication.SegmentReplicationState; import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; -import org.opensearch.indices.replication.checkpoint.PublishMergedSegmentRequest; +import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.CopyState; @@ -160,10 +158,8 @@ import org.opensearch.repositories.blobstore.OpenSearchBlobStoreRepositoryIntegTestCase; import org.opensearch.repositories.fs.FsRepository; import org.opensearch.snapshots.Snapshot; -import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.DummyShardLock; import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.test.transport.CapturingTransport; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -200,9 +196,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; /** @@ -701,33 +695,6 @@ protected IndexShard newShard( // This is fine since we are not testing the node stats now Function mockReplicationStatsProvider = mock(Function.class); when(mockReplicationStatsProvider.apply(any())).thenReturn(new ReplicationStats(800, 800, 500)); - - // mock transport service for merged segment warmer - CapturingTransport transport = new CapturingTransport(); - TransportService transportService = transport.createTransportService( - Settings.EMPTY, - mock(ThreadPool.class), - TransportService.NOOP_TRANSPORT_INTERCEPTOR, - boundAddress -> new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT), - null, - Collections.emptySet(), - NoopTracer.INSTANCE - ); - transportService.start(); - transportService.acceptIncomingRequests(); - TransportService spyTransportService = spy(transportService); - doAnswer(invocation -> { - ActionListenerResponseHandler handler = invocation.getArgument(3); - handler.handleResponse(TransportResponse.Empty.INSTANCE); - return mock(TransportResponse.class); - }).when(spyTransportService) - .sendRequest( - any(DiscoveryNode.class), - eq(SegmentReplicationTargetService.Actions.PUBLISH_MERGED_SEGMENT), - any(PublishMergedSegmentRequest.class), - any() - ); - indexShard = new IndexShard( routing, indexSettings, @@ -759,12 +726,13 @@ protected IndexShard newShard( false, discoveryNodes, mockReplicationStatsProvider, - new MergedSegmentWarmerFactory(spyTransportService, new RecoverySettings(nodeSettings, clusterSettings), null), + new MergedSegmentWarmerFactory(null, new RecoverySettings(nodeSettings, clusterSettings), null), false, () -> Boolean.FALSE, indexSettings::getRefreshInterval, new Object(), - clusterService.getClusterApplierService() + clusterService.getClusterApplierService(), + MergedSegmentPublisher.EMPTY ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); if (remoteStoreStatsTrackerFactory != null) { From c8625bb8dfaaca54b1a4ae6a50602eccc8301965 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 4 Jun 2025 11:15:22 +0800 Subject: [PATCH 09/23] reuse Store#getSegmentMetadataMap Signed-off-by: guojialiang --- .../opensearch/index/shard/IndexShard.java | 6 +++- .../org/opensearch/index/store/Store.java | 35 ------------------- .../SegmentReplicationIndexShardTests.java | 5 ++- 3 files changed, 9 insertions(+), 37 deletions(-) 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 7ac60bdf81341..96a4a1e10d9bb 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -57,6 +57,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.ThreadInterruptedException; +import org.apache.lucene.util.Version; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionRunnable; @@ -1869,7 +1870,10 @@ public void publishMergedSegment(SegmentCommitInfo segmentCommitInfo) throws IOE * @throws IOException When there is an error computing segment metadata from the store. */ public ReplicationSegmentCheckpoint computeReplicationSegmentCheckpoint(SegmentCommitInfo segmentCommitInfo) throws IOException { - Map segmentMetadataMap = store.getSegmentMetadataMap(segmentCommitInfo); + // Only need to get the file metadata information in segmentCommitInfo and reuse Store#getSegmentMetadataMap. + SegmentInfos segmentInfos = new SegmentInfos(Version.LATEST.major); + segmentInfos.add(segmentCommitInfo); + Map segmentMetadataMap = store.getSegmentMetadataMap(segmentInfos); return new ReplicationSegmentCheckpoint( shardId, getOperationPrimaryTerm(), 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 5aec02b517ee1..48ed4d9e75dcb 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -389,21 +389,6 @@ public Map getSegmentMetadataMap(SegmentInfos segment } } - /** - * Segment Replication method - Fetch a map of StoreFileMetadata for segmentCommitInfo. - * @param segmentCommitInfo {@link SegmentCommitInfo} from which to compute metadata. - * @return {@link Map} map file name to {@link StoreFileMetadata}. - */ - public Map getSegmentMetadataMap(SegmentCommitInfo segmentCommitInfo) throws IOException { - failIfCorrupted(); - try { - return loadMetadata(segmentCommitInfo, directory, logger); - } catch (NoSuchFileException | CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - markStoreCorrupted(ex); - throw ex; - } - } - /** * Segment Replication method * Returns a diff between the Maps of StoreFileMetadata that can be used for getting list of files to copy over to a replica for segment replication. The returned diff will hold a list of files that are: @@ -1208,26 +1193,6 @@ static LoadedMetadata loadMetadata(SegmentInfos segmentInfos, Directory director return new LoadedMetadata(unmodifiableMap(builder), unmodifiableMap(commitUserDataBuilder), numDocs); } - static Map loadMetadata(SegmentCommitInfo info, Directory directory, Logger logger) throws IOException { - Map builder = new HashMap<>(); - final Version version = info.info.getVersion(); - if (version == null) { - // version is written since 3.1+: we should have already hit IndexFormatTooOld. - throw new IllegalArgumentException("expected valid version value: " + info.info.toString()); - } - for (String file : info.files()) { - checksumFromLuceneFile( - directory, - file, - builder, - logger, - version, - SEGMENT_INFO_EXTENSION.equals(IndexFileNames.getExtension(file)) - ); - } - return unmodifiableMap(builder); - } - private static void checksumFromLuceneFile( Directory directory, String file, 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 8a718b288a771..9cb525efe61fc 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -12,6 +12,7 @@ import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.Version; import org.opensearch.ExceptionsHelper; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; @@ -1029,7 +1030,9 @@ public void testComputeReplicationSegmentCheckpoint() throws Exception { shards.flush(); SegmentInfos segmentInfos = Lucene.readSegmentInfos(primaryShard.store().directory()); for (SegmentCommitInfo segmentCommitInfo : segmentInfos) { - Map segmentMetadataMap = primaryShard.store().getSegmentMetadataMap(segmentCommitInfo); + SegmentInfos tempSegmentInfos = new SegmentInfos(Version.LATEST.major); + tempSegmentInfos.add(segmentCommitInfo); + Map segmentMetadataMap = primaryShard.store().getSegmentMetadataMap(tempSegmentInfos); ReplicationSegmentCheckpoint expectedCheckpoint = new ReplicationSegmentCheckpoint( primaryShard.shardId, primaryShard.getOperationPrimaryTerm(), From 80e5372f8420a98351e04aaed19fbc0ae854a20e Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 4 Jun 2025 11:57:54 +0800 Subject: [PATCH 10/23] refactor updateReplicationRateLimiter Signed-off-by: guojialiang --- .../indices/recovery/RecoverySettings.java | 50 ++++++------------- 1 file changed, 15 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java index 4c7dd361c8ffb..b8b572f8037ef 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySettings.java @@ -252,8 +252,8 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { this.replicationMaxBytesPerSec = INDICES_REPLICATION_MAX_BYTES_PER_SEC_SETTING.get(settings); this.mergedSegmentReplicationMaxBytesPerSec = INDICES_MERGED_SEGMENT_REPLICATION_MAX_BYTES_PER_SEC_SETTING.get(settings); this.mergedSegmentReplicationTimeout = INDICES_MERGED_SEGMENT_REPLICATION_TIMEOUT_SETTING.get(settings); - updateReplicationRateLimiter(); - updateMergedSegmentReplicationRateLimiter(); + replicationRateLimiter = getReplicationRateLimiter(replicationMaxBytesPerSec); + mergedSegmentReplicationRateLimiter = getReplicationRateLimiter(mergedSegmentReplicationMaxBytesPerSec); logger.debug("using recovery max_bytes_per_sec[{}]", recoveryMaxBytesPerSec); this.internalRemoteUploadTimeout = INDICES_INTERNAL_REMOTE_UPLOAD_TIMEOUT.get(settings); @@ -376,31 +376,31 @@ private void setRecoveryMaxBytesPerSec(ByteSizeValue recoveryMaxBytesPerSec) { } else { recoveryRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); } - if (replicationMaxBytesPerSec.getBytes() < 0) updateReplicationRateLimiter(); - if (mergedSegmentReplicationMaxBytesPerSec.getBytes() < 0) updateMergedSegmentReplicationRateLimiter(); + if (replicationMaxBytesPerSec.getBytes() < 0) { + replicationRateLimiter = getReplicationRateLimiter(replicationMaxBytesPerSec); + } + if (mergedSegmentReplicationMaxBytesPerSec.getBytes() < 0) { + mergedSegmentReplicationRateLimiter = getReplicationRateLimiter(mergedSegmentReplicationMaxBytesPerSec); + } } private void setReplicationMaxBytesPerSec(ByteSizeValue replicationMaxBytesPerSec) { this.replicationMaxBytesPerSec = replicationMaxBytesPerSec; - updateReplicationRateLimiter(); + replicationRateLimiter = getReplicationRateLimiter(replicationMaxBytesPerSec); } - private void updateReplicationRateLimiter() { + private SimpleRateLimiter getReplicationRateLimiter(ByteSizeValue replicationMaxBytesPerSec) { if (replicationMaxBytesPerSec.getBytes() >= 0) { if (replicationMaxBytesPerSec.getBytes() == 0) { - replicationRateLimiter = null; - } else if (replicationRateLimiter != null) { - replicationRateLimiter.setMBPerSec(replicationMaxBytesPerSec.getMbFrac()); + return null; } else { - replicationRateLimiter = new SimpleRateLimiter(replicationMaxBytesPerSec.getMbFrac()); + return new SimpleRateLimiter(replicationMaxBytesPerSec.getMbFrac()); } } else { // when replicationMaxBytesPerSec = -1B, use setting of recovery if (recoveryMaxBytesPerSec.getBytes() <= 0) { - replicationRateLimiter = null; - } else if (replicationRateLimiter != null) { - replicationRateLimiter.setMBPerSec(recoveryMaxBytesPerSec.getMbFrac()); + return null; } else { - replicationRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); + return new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); } } } @@ -411,33 +411,13 @@ public TimeValue getMergedSegmentReplicationTimeout() { private void setMergedSegmentReplicationMaxBytesPerSec(ByteSizeValue mergedSegmentReplicationMaxBytesPerSec) { this.mergedSegmentReplicationMaxBytesPerSec = mergedSegmentReplicationMaxBytesPerSec; - updateMergedSegmentReplicationRateLimiter(); + mergedSegmentReplicationRateLimiter = getReplicationRateLimiter(mergedSegmentReplicationMaxBytesPerSec); } public void setMergedSegmentReplicationTimeout(TimeValue mergedSegmentReplicationTimeout) { this.mergedSegmentReplicationTimeout = mergedSegmentReplicationTimeout; } - private void updateMergedSegmentReplicationRateLimiter() { - if (mergedSegmentReplicationMaxBytesPerSec.getBytes() >= 0) { - if (mergedSegmentReplicationMaxBytesPerSec.getBytes() == 0) { - mergedSegmentReplicationRateLimiter = null; - } else if (mergedSegmentReplicationRateLimiter != null) { - mergedSegmentReplicationRateLimiter.setMBPerSec(mergedSegmentReplicationMaxBytesPerSec.getMbFrac()); - } else { - mergedSegmentReplicationRateLimiter = new SimpleRateLimiter(mergedSegmentReplicationMaxBytesPerSec.getMbFrac()); - } - } else { // when mergeReplicationMaxBytesPerSec = -1B, use setting of recovery - if (recoveryMaxBytesPerSec.getBytes() <= 0) { - mergedSegmentReplicationRateLimiter = null; - } else if (mergedSegmentReplicationRateLimiter != null) { - mergedSegmentReplicationRateLimiter.setMBPerSec(recoveryMaxBytesPerSec.getMbFrac()); - } else { - mergedSegmentReplicationRateLimiter = new SimpleRateLimiter(recoveryMaxBytesPerSec.getMbFrac()); - } - } - } - public int getMaxConcurrentFileChunks() { return maxConcurrentFileChunks; } From 73d6616bb93deb7da1afddc83ad67f3ddce57e28 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 4 Jun 2025 16:58:08 +0800 Subject: [PATCH 11/23] extract an abstract base class (AbstractSegmentReplicationTarget) from SegmentReplicationTarget Signed-off-by: guojialiang --- .../AbstractSegmentReplicationTarget.java | 284 ++++++++++++++++++ .../MergedSegmentReplicationTarget.java | 52 ++-- .../replication/SegmentReplicationTarget.java | 256 ++-------------- .../replication/SegmentReplicator.java | 12 +- .../index/shard/IndexShardTestCase.java | 4 +- 5 files changed, 340 insertions(+), 268 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java diff --git a/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java new file mode 100644 index 0000000000000..2275034f1ec81 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/AbstractSegmentReplicationTarget.java @@ -0,0 +1,284 @@ +/* + * 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.message.ParameterizedMessage; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.opensearch.OpenSearchCorruptionException; +import org.opensearch.action.StepListener; +import org.opensearch.common.util.CancellableThreads; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.MultiFileWriter; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationFailedException; +import org.opensearch.indices.replication.common.ReplicationListener; +import org.opensearch.indices.replication.common.ReplicationLuceneIndex; +import org.opensearch.indices.replication.common.ReplicationTarget; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.function.BiConsumer; +import java.util.stream.Collectors; + +/** + * Abstract base class for segment replication. + * + * @opensearch.internal + */ +public abstract class AbstractSegmentReplicationTarget extends ReplicationTarget { + + protected final ReplicationCheckpoint checkpoint; + protected final SegmentReplicationSource source; + protected final SegmentReplicationState state; + protected final MultiFileWriter multiFileWriter; + + public AbstractSegmentReplicationTarget( + String name, + IndexShard indexShard, + ReplicationCheckpoint checkpoint, + SegmentReplicationSource source, + ReplicationListener listener + ) { + super(name, indexShard, new ReplicationLuceneIndex(), listener); + this.checkpoint = checkpoint; + this.source = source; + this.state = new SegmentReplicationState( + indexShard.routingEntry(), + stateIndex, + getId(), + source.getDescription(), + indexShard.recoveryState().getTargetNode() + ); + this.multiFileWriter = new MultiFileWriter(indexShard.store(), stateIndex, getPrefix(), logger, this::ensureRefCount); + } + + @Override + protected void closeInternal() { + try { + multiFileWriter.close(); + } finally { + super.closeInternal(); + } + } + + @Override + protected void onCancel(String reason) { + try { + notifyListener(new ReplicationFailedException(reason), false); + } finally { + source.cancel(); + cancellableThreads.cancel(reason); + } + } + + @Override + protected void onDone() { + state.setStage(SegmentReplicationState.Stage.DONE); + } + + @Override + public SegmentReplicationState state() { + return state; + } + + @Override + public String description() { + return String.format( + Locale.ROOT, + "Id:[%d] Checkpoint [%s] Shard:[%s] Source:[%s]", + getId(), + getCheckpoint(), + shardId(), + source.getDescription() + ); + } + + @Override + public void notifyListener(ReplicationFailedException e, boolean sendShardFailure) { + listener.onFailure(state(), e, sendShardFailure); + } + + @Override + public boolean reset(CancellableThreads newTargetCancellableThreads) throws IOException { + // TODO + return false; + } + + public ReplicationCheckpoint getCheckpoint() { + return this.checkpoint; + } + + @Override + public void writeFileChunk( + StoreFileMetadata metadata, + long position, + BytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener + ) { + try { + multiFileWriter.writeFileChunk(metadata, position, content, lastChunk); + listener.onResponse(null); + } catch (Exception e) { + listener.onFailure(e); + } + } + + /** + * Start the Replication event. + * + * @param listener {@link ActionListener} listener. + */ + public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { + cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { + throw new CancellableThreads.ExecutionCancelledException("replication was canceled reason [" + reason + "]"); + }); + // TODO: Remove this useless state. + state.setStage(SegmentReplicationState.Stage.REPLICATING); + final StepListener checkpointInfoListener = new StepListener<>(); + final StepListener getFilesListener = new StepListener<>(); + + logger.trace(new ParameterizedMessage("Starting Replication Target: {}", description())); + // Get list of files to copy from this checkpoint. + state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); + cancellableThreads.checkForCancel(); + getCheckpointMetadata(checkpointInfoListener); + + checkpointInfoListener.whenComplete(checkpointInfo -> { + updateCheckpoint(checkpointInfo.getCheckpoint(), checkpointUpdater); + final List filesToFetch = getFiles(checkpointInfo); + state.setStage(SegmentReplicationState.Stage.GET_FILES); + cancellableThreads.checkForCancel(); + getFilesFromSource(checkpointInfo, filesToFetch, getFilesListener); + }, listener::onFailure); + + getFilesListener.whenComplete(response -> { + cancellableThreads.checkForCancel(); + state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + finalizeReplication(checkpointInfoListener.result()); + listener.onResponse(null); + }, listener::onFailure); + } + + protected abstract void getCheckpointMetadata(StepListener checkpointInfoListener); + + protected abstract void updateCheckpoint( + ReplicationCheckpoint checkpoint, + BiConsumer checkpointUpdater + ); + + protected abstract void getFilesFromSource( + CheckpointInfoResponse checkpointInfo, + List filesToFetch, + StepListener getFilesListener + ); + + protected abstract void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws Exception; + + protected List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { + cancellableThreads.checkForCancel(); + state.setStage(SegmentReplicationState.Stage.FILE_DIFF); + + // Return an empty list for warm indices, In this case, replica shards don't require downloading files from remote storage + // as replicas will sync all files from remote in case of failure. + if (indexShard.indexSettings().isWarmIndex()) { + return Collections.emptyList(); + } + final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); + // local files + final Set localFiles = Set.of(indexShard.store().directory().listAll()); + // set of local files that can be reused + final Set reuseFiles = diff.missing.stream() + .filter(storeFileMetadata -> localFiles.contains(storeFileMetadata.name())) + .filter(this::validateLocalChecksum) + .map(StoreFileMetadata::name) + .collect(Collectors.toSet()); + + final List missingFiles = diff.missing.stream() + .filter(md -> reuseFiles.contains(md.name()) == false) + .collect(Collectors.toList()); + + logger.trace( + () -> new ParameterizedMessage( + "Replication diff for checkpoint {} {} {}", + checkpointInfo.getCheckpoint(), + missingFiles, + diff.different + ) + ); + /* + * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming + * snapshot from source that means the local copy of the segment has been corrupted/changed in some way and we throw an + * IllegalStateException to fail the shard + */ + if (diff.different.isEmpty() == false) { + throw new OpenSearchCorruptionException( + new ParameterizedMessage( + "Shard {} has local copies of segments that differ from the primary {}", + indexShard.shardId(), + diff.different + ).getFormattedMessage() + ); + } + + for (StoreFileMetadata file : missingFiles) { + state.getIndex().addFileDetail(file.name(), file.length(), false); + } + return missingFiles; + } + + // pkg private for tests + private boolean validateLocalChecksum(StoreFileMetadata file) { + try (IndexInput indexInput = indexShard.store().directory().openInput(file.name(), IOContext.READONCE)) { + String checksum = Store.digestToString(CodecUtil.retrieveChecksum(indexInput)); + if (file.checksum().equals(checksum)) { + return true; + } else { + // clear local copy with mismatch. Safe because file is not referenced by active reader. + store.deleteQuiet(file.name()); + return false; + } + } catch (IOException e) { + logger.warn("Error reading " + file, e); + // Delete file on exceptions so that it can be re-downloaded. This is safe to do as this file is local only + // and not referenced by reader. + try { + indexShard.store().directory().deleteFile(file.name()); + } catch (IOException ex) { + throw new UncheckedIOException("Error reading " + file, e); + } + return false; + } + } + + /** + * Updates the state to reflect recovery progress for the given file and + * updates the last access time for the target. + * @param fileName Name of the file being downloaded + * @param bytesRecovered Number of bytes recovered + */ + protected void updateFileRecoveryBytes(String fileName, long bytesRecovered) { + ReplicationLuceneIndex index = state.getIndex(); + if (index != null) { + index.addRecoveredBytesToFile(fileName, bytesRecovered); + } + setLastAccessTime(); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java index 08fdd00ec030d..75cd3985ffc88 100644 --- a/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/MergedSegmentReplicationTarget.java @@ -32,11 +32,8 @@ package org.opensearch.indices.replication; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.StepListener; import org.opensearch.common.UUIDs; -import org.opensearch.common.util.CancellableThreads; -import org.opensearch.core.action.ActionListener; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; @@ -50,7 +47,7 @@ * * @opensearch.internal */ -public class MergedSegmentReplicationTarget extends SegmentReplicationTarget { +public class MergedSegmentReplicationTarget extends AbstractSegmentReplicationTarget { public final static String MERGE_REPLICATION_PREFIX = "merge."; public MergedSegmentReplicationTarget( @@ -59,7 +56,7 @@ public MergedSegmentReplicationTarget( SegmentReplicationSource source, ReplicationListener listener ) { - super(indexShard, checkpoint, source, listener); + super("merged_segment_replication_target", indexShard, checkpoint, source, listener); } @Override @@ -68,32 +65,29 @@ protected String getPrefix() { } @Override - public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { - state.setStage(SegmentReplicationState.Stage.REPLICATING); - cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { - throw new CancellableThreads.ExecutionCancelledException("merge replication was canceled reason [" + reason + "]"); - }); - - final StepListener getFilesListener = new StepListener<>(); + protected void getCheckpointMetadata(StepListener checkpointInfoListener) { + checkpointInfoListener.onResponse(new CheckpointInfoResponse(checkpoint, checkpoint.getMetadataMap(), null)); + } - logger.trace(new ParameterizedMessage("Starting Merge Replication Target: {}", description())); + @Override + protected void updateCheckpoint(ReplicationCheckpoint checkpoint, BiConsumer checkpointUpdater) {} - state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); - List filesToFetch; - try { - filesToFetch = getFiles(new CheckpointInfoResponse(checkpoint, checkpoint.getMetadataMap(), null)); - } catch (Exception e) { - listener.onFailure(e); - return; - } - state.setStage(SegmentReplicationState.Stage.GET_FILES); - cancellableThreads.checkForCancel(); + @Override + protected void getFilesFromSource( + CheckpointInfoResponse checkpointInfo, + List filesToFetch, + StepListener getFilesListener + ) { source.getMergedSegmentFiles(getId(), checkpoint, filesToFetch, indexShard, this::updateFileRecoveryBytes, getFilesListener); - getFilesListener.whenComplete(response -> { - state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); - cancellableThreads.checkForCancel(); - multiFileWriter.renameAllTempFiles(); - listener.onResponse(null); - }, listener::onFailure); + } + + @Override + protected void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws Exception { + multiFileWriter.renameAllTempFiles(); + } + + @Override + public MergedSegmentReplicationTarget retryCopy() { + return new MergedSegmentReplicationTarget(indexShard, checkpoint, source, listener); } } 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 a75623393f5ca..b4a40ba78ea42 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -8,53 +8,32 @@ package org.opensearch.indices.replication; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.action.StepListener; import org.opensearch.common.UUIDs; import org.opensearch.common.lucene.Lucene; import org.opensearch.common.util.CancellableThreads; -import org.opensearch.core.action.ActionListener; -import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.recovery.MultiFileWriter; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationFailedException; import org.opensearch.indices.replication.common.ReplicationListener; -import org.opensearch.indices.replication.common.ReplicationLuceneIndex; -import org.opensearch.indices.replication.common.ReplicationTarget; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Collections; import java.util.List; -import java.util.Locale; -import java.util.Set; import java.util.function.BiConsumer; -import java.util.stream.Collectors; /** * Represents the target of a replication event. * * @opensearch.internal */ -public class SegmentReplicationTarget extends ReplicationTarget { - - protected final ReplicationCheckpoint checkpoint; - protected final SegmentReplicationSource source; - protected final SegmentReplicationState state; - protected final MultiFileWriter multiFileWriter; - +public class SegmentReplicationTarget extends AbstractSegmentReplicationTarget { public final static String REPLICATION_PREFIX = "replication."; public SegmentReplicationTarget( @@ -63,36 +42,7 @@ public SegmentReplicationTarget( SegmentReplicationSource source, ReplicationListener listener ) { - super("replication_target", indexShard, new ReplicationLuceneIndex(), listener); - this.checkpoint = checkpoint; - this.source = source; - this.state = new SegmentReplicationState( - indexShard.routingEntry(), - stateIndex, - getId(), - source.getDescription(), - indexShard.recoveryState().getTargetNode() - ); - this.multiFileWriter = new MultiFileWriter(indexShard.store(), stateIndex, getPrefix(), logger, this::ensureRefCount); - } - - @Override - protected void closeInternal() { - try { - multiFileWriter.close(); - } finally { - super.closeInternal(); - } - } - - @Override - protected void onCancel(String reason) { - try { - notifyListener(new ReplicationFailedException(reason), false); - } finally { - source.cancel(); - cancellableThreads.cancel(reason); - } + super("replication_target", indexShard, checkpoint, source, listener); } @Override @@ -101,198 +51,33 @@ protected String getPrefix() { } @Override - protected void onDone() { - state.setStage(SegmentReplicationState.Stage.DONE); + protected void getCheckpointMetadata(StepListener checkpointInfoListener) { + source.getCheckpointMetadata(getId(), checkpoint, checkpointInfoListener); } @Override - public SegmentReplicationState state() { - return state; - } - - public SegmentReplicationTarget retryCopy() { - return new SegmentReplicationTarget(indexShard, checkpoint, source, listener); + protected void updateCheckpoint(ReplicationCheckpoint checkpoint, BiConsumer checkpointUpdater) { + checkpointUpdater.accept(checkpoint, this.indexShard); } @Override - public String description() { - return String.format( - Locale.ROOT, - "Id:[%d] Checkpoint [%s] Shard:[%s] Source:[%s]", + protected void getFilesFromSource( + CheckpointInfoResponse checkpointInfo, + List filesToFetch, + StepListener getFilesListener + ) { + source.getSegmentFiles( getId(), - getCheckpoint(), - shardId(), - source.getDescription() + checkpointInfo.getCheckpoint(), + filesToFetch, + indexShard, + this::updateFileRecoveryBytes, + getFilesListener ); } @Override - public void notifyListener(ReplicationFailedException e, boolean sendShardFailure) { - listener.onFailure(state(), e, sendShardFailure); - } - - @Override - public boolean reset(CancellableThreads newTargetCancellableThreads) throws IOException { - // TODO - return false; - } - - public ReplicationCheckpoint getCheckpoint() { - return this.checkpoint; - } - - @Override - public void writeFileChunk( - StoreFileMetadata metadata, - long position, - BytesReference content, - boolean lastChunk, - int totalTranslogOps, - ActionListener listener - ) { - try { - multiFileWriter.writeFileChunk(metadata, position, content, lastChunk); - listener.onResponse(null); - } catch (Exception e) { - listener.onFailure(e); - } - } - - /** - * Start the Replication event. - * - * @param listener {@link ActionListener} listener. - */ - public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { - cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { - throw new CancellableThreads.ExecutionCancelledException("replication was canceled reason [" + reason + "]"); - }); - // TODO: Remove this useless state. - state.setStage(SegmentReplicationState.Stage.REPLICATING); - final StepListener checkpointInfoListener = new StepListener<>(); - final StepListener getFilesListener = new StepListener<>(); - - logger.trace(new ParameterizedMessage("Starting Replication Target: {}", description())); - // Get list of files to copy from this checkpoint. - state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); - cancellableThreads.checkForCancel(); - source.getCheckpointMetadata(getId(), checkpoint, checkpointInfoListener); - - checkpointInfoListener.whenComplete(checkpointInfo -> { - checkpointUpdater.accept(checkpointInfo.getCheckpoint(), this.indexShard); - - final List filesToFetch = getFiles(checkpointInfo); - state.setStage(SegmentReplicationState.Stage.GET_FILES); - cancellableThreads.checkForCancel(); - source.getSegmentFiles( - getId(), - checkpointInfo.getCheckpoint(), - filesToFetch, - indexShard, - this::updateFileRecoveryBytes, - getFilesListener - ); - }, listener::onFailure); - - getFilesListener.whenComplete(response -> { - finalizeReplication(checkpointInfoListener.result()); - listener.onResponse(null); - }, listener::onFailure); - } - - protected List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { - cancellableThreads.checkForCancel(); - state.setStage(SegmentReplicationState.Stage.FILE_DIFF); - - // Return an empty list for warm indices, In this case, replica shards don't require downloading files from remote storage - // as replicas will sync all files from remote in case of failure. - if (indexShard.indexSettings().isWarmIndex()) { - return Collections.emptyList(); - } - final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); - // local files - final Set localFiles = Set.of(indexShard.store().directory().listAll()); - // set of local files that can be reused - final Set reuseFiles = diff.missing.stream() - .filter(storeFileMetadata -> localFiles.contains(storeFileMetadata.name())) - .filter(this::validateLocalChecksum) - .map(StoreFileMetadata::name) - .collect(Collectors.toSet()); - - final List missingFiles = diff.missing.stream() - .filter(md -> reuseFiles.contains(md.name()) == false) - .collect(Collectors.toList()); - - logger.trace( - () -> new ParameterizedMessage( - "Replication diff for checkpoint {} {} {}", - checkpointInfo.getCheckpoint(), - missingFiles, - diff.different - ) - ); - /* - * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming - * snapshot from source that means the local copy of the segment has been corrupted/changed in some way and we throw an - * IllegalStateException to fail the shard - */ - if (diff.different.isEmpty() == false) { - throw new OpenSearchCorruptionException( - new ParameterizedMessage( - "Shard {} has local copies of segments that differ from the primary {}", - indexShard.shardId(), - diff.different - ).getFormattedMessage() - ); - } - - for (StoreFileMetadata file : missingFiles) { - state.getIndex().addFileDetail(file.name(), file.length(), false); - } - return missingFiles; - } - - // pkg private for tests - private boolean validateLocalChecksum(StoreFileMetadata file) { - try (IndexInput indexInput = indexShard.store().directory().openInput(file.name(), IOContext.READONCE)) { - String checksum = Store.digestToString(CodecUtil.retrieveChecksum(indexInput)); - if (file.checksum().equals(checksum)) { - return true; - } else { - // clear local copy with mismatch. Safe because file is not referenced by active reader. - store.deleteQuiet(file.name()); - return false; - } - } catch (IOException e) { - logger.warn("Error reading " + file, e); - // Delete file on exceptions so that it can be re-downloaded. This is safe to do as this file is local only - // and not referenced by reader. - try { - indexShard.store().directory().deleteFile(file.name()); - } catch (IOException ex) { - throw new UncheckedIOException("Error reading " + file, e); - } - return false; - } - } - - /** - * Updates the state to reflect recovery progress for the given file and - * updates the last access time for the target. - * @param fileName Name of the file being downloaded - * @param bytesRecovered Number of bytes recovered - */ - protected void updateFileRecoveryBytes(String fileName, long bytesRecovered) { - ReplicationLuceneIndex index = state.getIndex(); - if (index != null) { - index.addRecoveredBytesToFile(fileName, bytesRecovered); - } - setLastAccessTime(); - } - - private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws OpenSearchCorruptionException { - cancellableThreads.checkForCancel(); - state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + protected void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws Exception { // Handle empty SegmentInfos bytes for recovering replicas if (checkpointInfoResponse.getInfosBytes() == null) { return; @@ -340,4 +125,9 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) } } } + + @Override + public SegmentReplicationTarget retryCopy() { + return new SegmentReplicationTarget(indexShard, checkpoint, source, listener); + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java index b653eda3cc5f4..d1683f57315f4 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -280,7 +280,7 @@ public long getTimestamp() { /** * Runnable implementation to trigger a replication event. */ - private class ReplicationRunner extends AbstractRunnable { + private class ReplicationRunner extends AbstractRunnable { final long replicationId; final ReplicationCollection onGoingReplications; @@ -309,12 +309,14 @@ public void doRun() { private void start( final long replicationId, - ReplicationCollection onGoingReplications, + ReplicationCollection onGoingReplications, Map completedReplications ) { - final SegmentReplicationTarget target; + final AbstractSegmentReplicationTarget target; try ( - ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.get(replicationId) + ReplicationCollection.ReplicationRef replicationRef = onGoingReplications.get( + replicationId + ) ) { // This check is for handling edge cases where the reference is removed before the ReplicationRunner is started by the // threadpool. @@ -362,7 +364,7 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) threadPool.generic().execute(new ReplicationRunner(replicationId, onGoingReplications, completedReplications)); } - private boolean isStoreCorrupt(SegmentReplicationTarget target) { + private boolean isStoreCorrupt(AbstractSegmentReplicationTarget target) { // ensure target is not already closed. In that case // we can assume the store is not corrupt and that the replication // event completed successfully. 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 c56902f73b0b1..d42a212d5e5ac 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 @@ -133,6 +133,7 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.recovery.StartRecoveryRequest; +import org.opensearch.indices.replication.AbstractSegmentReplicationTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; import org.opensearch.indices.replication.MergedSegmentReplicationTarget; @@ -1808,7 +1809,8 @@ protected final void replicateMergedSegments(IndexShard primaryShard, List Date: Wed, 4 Jun 2025 21:28:06 +0800 Subject: [PATCH 12/23] reduce unnecessary exception judgment Signed-off-by: guojialiang --- .../SegmentReplicationSourceService.java | 33 ++----------------- 1 file changed, 2 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index 4b8fbef18c861..377319860b9ce 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -25,12 +25,9 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; -import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardClosedException; -import org.opensearch.index.shard.IndexShardNotStartedException; import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; @@ -191,35 +188,9 @@ public void messageReceived(GetSegmentFilesRequest request, TransportChannel cha ); final ShardId shardId = request.getCheckpoint().getShardId(); IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); - if (indexService == null) { - listener.onFailure(new IndexNotFoundException(shardId.getIndexName())); - return; - } IndexShard indexShard = indexService.getShard(shardId.id()); - if (indexShard == null || indexShard.state().equals(IndexShardState.CLOSED)) { - listener.onFailure(new IndexShardClosedException(shardId)); - return; - } - if (indexShard.state().equals(IndexShardState.STARTED) == false) { - listener.onFailure(new IndexShardNotStartedException(shardId, indexShard.state())); - return; - } - if (indexShard.routingEntry().primary() == false || indexShard.isPrimaryMode() == false) { - listener.onFailure(new IllegalArgumentException(String.format(Locale.ROOT, "%s is not primary", shardId))); - return; - } - if (indexShard.getOperationPrimaryTerm() > request.getCheckpoint().getPrimaryTerm()) { - listener.onFailure( - new IllegalArgumentException( - String.format( - Locale.ROOT, - "request primary term %d is lower than %d", - request.getCheckpoint().getPrimaryTerm(), - indexShard.getOperationPrimaryTerm() - ) - ) - ); - return; + if (indexShard.state().equals(IndexShardState.STARTED) == false || indexShard.isPrimaryMode() == false) { + throw new IllegalStateException(String.format(Locale.ROOT, "%s is not a started primary shard", shardId)); } RemoteSegmentFileChunkWriter mergedSegmentFileChunkWriter = new RemoteSegmentFileChunkWriter( From 5d18ed16b72e00b3f600826759e78645cdf6b32d Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 4 Jun 2025 23:42:08 +0800 Subject: [PATCH 13/23] fix UT Signed-off-by: guojialiang --- .../SegmentReplicationSourceServiceTests.java | 54 ++----------------- 1 file changed, 4 insertions(+), 50 deletions(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java index 7de90b34e8fc2..3e2054af29841 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java @@ -20,11 +20,9 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; -import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardClosedException; import org.opensearch.index.shard.IndexShardState; import org.opensearch.index.shard.ReplicationGroup; import org.opensearch.indices.IndicesService; @@ -179,28 +177,6 @@ public void onFailure(Exception e) { }); } - public void testGetMergedSegmentFiles_indexNotFound() { - when(mockIndicesService.indexServiceSafe(mockIndexShard.shardId().getIndex())).thenReturn(null); - final GetSegmentFilesRequest request = new GetSegmentFilesRequest( - 1, - "allocationId", - localNode, - Collections.emptyList(), - testCheckpoint - ); - executeGetMergedSegmentFiles(request, new ActionListener<>() { - @Override - public void onResponse(GetSegmentFilesResponse response) { - Assert.fail("Test should succeed"); - } - - @Override - public void onFailure(Exception e) { - assert e.getCause() instanceof IndexNotFoundException; - } - }); - } - public void testGetMergedSegmentFiles_shardClosed() { when(mockIndexShard.state()).thenReturn(IndexShardState.CLOSED); final GetSegmentFilesRequest request = new GetSegmentFilesRequest( @@ -213,12 +189,12 @@ public void testGetMergedSegmentFiles_shardClosed() { executeGetMergedSegmentFiles(request, new ActionListener<>() { @Override public void onResponse(GetSegmentFilesResponse response) { - Assert.fail("Test should succeed"); + Assert.fail("Test should fail"); } @Override public void onFailure(Exception e) { - assert e.getCause() instanceof IndexShardClosedException; + assert e.getCause() instanceof IllegalStateException; } }); } @@ -235,34 +211,12 @@ public void testGetMergedSegmentFiles_shardNonPrimary() { executeGetMergedSegmentFiles(request, new ActionListener<>() { @Override public void onResponse(GetSegmentFilesResponse response) { - Assert.fail("Test should succeed"); - } - - @Override - public void onFailure(Exception e) { - assert e.getCause() instanceof IllegalArgumentException; - } - }); - } - - public void testGetMergedSegmentFiles_receiveLowerPrimaryTermRequest() { - final GetSegmentFilesRequest request = new GetSegmentFilesRequest( - 1, - "allocationId", - localNode, - Collections.emptyList(), - testCheckpoint - ); - when(mockIndexShard.getOperationPrimaryTerm()).thenReturn(request.getCheckpoint().getPrimaryTerm() + 1); - executeGetMergedSegmentFiles(request, new ActionListener<>() { - @Override - public void onResponse(GetSegmentFilesResponse response) { - Assert.fail("Test should succeed"); + Assert.fail("Test should fail"); } @Override public void onFailure(Exception e) { - assert e.getCause() instanceof IllegalArgumentException; + assert e.getCause() instanceof IllegalStateException; } }); } From 779b532d62fe3f819329e4b0c90c85078955d6fa Mon Sep 17 00:00:00 2001 From: guojialiang Date: Thu, 5 Jun 2025 11:02:56 +0800 Subject: [PATCH 14/23] add PublishMergedSegmentActionTests Signed-off-by: guojialiang --- .../SegmentReplicationIndexShardTests.java | 30 --- .../PublishMergedSegmentActionTests.java | 232 ++++++++++++++++++ 2 files changed, 232 insertions(+), 30 deletions(-) create mode 100644 server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java 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 9cb525efe61fc..8b24b963da6fa 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -9,10 +9,8 @@ package org.opensearch.index.shard; import org.apache.lucene.codecs.Codec; -import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.util.Version; import org.opensearch.ExceptionsHelper; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; @@ -28,7 +26,6 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lease.Releasable; -import org.opensearch.common.lucene.Lucene; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -64,7 +61,6 @@ import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.CopyState; import org.opensearch.indices.replication.common.ReplicationFailedException; @@ -1021,32 +1017,6 @@ public void testReuseReplicationCheckpointWhenLatestInfosIsUnChanged() throws Ex } } - public void testComputeReplicationSegmentCheckpoint() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { - final IndexShard primaryShard = shards.getPrimary(); - shards.startAll(); - shards.indexDocs(10); - shards.refresh("test"); - shards.flush(); - SegmentInfos segmentInfos = Lucene.readSegmentInfos(primaryShard.store().directory()); - for (SegmentCommitInfo segmentCommitInfo : segmentInfos) { - SegmentInfos tempSegmentInfos = new SegmentInfos(Version.LATEST.major); - tempSegmentInfos.add(segmentCommitInfo); - Map segmentMetadataMap = primaryShard.store().getSegmentMetadataMap(tempSegmentInfos); - ReplicationSegmentCheckpoint expectedCheckpoint = new ReplicationSegmentCheckpoint( - primaryShard.shardId, - primaryShard.getOperationPrimaryTerm(), - segmentMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), - primaryShard.getDefaultCodecName(), - segmentMetadataMap, - segmentCommitInfo.info.name - ); - ReplicationSegmentCheckpoint checkpoint = primaryShard.computeReplicationSegmentCheckpoint(segmentCommitInfo); - assertEquals(expectedCheckpoint, checkpoint); - } - } - } - public void testComputeReplicationCheckpointNullInfosReturnsEmptyCheckpoint() throws Exception { try (ReplicationGroup shards = createGroup(1, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { final IndexShard primaryShard = shards.getPrimary(); diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java new file mode 100644 index 0000000000000..0533f47151568 --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java @@ -0,0 +1,232 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.apache.lucene.codecs.Codec; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.action.support.replication.TransportReplicationAction; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexService; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.telemetry.tracing.noop.NoopTracer; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.util.Collections; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.*; +import static org.opensearch.index.remote.RemoteStoreTestsHelper.createIndexSettings; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; + +public class PublishMergedSegmentActionTests extends OpenSearchTestCase { + + private ThreadPool threadPool; + private CapturingTransport transport; + private ClusterService clusterService; + private TransportService transportService; + private ShardStateAction shardStateAction; + + @Override + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getClass().getName()); + transport = new CapturingTransport(); + clusterService = createClusterService(threadPool); + transportService = transport.createTransportService( + clusterService.getSettings(), + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> clusterService.localNode(), + null, + Collections.emptySet(), + NoopTracer.INSTANCE + ); + transportService.start(); + transportService.acceptIncomingRequests(); + shardStateAction = new ShardStateAction(clusterService, transportService, null, null, threadPool); + } + + @Override + public void tearDown() throws Exception { + try { + IOUtils.close(transportService, clusterService, transport); + } finally { + terminate(threadPool); + } + super.tearDown(); + } + + public void testPublishMergedSegmentActionOnPrimary() { + final IndicesService indicesService = mock(IndicesService.class); + + final Index index = new Index("index", "uuid"); + final IndexService indexService = mock(IndexService.class); + when(indicesService.indexServiceSafe(index)).thenReturn(indexService); + + final int id = randomIntBetween(0, 4); + final IndexShard indexShard = mock(IndexShard.class); + when(indexService.getShard(id)).thenReturn(indexShard); + + final ShardId shardId = new ShardId(index, id); + when(indexShard.shardId()).thenReturn(shardId); + + final SegmentReplicationTargetService mockTargetService = mock(SegmentReplicationTargetService.class); + + final PublishMergedSegmentAction action = new PublishMergedSegmentAction( + Settings.EMPTY, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mockTargetService + ); + + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); + + action.shardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { + // we should forward the request containing the current publish checkpoint to the replica + assertThat(result.replicaRequest(), sameInstance(request)); + })); + } + + public void testPublishMergedSegmentActionOnReplica() { + final IndicesService indicesService = mock(IndicesService.class); + + final Index index = new Index("index", "uuid"); + final IndexService indexService = mock(IndexService.class); + when(indicesService.indexServiceSafe(index)).thenReturn(indexService); + final int id = randomIntBetween(0, 4); + final IndexShard indexShard = mock(IndexShard.class); + when(indexService.getShard(id)).thenReturn(indexShard); + + final ShardId shardId = new ShardId(index, id); + when(indexShard.shardId()).thenReturn(shardId); + when(indexShard.indexSettings()).thenReturn( + createIndexSettings(false, Settings.builder().put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), "SEGMENT").build()) + ); + final SegmentReplicationTargetService mockTargetService = mock(SegmentReplicationTargetService.class); + + final PublishMergedSegmentAction action = new PublishMergedSegmentAction( + Settings.EMPTY, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mockTargetService + ); + + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + + final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); + + final PlainActionFuture listener = PlainActionFuture.newFuture(); + action.shardOperationOnReplica(request, indexShard, listener); + final TransportReplicationAction.ReplicaResult result = listener.actionGet(); + + // onNewMergedSegmentCheckpoint should be called on shard with checkpoint request + verify(mockTargetService, times(1)).onNewMergedSegmentCheckpoint(checkpoint, indexShard); + + // the result should indicate success + final AtomicBoolean success = new AtomicBoolean(); + result.runPostReplicaActions(ActionListener.wrap(r -> success.set(true), e -> fail(e.toString()))); + assertTrue(success.get()); + + } + + public void testPublishMergedSegmentActionOnDocrepReplicaDuringMigration() { + final IndicesService indicesService = mock(IndicesService.class); + + final Index index = new Index("index", "uuid"); + final IndexService indexService = mock(IndexService.class); + when(indicesService.indexServiceSafe(index)).thenReturn(indexService); + final int id = randomIntBetween(0, 4); + final IndexShard indexShard = mock(IndexShard.class); + when(indexService.getShard(id)).thenReturn(indexShard); + + final ShardId shardId = new ShardId(index, id); + when(indexShard.shardId()).thenReturn(shardId); + when(indexShard.indexSettings()).thenReturn(createIndexSettings(false)); + final SegmentReplicationTargetService mockTargetService = mock(SegmentReplicationTargetService.class); + + final PublishMergedSegmentAction action = new PublishMergedSegmentAction( + Settings.EMPTY, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mockTargetService + ); + + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + + final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); + + final PlainActionFuture listener = PlainActionFuture.newFuture(); + action.shardOperationOnReplica(request, indexShard, listener); + final TransportReplicationAction.ReplicaResult result = listener.actionGet(); + // no interaction with SegmentReplicationTargetService object + verify(mockTargetService, never()).onNewMergedSegmentCheckpoint(any(), any()); + // the result should indicate success + final AtomicBoolean success = new AtomicBoolean(); + result.runPostReplicaActions(ActionListener.wrap(r -> success.set(true), e -> fail(e.toString()))); + assertTrue(success.get()); + } + + public void testGetReplicationModeWithRemoteTranslog() { + final PublishMergedSegmentAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(createIndexSettings(true)); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final PublishMergedSegmentAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(createIndexSettings(false)); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private PublishMergedSegmentAction createAction() { + return new PublishMergedSegmentAction( + Settings.EMPTY, + transportService, + clusterService, + mock(IndicesService.class), + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mock(SegmentReplicationTargetService.class) + ); + } + +} From 897d0bb3ba953900fb0b23c51528b90be3a5325e Mon Sep 17 00:00:00 2001 From: guojialiang Date: Thu, 5 Jun 2025 11:32:20 +0800 Subject: [PATCH 15/23] gradle spotlessApply Signed-off-by: guojialiang --- .../PublishMergedSegmentActionTests.java | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java index 0533f47151568..9376ca6351b11 100644 --- a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java @@ -36,10 +36,15 @@ import java.util.Collections; import java.util.concurrent.atomic.AtomicBoolean; -import static org.hamcrest.Matchers.sameInstance; -import static org.mockito.Mockito.*; import static org.opensearch.index.remote.RemoteStoreTestsHelper.createIndexSettings; import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class PublishMergedSegmentActionTests extends OpenSearchTestCase { @@ -106,7 +111,14 @@ public void testPublishMergedSegmentActionOnPrimary() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + 1, + 1111, + Codec.getDefault().getName(), + Collections.emptyMap(), + "_1" + ); final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); action.shardOperationOnPrimary(request, indexShard, ActionTestUtils.assertNoFailureListener(result -> { @@ -143,7 +155,14 @@ public void testPublishMergedSegmentActionOnReplica() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + 1, + 1111, + Codec.getDefault().getName(), + Collections.emptyMap(), + "_1" + ); final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); @@ -187,7 +206,14 @@ public void testPublishMergedSegmentActionOnDocrepReplicaDuringMigration() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint(indexShard.shardId(), 1, 1111, Codec.getDefault().getName(), Collections.emptyMap(), "_1"); + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + 1, + 1111, + Codec.getDefault().getName(), + Collections.emptyMap(), + "_1" + ); final PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); From f04152d5b32fcadf3a0e64259e32303ebb9e8925 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Thu, 5 Jun 2025 19:39:20 +0800 Subject: [PATCH 16/23] add test Signed-off-by: guojialiang --- .../PublishMergedSegmentActionTests.java | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java index 9376ca6351b11..6b3407b943fb8 100644 --- a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java @@ -16,8 +16,11 @@ import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.AllocationId; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.Index; @@ -25,6 +28,8 @@ import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.OpenSearchTestCase; @@ -84,6 +89,58 @@ public void tearDown() throws Exception { super.tearDown(); } + public void testPublishMergedSegment() { + final IndicesService indicesService = mock(IndicesService.class); + + final Index index = new Index("index", "uuid"); + final IndexService indexService = mock(IndexService.class); + when(indicesService.indexServiceSafe(index)).thenReturn(indexService); + + final int id = randomIntBetween(0, 4); + final IndexShard indexShard = mock(IndexShard.class); + when(indexService.getShard(id)).thenReturn(indexShard); + + final ShardId shardId = new ShardId(index, id); + when(indexShard.shardId()).thenReturn(shardId); + + ShardRouting shardRouting = mock(ShardRouting.class); + AllocationId allocationId = mock(AllocationId.class); + RecoveryState recoveryState = mock(RecoveryState.class); + RecoverySettings recoverySettings = mock(RecoverySettings.class); + when(recoverySettings.getMergedSegmentReplicationTimeout()).thenReturn(new TimeValue(1000)); + when(shardRouting.allocationId()).thenReturn(allocationId); + when(allocationId.getId()).thenReturn("1"); + when(recoveryState.getTargetNode()).thenReturn(clusterService.localNode()); + when(indexShard.routingEntry()).thenReturn(shardRouting); + when(indexShard.getPendingPrimaryTerm()).thenReturn(1L); + when(indexShard.recoveryState()).thenReturn(recoveryState); + when(indexShard.getRecoverySettings()).thenReturn(recoverySettings); + + final SegmentReplicationTargetService mockTargetService = mock(SegmentReplicationTargetService.class); + + final PublishMergedSegmentAction action = new PublishMergedSegmentAction( + Settings.EMPTY, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mockTargetService + ); + + final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + indexShard.shardId(), + 1, + 1111, + Codec.getDefault().getName(), + Collections.emptyMap(), + "_1" + ); + + action.publish(indexShard, checkpoint); + } + public void testPublishMergedSegmentActionOnPrimary() { final IndicesService indicesService = mock(IndicesService.class); From 9f5f4c0067243dcfdf89a84bb6684d3ce198e452 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 25 Jun 2025 11:52:50 +0800 Subject: [PATCH 17/23] rename ReplicationSegmentCheckpoint to MergeSegmentCheckpoint Signed-off-by: guojialiang --- .../index/engine/LocalMergedSegmentWarmer.java | 3 --- .../java/org/opensearch/index/shard/IndexShard.java | 12 ++++++------ ...ntCheckpoint.java => MergeSegmentCheckpoint.java} | 8 ++++---- .../checkpoint/MergedSegmentPublisher.java | 8 ++------ .../checkpoint/PublishMergedSegmentAction.java | 2 +- .../checkpoint/PublishMergedSegmentRequest.java | 8 ++++---- .../MergedSegmentReplicationTargetTests.java | 6 +++--- .../PrimaryShardReplicationSourceTests.java | 6 +++--- .../checkpoint/PublishMergedSegmentActionTests.java | 8 ++++---- .../checkpoint/PublishMergedSegmentRequestTests.java | 4 ++-- 10 files changed, 29 insertions(+), 36 deletions(-) rename server/src/main/java/org/opensearch/indices/replication/checkpoint/{ReplicationSegmentCheckpoint.java => MergeSegmentCheckpoint.java} (93%) diff --git a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java index 7fef9c30fbf97..33f1a2b547b78 100644 --- a/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java +++ b/server/src/main/java/org/opensearch/index/engine/LocalMergedSegmentWarmer.java @@ -32,8 +32,6 @@ package org.opensearch.index.engine; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.SegmentCommitInfo; @@ -51,7 +49,6 @@ * @opensearch.internal */ public class LocalMergedSegmentWarmer implements IndexWriter.IndexReaderWarmer { - private static final Logger logger = LogManager.getLogger(LocalMergedSegmentWarmer.class); private final TransportService transportService; private final RecoverySettings recoverySettings; private final ClusterService clusterService; 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 0ca2136d86336..b13cc5f311cc1 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -201,9 +201,9 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; +import org.opensearch.indices.replication.checkpoint.MergeSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.MergedSegmentPublisher; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationTimer; import org.opensearch.repositories.RepositoriesService; @@ -1860,23 +1860,23 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th public void publishMergedSegment(SegmentCommitInfo segmentCommitInfo) throws IOException { assert mergedSegmentPublisher != null; - mergedSegmentPublisher.publish(this, computeReplicationSegmentCheckpoint(segmentCommitInfo)); + mergedSegmentPublisher.publish(this, computeMergeSegmentCheckpoint(segmentCommitInfo)); } /** - * Compute {@link ReplicationSegmentCheckpoint} from a SegmentCommitInfo. + * Compute {@link MergeSegmentCheckpoint} from a SegmentCommitInfo. * This function fetches a metadata snapshot from the store that comes with an IO cost. * * @param segmentCommitInfo {@link SegmentCommitInfo} segmentCommitInfo to use to compute. - * @return {@link ReplicationSegmentCheckpoint} Checkpoint computed from the segmentCommitInfo. + * @return {@link MergeSegmentCheckpoint} Checkpoint computed from the segmentCommitInfo. * @throws IOException When there is an error computing segment metadata from the store. */ - public ReplicationSegmentCheckpoint computeReplicationSegmentCheckpoint(SegmentCommitInfo segmentCommitInfo) throws IOException { + public MergeSegmentCheckpoint computeMergeSegmentCheckpoint(SegmentCommitInfo segmentCommitInfo) throws IOException { // Only need to get the file metadata information in segmentCommitInfo and reuse Store#getSegmentMetadataMap. SegmentInfos segmentInfos = new SegmentInfos(Version.LATEST.major); segmentInfos.add(segmentCommitInfo); Map segmentMetadataMap = store.getSegmentMetadataMap(segmentInfos); - return new ReplicationSegmentCheckpoint( + return new MergeSegmentCheckpoint( shardId, getOperationPrimaryTerm(), segmentMetadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergeSegmentCheckpoint.java similarity index 93% rename from server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java rename to server/src/main/java/org/opensearch/indices/replication/checkpoint/MergeSegmentCheckpoint.java index 7cecd1720950d..424a2cc98939e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationSegmentCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergeSegmentCheckpoint.java @@ -50,10 +50,10 @@ * @opensearch.internal */ @ExperimentalApi -public class ReplicationSegmentCheckpoint extends ReplicationCheckpoint { +public class MergeSegmentCheckpoint extends ReplicationCheckpoint { private final String segmentName; - public ReplicationSegmentCheckpoint( + public MergeSegmentCheckpoint( ShardId shardId, long primaryTerm, long length, @@ -65,7 +65,7 @@ public ReplicationSegmentCheckpoint( this.segmentName = segmentName; } - public ReplicationSegmentCheckpoint(StreamInput in) throws IOException { + public MergeSegmentCheckpoint(StreamInput in) throws IOException { super(in); segmentName = in.readString(); } @@ -89,7 +89,7 @@ public void writeTo(StreamOutput out) throws IOException { public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - ReplicationSegmentCheckpoint that = (ReplicationSegmentCheckpoint) o; + MergeSegmentCheckpoint that = (MergeSegmentCheckpoint) o; return getPrimaryTerm() == that.getPrimaryTerm() && segmentName.equals(that.segmentName) && Objects.equals(getShardId(), that.getShardId()) diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java index a0c270cf9e3ce..4b49e71093c55 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/MergedSegmentPublisher.java @@ -8,8 +8,6 @@ package org.opensearch.indices.replication.checkpoint; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.inject.Inject; import org.opensearch.index.shard.IndexShard; @@ -23,8 +21,6 @@ */ @ExperimentalApi public class MergedSegmentPublisher { - protected static Logger logger = LogManager.getLogger(MergedSegmentPublisher.class); - private final PublishAction publishAction; // This Component is behind feature flag so we are manually binding this in IndicesModule. @@ -37,7 +33,7 @@ public MergedSegmentPublisher(PublishAction publishAction) { this.publishAction = Objects.requireNonNull(publishAction); } - public void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint) { + public void publish(IndexShard indexShard, MergeSegmentCheckpoint checkpoint) { publishAction.publish(indexShard, checkpoint); } @@ -48,7 +44,7 @@ public void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoi */ @ExperimentalApi public interface PublishAction { - void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint); + void publish(IndexShard indexShard, MergeSegmentCheckpoint checkpoint); } /** diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java index d3fe340d29dbd..9e12d67b2597d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java @@ -110,7 +110,7 @@ public ReplicationMode getReplicationMode(IndexShard indexShard) { /** * Publish merged segment request to shard */ - final void publish(IndexShard indexShard, ReplicationSegmentCheckpoint checkpoint) { + final void publish(IndexShard indexShard, MergeSegmentCheckpoint checkpoint) { String primaryAllocationId = indexShard.routingEntry().allocationId().getId(); long primaryTerm = indexShard.getPendingPrimaryTerm(); final ThreadContext threadContext = threadPool.getThreadContext(); diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java index b5f183befef63..0dbb54846522a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequest.java @@ -21,16 +21,16 @@ * @opensearch.internal */ public class PublishMergedSegmentRequest extends ReplicationRequest { - private final ReplicationSegmentCheckpoint mergedSegment; + private final MergeSegmentCheckpoint mergedSegment; - public PublishMergedSegmentRequest(ReplicationSegmentCheckpoint mergedSegment) { + public PublishMergedSegmentRequest(MergeSegmentCheckpoint mergedSegment) { super(mergedSegment.getShardId()); this.mergedSegment = mergedSegment; } public PublishMergedSegmentRequest(StreamInput in) throws IOException { super(in); - this.mergedSegment = new ReplicationSegmentCheckpoint(in); + this.mergedSegment = new MergeSegmentCheckpoint(in); } @Override @@ -56,7 +56,7 @@ public String toString() { return "PublishMergedSegmentRequest{" + "mergedSegment=" + mergedSegment + '}'; } - public ReplicationSegmentCheckpoint getMergedSegment() { + public MergeSegmentCheckpoint getMergedSegment() { return mergedSegment; } } diff --git a/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java index 489f67cb9fa61..53c8e29acf2c9 100644 --- a/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/MergedSegmentReplicationTargetTests.java @@ -22,8 +22,8 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.replication.checkpoint.MergeSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.indices.replication.common.ReplicationFailedException; import org.opensearch.indices.replication.common.ReplicationType; import org.junit.Assert; @@ -41,7 +41,7 @@ public class MergedSegmentReplicationTargetTests extends IndexShardTestCase { private MergedSegmentReplicationTarget mergedSegmentReplicationTarget; private IndexShard indexShard, spyIndexShard; - private ReplicationSegmentCheckpoint mergedSegment; + private MergeSegmentCheckpoint mergedSegment; private ByteBuffersDataOutput buffer; private static final String SEGMENT_NAME = "_0.si"; @@ -71,7 +71,7 @@ public void setUp() throws Exception { try (ByteBuffersIndexOutput indexOutput = new ByteBuffersIndexOutput(buffer, "", null)) { testSegmentInfos.write(indexOutput); } - mergedSegment = new ReplicationSegmentCheckpoint( + mergedSegment = new MergeSegmentCheckpoint( spyIndexShard.shardId(), spyIndexShard.getPendingPrimaryTerm(), 1, 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 132088561cdae..f9b236fad5b02 100644 --- a/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/PrimaryShardReplicationSourceTests.java @@ -21,8 +21,8 @@ import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.checkpoint.MergeSegmentCheckpoint; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.checkpoint.ReplicationSegmentCheckpoint; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.transport.CapturingTransport; @@ -136,7 +136,7 @@ public void testGetSegmentFiles() { public void testGetMergedSegmentFiles() { StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", 1L, "checksum", Version.LATEST); - final ReplicationCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final ReplicationCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), PRIMARY_TERM, 1, @@ -195,7 +195,7 @@ public void testTransportTimeoutForGetSegmentFilesAction() { public void testTransportTimeoutForGetMergedSegmentFilesAction() { long fileSize = (long) (Math.pow(10, 9)); StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", fileSize, "checksum", Version.LATEST); - final ReplicationCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final ReplicationCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), PRIMARY_TERM, 1, diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java index 6b3407b943fb8..38c2c3d54f95e 100644 --- a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentActionTests.java @@ -129,7 +129,7 @@ public void testPublishMergedSegment() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), 1, 1111, @@ -168,7 +168,7 @@ public void testPublishMergedSegmentActionOnPrimary() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), 1, 1111, @@ -212,7 +212,7 @@ public void testPublishMergedSegmentActionOnReplica() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), 1, 1111, @@ -263,7 +263,7 @@ public void testPublishMergedSegmentActionOnDocrepReplicaDuringMigration() { mockTargetService ); - final ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + final MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( indexShard.shardId(), 1, 1111, diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java index d041a08593aba..e039a598164b4 100644 --- a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentRequestTests.java @@ -22,7 +22,7 @@ public class PublishMergedSegmentRequestTests extends OpenSearchTestCase { public void testPublishMergedSegmentRequest() { - ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( new ShardId(new Index("1", "1"), 0), 0, 0, @@ -37,7 +37,7 @@ public void testPublishMergedSegmentRequest() { } public void testSerialize() throws Exception { - ReplicationSegmentCheckpoint checkpoint = new ReplicationSegmentCheckpoint( + MergeSegmentCheckpoint checkpoint = new MergeSegmentCheckpoint( new ShardId(new Index("1", "1"), 0), 0, 0, From 01839abaecb4b8e96fadacb6ecc375612fd40dff Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 25 Jun 2025 14:14:48 +0800 Subject: [PATCH 18/23] add some description Signed-off-by: guojialiang --- .../index/shard/SegmentReplicationIndexShardTests.java | 2 ++ 1 file changed, 2 insertions(+) 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 8b24b963da6fa..3358c39edbac4 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -156,6 +156,7 @@ public void testReplication() throws Exception { @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) public void testMergedSegmentReplication() throws Exception { + // Test that the pre-copy merged segment logic does not block the merge process of the primary shard when there are 1 replica shard. try (ReplicationGroup shards = createGroup(1, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { shards.startAll(); final IndexShard primaryShard = shards.getPrimary(); @@ -185,6 +186,7 @@ public void testMergedSegmentReplication() throws Exception { @LockFeatureFlag(MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG) public void testMergedSegmentReplicationWithZeroReplica() throws Exception { + // Test that the pre-copy merged segment logic does not block the merge process of the primary shard when there are 0 replica shard. try (ReplicationGroup shards = createGroup(0, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { shards.startAll(); final IndexShard primaryShard = shards.getPrimary(); From d558c8559106282e1a8e25aaedb71a24a7cb0552 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 25 Jun 2025 14:59:47 +0800 Subject: [PATCH 19/23] refactor code Signed-off-by: guojialiang --- .../SegmentReplicationSourceService.java | 57 +++++++++++-------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index 377319860b9ce..c748c46535e3d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.RateLimiter; import org.opensearch.action.support.ChannelActionListener; import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.ClusterStateListener; @@ -21,6 +22,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.CancellableThreads; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; @@ -35,6 +37,7 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RetryableTransportClient; import org.opensearch.indices.replication.common.ReplicationTimer; +import org.opensearch.indices.replication.common.SegmentReplicationTransportRequest; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportChannel; @@ -46,6 +49,7 @@ import java.util.Locale; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Supplier; /** * Service class that handles segment replication requests from replica shards. @@ -124,19 +128,11 @@ private class CheckpointInfoRequestHandler implements TransportRequestHandler {}, + request, + request.getCheckpoint().getShardId(), + recoverySettings.internalActionRetryTimeout(), recoverySettings::replicationRateLimiter ); final SegmentReplicationSourceHandler handler = ongoingSegmentReplications.prepareForReplication( @@ -193,20 +189,12 @@ public void messageReceived(GetSegmentFilesRequest request, TransportChannel cha throw new IllegalStateException(String.format(Locale.ROOT, "%s is not a started primary shard", shardId)); } - RemoteSegmentFileChunkWriter mergedSegmentFileChunkWriter = new RemoteSegmentFileChunkWriter( - request.getReplicationId(), - indexShard.getRecoverySettings(), - new RetryableTransportClient( - transportService, - request.getTargetNode(), - indexShard.getRecoverySettings().getMergedSegmentReplicationTimeout(), - logger - ), - request.getCheckpoint().getShardId(), + RemoteSegmentFileChunkWriter mergedSegmentFileChunkWriter = getRemoteSegmentFileChunkWriter( SegmentReplicationTargetService.Actions.MERGED_SEGMENT_FILE_CHUNK, - new AtomicLong(0), - (throttleTime) -> {}, - indexShard.getRecoverySettings()::mergedSegmentReplicationRateLimiter + request, + request.getCheckpoint().getShardId(), + recoverySettings.getMergedSegmentReplicationTimeout(), + recoverySettings::mergedSegmentReplicationRateLimiter ); SegmentFileTransferHandler mergedSegmentFileTransferHandler = new SegmentFileTransferHandler( @@ -241,6 +229,25 @@ public void onFailure(Exception e) { } } + private RemoteSegmentFileChunkWriter getRemoteSegmentFileChunkWriter( + String action, + SegmentReplicationTransportRequest request, + ShardId shardId, + TimeValue retryTimeout, + Supplier rateLimiterSupplier + ) { + return new RemoteSegmentFileChunkWriter( + request.getReplicationId(), + recoverySettings, + new RetryableTransportClient(transportService, request.getTargetNode(), retryTimeout, logger), + shardId, + action, + new AtomicLong(0), + (throttleTime) -> {}, + rateLimiterSupplier + ); + } + @Override public void clusterChanged(ClusterChangedEvent event) { if (event.nodesRemoved()) { From 96fc56c4f1196e8633ebd687eaaac1d8d259c8d4 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 25 Jun 2025 18:54:33 +0800 Subject: [PATCH 20/23] extract an abstract base class (AbstractPublishCheckpointAction) from PublishCheckpointAction Signed-off-by: guojialiang --- .../AbstractPublishCheckpointAction.java | 244 ++++++++++++++++++ .../checkpoint/PublishCheckpointAction.java | 135 +--------- .../PublishMergedSegmentAction.java | 163 ++---------- 3 files changed, 267 insertions(+), 275 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/replication/checkpoint/AbstractPublishCheckpointAction.java diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/AbstractPublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/AbstractPublishCheckpointAction.java new file mode 100644 index 0000000000000..ddf7a1f61030c --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/AbstractPublishCheckpointAction.java @@ -0,0 +1,244 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.checkpoint; + +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.ExceptionsHelper; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.action.support.replication.ReplicationRequest; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.action.support.replication.ReplicationTask; +import org.opensearch.action.support.replication.TransportReplicationAction; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.util.concurrent.ThreadContextAccess; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.index.IndexNotFoundException; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardClosedException; +import org.opensearch.index.shard.ShardNotInPrimaryModeException; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.common.ReplicationTimer; +import org.opensearch.node.NodeClosedException; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportResponseHandler; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Objects; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * Abstract base class for publish checkpoint. + * + * @opensearch.api + */ + +public abstract class AbstractPublishCheckpointAction< + Request extends ReplicationRequest, + ReplicaRequest extends ReplicationRequest> extends TransportReplicationAction< + Request, + ReplicaRequest, + ReplicationResponse> { + + private final Logger logger; + + public AbstractPublishCheckpointAction( + Settings settings, + String actionName, + TransportService transportService, + ClusterService clusterService, + IndicesService indicesService, + ThreadPool threadPool, + ShardStateAction shardStateAction, + ActionFilters actionFilters, + Writeable.Reader requestReader, + Writeable.Reader replicaRequestReader, + String threadPoolName, + Logger logger + ) { + super( + settings, + actionName, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + actionFilters, + requestReader, + replicaRequestReader, + threadPoolName + ); + this.logger = logger; + } + + @Override + protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { + return new ReplicationResponse(in); + } + + @Override + public ReplicationMode getReplicationMode(IndexShard indexShard) { + if (indexShard.indexSettings().isAssignedOnRemoteNode()) { + return ReplicationMode.FULL_REPLICATION; + } + return super.getReplicationMode(indexShard); + } + + /** + * Publish checkpoint request to shard + */ + final void doPublish( + IndexShard indexShard, + ReplicationCheckpoint checkpoint, + TransportRequest request, + String action, + boolean waitForCompletion, + TimeValue waitTimeout + ) { + String primaryAllocationId = indexShard.routingEntry().allocationId().getId(); + long primaryTerm = indexShard.getPendingPrimaryTerm(); + final ThreadContext threadContext = threadPool.getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we have to execute under the system context so that if security is enabled the sync is authorized + ThreadContextAccess.doPrivilegedVoid(threadContext::markAsSystemContext); + final ReplicationTask task = (ReplicationTask) taskManager.register("transport", action, request); + final ReplicationTimer timer = new ReplicationTimer(); + timer.start(); + CountDownLatch latch = new CountDownLatch(1); + transportService.sendChildRequest( + indexShard.recoveryState().getTargetNode(), + transportPrimaryAction, + new ConcreteShardRequest<>(request, primaryAllocationId, primaryTerm), + task, + transportOptions, + new TransportResponseHandler() { + @Override + public ReplicationResponse read(StreamInput in) throws IOException { + return newResponseInstance(in); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public void handleResponse(ReplicationResponse response) { + try { + timer.stop(); + logger.debug( + () -> new ParameterizedMessage( + "[shardId {}] Completed publishing checkpoint [{}], timing: {}", + indexShard.shardId().getId(), + checkpoint, + timer.time() + ) + ); + task.setPhase("finished"); + taskManager.unregister(task); + } finally { + latch.countDown(); + } + } + + @Override + public void handleException(TransportException e) { + try { + timer.stop(); + logger.debug( + "[shardId {}] Failed to publish checkpoint [{}], timing: {}", + indexShard.shardId().getId(), + checkpoint, + timer.time() + ); + task.setPhase("finished"); + taskManager.unregister(task); + if (ExceptionsHelper.unwrap( + e, + NodeClosedException.class, + IndexNotFoundException.class, + AlreadyClosedException.class, + IndexShardClosedException.class, + ShardNotInPrimaryModeException.class + ) != null) { + // Node is shutting down or the index was deleted or the shard is closed + return; + } + logger.warn( + new ParameterizedMessage( + "{} segment replication checkpoint [{}] publishing failed", + indexShard.shardId(), + checkpoint + ), + e + ); + } finally { + latch.countDown(); + } + } + } + ); + logger.trace( + () -> new ParameterizedMessage( + "[shardId {}] Publishing replication checkpoint [{}]", + checkpoint.getShardId().getId(), + checkpoint + ) + ); + if (waitForCompletion) { + try { + latch.await(waitTimeout.seconds(), TimeUnit.SECONDS); + } catch (InterruptedException e) { + logger.warn( + () -> new ParameterizedMessage("Interrupted while waiting for publish checkpoint complete [{}]", checkpoint), + e + ); + } + } + } + } + + @Override + final protected void shardOperationOnReplica(ReplicaRequest shardRequest, IndexShard replica, ActionListener listener) { + Objects.requireNonNull(shardRequest); + Objects.requireNonNull(replica); + ActionListener.completeWith(listener, () -> { + logger.trace(() -> new ParameterizedMessage("Checkpoint {} received on replica {}", shardRequest, replica.shardId())); + // Condition for ensuring that we ignore Segrep checkpoints received on Docrep shard copies. + // This case will hit iff the replica hosting node is not remote enabled and replication type != SEGMENT + if (replica.indexSettings().isAssignedOnRemoteNode() == false && replica.indexSettings().isSegRepLocalEnabled() == false) { + logger.trace("Received segrep checkpoint on a docrep shard copy during an ongoing remote migration. NoOp."); + return new ReplicaResult(); + } + doReplicaOperation(shardRequest, replica); + return new ReplicaResult(); + }); + } + + /** + * Execute the specified replica operation. + * + * @param shardRequest the request to the replica shard + * @param replica the replica shard to perform the operation on + */ + protected abstract void doReplicaOperation(ReplicaRequest shardRequest, IndexShard replica); +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java index 7181355333be7..b4279e220a90f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -10,14 +10,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.ReplicationResponse; -import org.opensearch.action.support.replication.ReplicationTask; -import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.annotation.PublicApi; @@ -25,37 +19,21 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.common.util.concurrent.ThreadContextAccess; import org.opensearch.core.action.ActionListener; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardClosedException; -import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.SegmentReplicationTargetService; -import org.opensearch.indices.replication.common.ReplicationTimer; -import org.opensearch.node.NodeClosedException; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.TransportException; -import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; -import java.io.IOException; -import java.util.Objects; - /** * Replication action responsible for publishing checkpoint to a replica shard. * * @opensearch.api */ @PublicApi(since = "2.2.0") -public class PublishCheckpointAction extends TransportReplicationAction< - PublishCheckpointRequest, - PublishCheckpointRequest, - ReplicationResponse> { +public class PublishCheckpointAction extends AbstractPublishCheckpointAction { public static final String ACTION_NAME = "indices:admin/publishCheckpoint"; protected static Logger logger = LogManager.getLogger(PublishCheckpointAction.class); @@ -94,7 +72,8 @@ public PublishCheckpointAction( actionFilters, PublishCheckpointRequest::new, PublishCheckpointRequest::new, - ThreadPool.Names.REFRESH + ThreadPool.Names.REFRESH, + logger ); this.replicationService = targetService; } @@ -104,102 +83,16 @@ protected Setting getRetryTimeoutSetting() { return PUBLISH_CHECK_POINT_RETRY_TIMEOUT; } - @Override - protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { - return new ReplicationResponse(in); - } - @Override protected void doExecute(Task task, PublishCheckpointRequest request, ActionListener listener) { assert false : "use PublishCheckpointAction#publish"; } - @Override - public ReplicationMode getReplicationMode(IndexShard indexShard) { - if (indexShard.indexSettings().isAssignedOnRemoteNode()) { - return ReplicationMode.FULL_REPLICATION; - } - return super.getReplicationMode(indexShard); - } - /** * Publish checkpoint request to shard */ final void publish(IndexShard indexShard, ReplicationCheckpoint checkpoint) { - String primaryAllocationId = indexShard.routingEntry().allocationId().getId(); - long primaryTerm = indexShard.getPendingPrimaryTerm(); - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - // we have to execute under the system context so that if security is enabled the sync is authorized - ThreadContextAccess.doPrivilegedVoid(threadContext::markAsSystemContext); - PublishCheckpointRequest request = new PublishCheckpointRequest(checkpoint); - final ReplicationTask task = (ReplicationTask) taskManager.register("transport", "segrep_publish_checkpoint", request); - final ReplicationTimer timer = new ReplicationTimer(); - timer.start(); - transportService.sendChildRequest( - indexShard.recoveryState().getTargetNode(), - transportPrimaryAction, - new ConcreteShardRequest<>(request, primaryAllocationId, primaryTerm), - task, - transportOptions, - new TransportResponseHandler() { - @Override - public ReplicationResponse read(StreamInput in) throws IOException { - return newResponseInstance(in); - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - - @Override - public void handleResponse(ReplicationResponse response) { - timer.stop(); - logger.debug( - () -> new ParameterizedMessage( - "[shardId {}] Completed publishing checkpoint [{}], timing: {}", - indexShard.shardId().getId(), - checkpoint, - timer.time() - ) - ); - task.setPhase("finished"); - taskManager.unregister(task); - } - - @Override - public void handleException(TransportException e) { - timer.stop(); - logger.debug("[shardId {}] Failed to publish checkpoint, timing: {}", indexShard.shardId().getId(), timer.time()); - task.setPhase("finished"); - taskManager.unregister(task); - if (ExceptionsHelper.unwrap( - e, - NodeClosedException.class, - IndexNotFoundException.class, - AlreadyClosedException.class, - IndexShardClosedException.class, - ShardNotInPrimaryModeException.class - ) != null) { - // Node is shutting down or the index was deleted or the shard is closed - return; - } - logger.warn( - new ParameterizedMessage("{} segment replication checkpoint publishing failed", indexShard.shardId()), - e - ); - } - } - ); - logger.trace( - () -> new ParameterizedMessage( - "[shardId {}] Publishing replication checkpoint [{}]", - checkpoint.getShardId().getId(), - checkpoint - ) - ); - } + doPublish(indexShard, checkpoint, new PublishCheckpointRequest(checkpoint), "segrep_publish_checkpoint", false, null); } @Override @@ -212,21 +105,9 @@ protected void shardOperationOnPrimary( } @Override - protected void shardOperationOnReplica(PublishCheckpointRequest request, IndexShard replica, ActionListener listener) { - Objects.requireNonNull(request); - Objects.requireNonNull(replica); - ActionListener.completeWith(listener, () -> { - logger.trace(() -> new ParameterizedMessage("Checkpoint {} received on replica {}", request, replica.shardId())); - // Condition for ensuring that we ignore Segrep checkpoints received on Docrep shard copies. - // This case will hit iff the replica hosting node is not remote enabled and replication type != SEGMENT - if (replica.indexSettings().isAssignedOnRemoteNode() == false && replica.indexSettings().isSegRepLocalEnabled() == false) { - logger.trace("Received segrep checkpoint on a docrep shard copy during an ongoing remote migration. NoOp."); - return new ReplicaResult(); - } - if (request.getCheckpoint().getShardId().equals(replica.shardId())) { - replicationService.onNewCheckpoint(request.getCheckpoint(), replica); - } - return new ReplicaResult(); - }); + protected void doReplicaOperation(PublishCheckpointRequest request, IndexShard replica) { + if (request.getCheckpoint().getShardId().equals(replica.shardId())) { + replicationService.onNewCheckpoint(request.getCheckpoint(), replica); + } } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java index 9e12d67b2597d..675a13c5c05ae 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishMergedSegmentAction.java @@ -10,52 +10,28 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.support.ActionFilters; -import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.ReplicationResponse; -import org.opensearch.action.support.replication.ReplicationTask; -import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.common.util.concurrent.ThreadContextAccess; import org.opensearch.core.action.ActionListener; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardClosedException; -import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.SegmentReplicationTargetService; -import org.opensearch.indices.replication.common.ReplicationTimer; -import org.opensearch.node.NodeClosedException; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; -import org.opensearch.transport.TransportException; -import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; -import java.io.IOException; -import java.util.Objects; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - /** * Replication action responsible for publishing merged segment to a replica shard. * * @opensearch.api */ @ExperimentalApi -public class PublishMergedSegmentAction extends TransportReplicationAction< - PublishMergedSegmentRequest, - PublishMergedSegmentRequest, - ReplicationResponse> { +public class PublishMergedSegmentAction extends AbstractPublishCheckpointAction { public static final String ACTION_NAME = "indices:admin/publish_merged_segment"; protected static Logger logger = LogManager.getLogger(PublishMergedSegmentAction.class); @@ -84,122 +60,29 @@ public PublishMergedSegmentAction( actionFilters, PublishMergedSegmentRequest::new, PublishMergedSegmentRequest::new, - ThreadPool.Names.GENERIC + ThreadPool.Names.GENERIC, + logger ); this.replicationService = targetService; } - @Override - protected ReplicationResponse newResponseInstance(StreamInput in) throws IOException { - return new ReplicationResponse(in); - } - @Override protected void doExecute(Task task, PublishMergedSegmentRequest request, ActionListener listener) { assert false : "use PublishMergedSegmentAction#publish"; } - @Override - public ReplicationMode getReplicationMode(IndexShard indexShard) { - if (indexShard.indexSettings().isAssignedOnRemoteNode()) { - return ReplicationMode.FULL_REPLICATION; - } - return super.getReplicationMode(indexShard); - } - /** * Publish merged segment request to shard */ final void publish(IndexShard indexShard, MergeSegmentCheckpoint checkpoint) { - String primaryAllocationId = indexShard.routingEntry().allocationId().getId(); - long primaryTerm = indexShard.getPendingPrimaryTerm(); - final ThreadContext threadContext = threadPool.getThreadContext(); - try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { - // we have to execute under the system context so that if security is enabled the sync is authorized - ThreadContextAccess.doPrivilegedVoid(threadContext::markAsSystemContext); - PublishMergedSegmentRequest request = new PublishMergedSegmentRequest(checkpoint); - final ReplicationTask task = (ReplicationTask) taskManager.register("transport", "segrep_publish_merged_segment", request); - final ReplicationTimer timer = new ReplicationTimer(); - timer.start(); - CountDownLatch latch = new CountDownLatch(1); - transportService.sendChildRequest( - indexShard.recoveryState().getTargetNode(), - transportPrimaryAction, - new ConcreteShardRequest<>(request, primaryAllocationId, primaryTerm), - task, - transportOptions, - new TransportResponseHandler() { - @Override - public ReplicationResponse read(StreamInput in) throws IOException { - return newResponseInstance(in); - } - - @Override - public String executor() { - return ThreadPool.Names.SAME; - } - - @Override - public void handleResponse(ReplicationResponse response) { - try { - timer.stop(); - logger.debug( - () -> new ParameterizedMessage( - "[shardId {}] Completed publishing merged segment [{}], timing: {}", - indexShard.shardId().getId(), - checkpoint, - timer.time() - ) - ); - task.setPhase("finished"); - taskManager.unregister(task); - } finally { - latch.countDown(); - } - } - - @Override - public void handleException(TransportException e) { - try { - timer.stop(); - logger.debug( - "[shardId {}] Failed to publish merged segment [{}], timing: {}", - indexShard.shardId().getId(), - checkpoint, - timer.time() - ); - task.setPhase("finished"); - taskManager.unregister(task); - if (ExceptionsHelper.unwrap( - e, - NodeClosedException.class, - IndexNotFoundException.class, - AlreadyClosedException.class, - IndexShardClosedException.class, - ShardNotInPrimaryModeException.class - ) != null) { - // Node is shutting down or the index was deleted or the shard is closed - return; - } - logger.warn( - new ParameterizedMessage("{} merged segment [{}] publishing failed", indexShard.shardId(), checkpoint), - e - ); - } finally { - latch.countDown(); - } - } - } - ); - logger.trace( - () -> new ParameterizedMessage("[shardId {}] Publishing merged segment [{}]", checkpoint.getShardId().getId(), checkpoint) - ); - try { - latch.await(indexShard.getRecoverySettings().getMergedSegmentReplicationTimeout().seconds(), TimeUnit.SECONDS); - } catch (InterruptedException e) { - logger.warn(() -> new ParameterizedMessage("Interrupted while waiting for pre copy merged segment [{}]", checkpoint), e); - } - } + doPublish( + indexShard, + checkpoint, + new PublishMergedSegmentRequest(checkpoint), + "segrep_publish_merged_segment", + true, + indexShard.getRecoverySettings().getMergedSegmentReplicationTimeout() + ); } @Override @@ -212,25 +95,9 @@ protected void shardOperationOnPrimary( } @Override - protected void shardOperationOnReplica( - PublishMergedSegmentRequest request, - IndexShard replica, - ActionListener listener - ) { - Objects.requireNonNull(request); - Objects.requireNonNull(replica); - ActionListener.completeWith(listener, () -> { - logger.trace(() -> new ParameterizedMessage("Merged segment {} received on replica {}", request, replica.shardId())); - // Condition for ensuring that we ignore Segrep checkpoints received on Docrep shard copies. - // This case will hit iff the replica hosting node is not remote enabled and replication type != SEGMENT - if (replica.indexSettings().isAssignedOnRemoteNode() == false && replica.indexSettings().isSegRepLocalEnabled() == false) { - logger.trace("Received merged segment on a docrep shard copy during an ongoing remote migration. NoOp."); - return new ReplicaResult(); - } - if (request.getMergedSegment().getShardId().equals(replica.shardId())) { - replicationService.onNewMergedSegmentCheckpoint(request.getMergedSegment(), replica); - } - return new ReplicaResult(); - }); + protected void doReplicaOperation(PublishMergedSegmentRequest request, IndexShard replica) { + if (request.getMergedSegment().getShardId().equals(replica.shardId())) { + replicationService.onNewMergedSegmentCheckpoint(request.getMergedSegment(), replica); + } } } From a0b4e3ae90c6c5bd9c860776bdd27fe082d04622 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Wed, 25 Jun 2025 20:55:29 +0800 Subject: [PATCH 21/23] fix :server:japicmp Signed-off-by: guojialiang --- .../replication/checkpoint/PublishCheckpointAction.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java index b4279e220a90f..957bd3e8f464f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.ReplicationResponse; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; @@ -83,6 +84,11 @@ protected Setting getRetryTimeoutSetting() { return PUBLISH_CHECK_POINT_RETRY_TIMEOUT; } + @Override + public ReplicationMode getReplicationMode(IndexShard indexShard) { + return super.getReplicationMode(indexShard); + } + @Override protected void doExecute(Task task, PublishCheckpointRequest request, ActionListener listener) { assert false : "use PublishCheckpointAction#publish"; From dedad5e6d19679eda5fccc9b17a641cf6734a26f Mon Sep 17 00:00:00 2001 From: guojialiang Date: Thu, 26 Jun 2025 10:59:01 +0800 Subject: [PATCH 22/23] update Signed-off-by: guojialiang --- .../replication/checkpoint/PublishCheckpointAction.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java index 957bd3e8f464f..1bba8db90d4f8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -85,13 +85,14 @@ protected Setting getRetryTimeoutSetting() { } @Override - public ReplicationMode getReplicationMode(IndexShard indexShard) { - return super.getReplicationMode(indexShard); + protected void doExecute(Task task, PublishCheckpointRequest request, ActionListener listener) { + assert false : "use PublishCheckpointAction#publish"; } @Override - protected void doExecute(Task task, PublishCheckpointRequest request, ActionListener listener) { - assert false : "use PublishCheckpointAction#publish"; + @Deprecated(forRemoval = true) + public ReplicationMode getReplicationMode(IndexShard indexShard) { + return super.getReplicationMode(indexShard); } /** From 3224f35f144cb947d13f938b40b5dd7729bd0713 Mon Sep 17 00:00:00 2001 From: guojialiang Date: Thu, 26 Jun 2025 11:34:16 +0800 Subject: [PATCH 23/23] update Signed-off-by: guojialiang --- .../indices/replication/checkpoint/PublishCheckpointAction.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java index 1bba8db90d4f8..bce70aa2b5c9e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -90,7 +90,6 @@ protected void doExecute(Task task, PublishCheckpointRequest request, ActionList } @Override - @Deprecated(forRemoval = true) public ReplicationMode getReplicationMode(IndexShard indexShard) { return super.getReplicationMode(indexShard); }