From 44a1134c50b2528e45821ed738a62d9ccd9c7d7d Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Sat, 18 Jan 2025 18:52:41 -0800 Subject: [PATCH 01/25] Implemented computation of segment replication stats at shard level The method implemented here computes the segment replication stats at the shard level, instead of relying on the primary shard to compute stats based on reports from its replicas. Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/index/shard/IndexShardIT.java | 5 +- .../org/opensearch/index/IndexModule.java | 11 ++- .../org/opensearch/index/IndexService.java | 12 ++- .../opensearch/index/shard/IndexShard.java | 23 ++--- .../metadata/RemoteSegmentMetadata.java | 4 +- .../opensearch/indices/IndicesService.java | 11 ++- .../replication/SegmentReplicationTarget.java | 10 +- .../SegmentReplicationTargetService.java | 1 + .../replication/SegmentReplicator.java | 68 +++++++++++++- .../checkpoint/ReplicationCheckpoint.java | 16 +++- .../main/java/org/opensearch/node/Node.java | 3 +- .../opensearch/index/IndexModuleTests.java | 3 +- .../index/seqno/ReplicationTrackerTests.java | 15 ++- .../SegmentReplicationTargetTests.java | 15 +-- .../replication/SegmentReplicatorTests.java | 93 +++++++++++++++++++ .../replication/common/CopyStateTests.java | 3 +- .../index/shard/IndexShardTestCase.java | 17 ++-- 17 files changed, 262 insertions(+), 48 deletions(-) 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 f97950f2652a3..41d00b94877af 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -114,6 +114,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.Predicate; import java.util.stream.Stream; @@ -136,6 +137,7 @@ import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static com.carrotsearch.randomizedtesting.RandomizedTest.randomAsciiLettersOfLength; +import static org.mockito.Mockito.mock; public class IndexShardIT extends OpenSearchSingleNodeTestCase { @@ -716,7 +718,8 @@ public static final IndexShard newIndexShard( null, DefaultRemoteStoreSettings.INSTANCE, false, - IndexShardTestUtils.getFakeDiscoveryNodes(initializingShardRouting) + IndexShardTestUtils.getFakeDiscoveryNodes(initializingShardRouting), + mock(BiFunction.class) ); } diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 79de97dc96fba..c55e5e1c90778 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -59,6 +59,7 @@ import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.indices.breaker.CircuitBreakerService; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; @@ -87,6 +88,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.ReplicationCheckpoint; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.repositories.RepositoriesService; import org.opensearch.script.ScriptService; @@ -653,7 +655,8 @@ public IndexService newIndexService( clusterDefaultRefreshIntervalSupplier, recoverySettings, remoteStoreSettings, - (s) -> {} + (s) -> {}, + null ); } @@ -679,7 +682,8 @@ public IndexService newIndexService( Supplier clusterDefaultRefreshIntervalSupplier, RecoverySettings recoverySettings, RemoteStoreSettings remoteStoreSettings, - Consumer replicator + Consumer replicator, + BiFunction segmentReplicationStatsProvider ) throws IOException { final IndexEventListener eventListener = freeze(); Function> readerWrapperFactory = indexReaderWrapper @@ -741,7 +745,8 @@ public IndexService newIndexService( remoteStoreSettings, fileCache, compositeIndexSettings, - replicator + replicator, + segmentReplicationStatsProvider ); success = true; return indexService; diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index f1b36194bf62d..69a950e604ecb 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -105,6 +105,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.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.plugins.IndexStorePlugin; @@ -197,6 +198,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final FileCache fileCache; private final CompositeIndexSettings compositeIndexSettings; private final Consumer replicator; + private final BiFunction segmentReplicationStatsProvider; public IndexService( IndexSettings indexSettings, @@ -235,7 +237,8 @@ public IndexService( RemoteStoreSettings remoteStoreSettings, FileCache fileCache, CompositeIndexSettings compositeIndexSettings, - Consumer replicator + Consumer replicator, + BiFunction segmentReplicationStatsProvider ) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; @@ -319,6 +322,7 @@ public IndexService( this.compositeIndexSettings = compositeIndexSettings; this.fileCache = fileCache; this.replicator = replicator; + this.segmentReplicationStatsProvider = segmentReplicationStatsProvider; updateFsyncTaskIfNecessary(); } @@ -395,7 +399,8 @@ public IndexService( remoteStoreSettings, null, null, - s -> {} + s -> {}, + null ); } @@ -691,7 +696,8 @@ protected void closeInternal() { recoverySettings, remoteStoreSettings, seedRemote, - discoveryNodes + discoveryNodes, + segmentReplicationStatsProvider ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); 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 eb3999718ca5b..50f43ff7ec94d 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -361,6 +361,7 @@ Runnable getGlobalCheckpointSyncer() { */ private final ShardMigrationState shardMigrationState; private DiscoveryNodes discoveryNodes; + private final BiFunction segmentReplicationStatsProvider; public IndexShard( final ShardRouting shardRouting, @@ -391,7 +392,8 @@ public IndexShard( final RecoverySettings recoverySettings, final RemoteStoreSettings remoteStoreSettings, boolean seedRemote, - final DiscoveryNodes discoveryNodes + final DiscoveryNodes discoveryNodes, + final BiFunction segmentReplicationStatsProvider ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -493,6 +495,7 @@ public boolean shouldCache(Query query) { this.fileDownloader = new RemoteStoreFileDownloader(shardRouting.shardId(), threadPool, recoverySettings); this.shardMigrationState = getShardMigrationState(indexSettings, seedRemote); this.discoveryNodes = discoveryNodes; + this.segmentReplicationStatsProvider = segmentReplicationStatsProvider; } public ThreadPool getThreadPool() { @@ -1768,7 +1771,8 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th segmentInfos.getVersion(), metadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), getEngine().config().getCodec().getName(), - metadataMap + metadataMap, + System.currentTimeMillis() ); logger.trace("Recomputed ReplicationCheckpoint for shard {}", checkpoint); return checkpoint; @@ -3209,17 +3213,10 @@ public Set getReplicationStatsForTrackedReplicas() } public ReplicationStats getReplicationStats() { - if (indexSettings.isSegRepEnabledOrRemoteNode() && routingEntry().primary()) { - final Set stats = getReplicationStatsForTrackedReplicas(); - long maxBytesBehind = stats.stream().mapToLong(SegmentReplicationShardStats::getBytesBehindCount).max().orElse(0L); - long totalBytesBehind = stats.stream().mapToLong(SegmentReplicationShardStats::getBytesBehindCount).sum(); - long maxReplicationLag = stats.stream() - .mapToLong(SegmentReplicationShardStats::getCurrentReplicationLagMillis) - .max() - .orElse(0L); - return new ReplicationStats(maxBytesBehind, totalBytesBehind, maxReplicationLag); - } - return new ReplicationStats(); + if (indexSettings.isSegRepEnabledOrRemoteNode() && !routingEntry().primary()) { + return segmentReplicationStatsProvider.apply(shardId, this.getLatestReplicationCheckpoint()); + } + return new ReplicationStats(0, 0, 0); } /** diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 41a145273e8ef..c04ede4b443a1 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -136,6 +136,7 @@ public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicatio out.writeLong(replicationCheckpoint.getSegmentInfosVersion()); out.writeLong(replicationCheckpoint.getLength()); out.writeString(replicationCheckpoint.getCodec()); + out.writeLong(replicationCheckpoint.getCreatedTimeStamp()); } private static ReplicationCheckpoint readCheckpointFromIndexInput( @@ -149,7 +150,8 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput( in.readLong(), in.readLong(), in.readString(), - toStoreFileMetadata(uploadedSegmentMetadataMap) + toStoreFileMetadata(uploadedSegmentMetadataMap), + in.readLong() ); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index b9bad5527e3f4..cbfcfb45d4aa2 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -105,6 +105,7 @@ import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.ReplicationStats; import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.cache.request.ShardRequestCache; import org.opensearch.index.compositeindex.CompositeIndexSettings; @@ -150,6 +151,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.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.Node; @@ -361,6 +363,7 @@ public class IndicesService extends AbstractLifecycleComponent private final FileCache fileCache; private final CompositeIndexSettings compositeIndexSettings; private final Consumer replicator; + private final BiFunction segmentReplicationStatsProvider; private volatile int maxSizeInRequestCache; @Override @@ -399,7 +402,8 @@ public IndicesService( RemoteStoreSettings remoteStoreSettings, FileCache fileCache, CompositeIndexSettings compositeIndexSettings, - Consumer replicator + Consumer replicator, + BiFunction segmentReplicationStatsProvider ) { this.settings = settings; this.threadPool = threadPool; @@ -509,6 +513,7 @@ protected void closeInternal() { this.compositeIndexSettings = compositeIndexSettings; this.fileCache = fileCache; this.replicator = replicator; + this.segmentReplicationStatsProvider = segmentReplicationStatsProvider; this.maxSizeInRequestCache = INDICES_REQUEST_CACHE_MAX_SIZE_ALLOWED_IN_CACHE_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() .addSettingsUpdateConsumer(INDICES_REQUEST_CACHE_MAX_SIZE_ALLOWED_IN_CACHE_SETTING, this::setMaxSizeInRequestCache); @@ -573,6 +578,7 @@ public IndicesService( remoteStoreSettings, null, null, + null, null ); } @@ -990,7 +996,8 @@ private synchronized IndexService createIndexService( this::getClusterDefaultRefreshInterval, this.recoverySettings, this.remoteStoreSettings, - replicator + replicator, + segmentReplicationStatsProvider ); } 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 7131b49a41834..8dfc4f45bc3c9 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -24,6 +24,7 @@ import org.opensearch.common.util.CancellableThreads; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; @@ -39,6 +40,7 @@ import java.util.List; import java.util.Locale; import java.util.Set; +import java.util.function.BiConsumer; import java.util.stream.Collectors; /** @@ -139,6 +141,10 @@ public ReplicationCheckpoint getCheckpoint() { return this.checkpoint; } + public SegmentReplicationSource getSource() { + return source; + } + @Override public void writeFileChunk( StoreFileMetadata metadata, @@ -161,7 +167,7 @@ public void writeFileChunk( * * @param listener {@link ActionListener} listener. */ - public void startReplication(ActionListener listener) { + public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { throw new CancellableThreads.ExecutionCancelledException("replication was canceled reason [" + reason + "]"); }); @@ -177,6 +183,8 @@ public void startReplication(ActionListener listener) { source.getCheckpointMetadata(getId(), checkpoint, checkpointInfoListener); checkpointInfoListener.whenComplete(checkpointInfo -> { + checkpointUpdater.accept(checkpointInfo.getCheckpoint(), this.indexShard.shardId()); + final List filesToFetch = getFiles(checkpointInfo); state.setStage(SegmentReplicationState.Stage.GET_FILES); cancellableThreads.checkForCancel(); 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 8fee3f671ecc9..bd11f1e3d02ec 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -294,6 +294,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe return; } updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); + replicator.updatePrimaryLastRefreshedCheckpoint(receivedCheckpoint, replicaShard.shardId()); // Checks if replica shard is in the correct STARTED state to process checkpoints (avoids parallel replication events taking place) // This check ensures we do not try to process a received checkpoint while the shard is still recovering, yet we stored the latest // checkpoint to be replayed once the shard is Active. 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 ad3bc1933208c..c540fa37607eb 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -13,14 +13,17 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.opensearch.OpenSearchCorruptionException; +import org.opensearch.action.StepListener; import org.opensearch.common.SetOnce; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.ReplicationStats; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationCollection; import org.opensearch.indices.replication.common.ReplicationFailedException; @@ -29,6 +32,7 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; /** * This class is responsible for managing segment replication events on replicas. @@ -43,8 +47,9 @@ public class SegmentReplicator { private final ReplicationCollection onGoingReplications; private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); + private final Map primaryLastRefreshedCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final Map lastOnGoingReplicationCheckpoint = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; - private final SetOnce sourceFactory; public SegmentReplicator(ThreadPool threadPool) { @@ -102,6 +107,50 @@ SegmentReplicationTarget startReplication( return target; } + public ReplicationStats getSegmentReplicationStats(ShardId shardId, ReplicationCheckpoint indexReplicationCheckPoint) { + assert shardId != null : "shardId cannot be null"; + assert indexReplicationCheckPoint != null : "indexReplicationCheckPoint cannot be null"; + ; + final Map indexStoreFileMetadata = indexReplicationCheckPoint.getMetadataMap(); + // If primaryLastRefreshedCheckpoint is null, we will default to indexReplicationCheckPoint + // so that we can avoid any failures + final ReplicationCheckpoint primaryLastRefreshedCheckpoint = Objects.requireNonNullElse( + this.primaryLastRefreshedCheckpoint.get(shardId), + indexReplicationCheckPoint + ); + final Map storeFileMetadata = primaryLastRefreshedCheckpoint.getMetadataMap(); + + final Store.RecoveryDiff diff = Store.segmentReplicationDiff(storeFileMetadata, indexStoreFileMetadata); + long bytesBehindSum = diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); + + final ReplicationCheckpoint lastOnGoingReplicationCheckpoint = this.lastOnGoingReplicationCheckpoint.get(shardId); + final long replicationLag = lastOnGoingReplicationCheckpoint != null + ? System.currentTimeMillis() - lastOnGoingReplicationCheckpoint.getCreatedTimeStamp() + : 0; + + return new ReplicationStats(bytesBehindSum, bytesBehindSum, bytesBehindSum > 0L ? replicationLag : 0); + } + + public void updatePrimaryLastRefreshedCheckpoint(ReplicationCheckpoint replicationCheckpoint, ShardId shardId) { + updateCheckpointIfAhead(primaryLastRefreshedCheckpoint, replicationCheckpoint, shardId); + } + + public void updateReplicationCheckpoints(ReplicationCheckpoint replicationCheckpoint, ShardId shardId) { + updateCheckpointIfAhead(lastOnGoingReplicationCheckpoint, replicationCheckpoint, shardId); + updatePrimaryLastRefreshedCheckpoint(replicationCheckpoint, shardId); + } + + private void updateCheckpointIfAhead( + Map checkpointMap, + ReplicationCheckpoint newCheckpoint, + ShardId shardId + ) { + final ReplicationCheckpoint existingCheckpoint = checkpointMap.get(shardId); + if (existingCheckpoint == null || newCheckpoint.isAheadOf(existingCheckpoint)) { + checkpointMap.put(shardId, newCheckpoint); + } + } + /** * Runnable implementation to trigger a replication event. */ @@ -153,7 +202,7 @@ public void onFailure(Exception e) { } onGoingReplications.fail(replicationId, new ReplicationFailedException("Segment Replication failed", e), false); } - }); + }, this::updateReplicationCheckpoints); } // pkg-private for integration tests @@ -163,6 +212,7 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) replicationId = onGoingReplications.startSafe(target, timeout); } catch (ReplicationFailedException e) { // replication already running for shard. + fetchPrimaryLastRefreshedCheckpoint(target); target.fail(e, false); return; } @@ -170,6 +220,20 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) threadPool.generic().execute(new ReplicationRunner(replicationId)); } + private void fetchPrimaryLastRefreshedCheckpoint(SegmentReplicationTarget target) { + // Only process search-only shards + if (!target.indexShard().routingEntry().isSearchOnly()) { + return; + } + + final StepListener checkpointInfoListener = new StepListener<>(); + target.getSource().getCheckpointMetadata(target.getId(), target.getCheckpoint(), checkpointInfoListener); + checkpointInfoListener.whenComplete( + checkpointInfo -> updatePrimaryLastRefreshedCheckpoint(checkpointInfo.getCheckpoint(), target.indexShard().shardId()), + checkpointInfoListener::onFailure + ); + } + private boolean isStoreCorrupt(SegmentReplicationTarget target) { // ensure target is not already closed. In that case // we can assume the store is not corrupt and that the replication diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 29410159a4955..009678b7502f8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -38,6 +38,7 @@ public class ReplicationCheckpoint implements Writeable, Comparable metadataMap; + private final long createdTimeStamp; public static ReplicationCheckpoint empty(ShardId shardId) { return empty(shardId, ""); @@ -55,10 +56,11 @@ private ReplicationCheckpoint(ShardId shardId, String codec) { length = 0L; this.codec = codec; this.metadataMap = Collections.emptyMap(); + this.createdTimeStamp = System.currentTimeMillis(); } public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen, long segmentInfosVersion, String codec) { - this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap()); + this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap(), System.currentTimeMillis()); } public ReplicationCheckpoint( @@ -68,7 +70,8 @@ public ReplicationCheckpoint( long segmentInfosVersion, long length, String codec, - Map metadataMap + Map metadataMap, + long createdTimeStamp ) { this.shardId = shardId; this.primaryTerm = primaryTerm; @@ -77,6 +80,7 @@ public ReplicationCheckpoint( this.length = length; this.codec = codec; this.metadataMap = metadataMap; + this.createdTimeStamp = createdTimeStamp; } public ReplicationCheckpoint(StreamInput in) throws IOException { @@ -96,6 +100,7 @@ public ReplicationCheckpoint(StreamInput in) throws IOException { } else { this.metadataMap = Collections.emptyMap(); } + this.createdTimeStamp = in.readLong(); } /** @@ -159,6 +164,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); } + out.writeLong(createdTimeStamp); } @Override @@ -197,6 +203,10 @@ public Map getMetadataMap() { return metadataMap; } + public long getCreatedTimeStamp() { + return createdTimeStamp; + } + @Override public String toString() { return "ReplicationCheckpoint{" @@ -212,6 +222,8 @@ public String toString() { + length + ", codec=" + codec + + ", timestamp=" + + createdTimeStamp + '}'; } } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 704a23890b07a..0c52571203594 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -949,7 +949,8 @@ protected Node( remoteStoreSettings, fileCache, compositeIndexSettings, - segmentReplicator::startReplication + segmentReplicator::startReplication, + segmentReplicator::getSegmentReplicationStats ); final IngestService ingestService = new IngestService( diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index bd86d3d396987..90f2b0b21cc8a 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -265,7 +265,8 @@ private IndexService newIndexService(IndexModule module) throws IOException { () -> IndexSettings.DEFAULT_REFRESH_INTERVAL, DefaultRecoverySettings.INSTANCE, DefaultRemoteStoreSettings.INSTANCE, - s -> {} + s -> {}, + null ); } diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index 233a99cbe4a73..899e80965e4fd 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -1844,7 +1844,8 @@ public void testSegmentReplicationCheckpointTracking() { 1, 1L, Codec.getDefault().getName(), - Map.of("segment_1", segment_1) + Map.of("segment_1", segment_1), + 0L ); final ReplicationCheckpoint secondCheckpoint = new ReplicationCheckpoint( tracker.shardId(), @@ -1853,7 +1854,8 @@ public void testSegmentReplicationCheckpointTracking() { 2, 51L, Codec.getDefault().getName(), - Map.of("segment_1", segment_1, "segment_2", segment_2) + Map.of("segment_1", segment_1, "segment_2", segment_2), + 0L ); final ReplicationCheckpoint thirdCheckpoint = new ReplicationCheckpoint( tracker.shardId(), @@ -1862,7 +1864,8 @@ public void testSegmentReplicationCheckpointTracking() { 3, 151L, Codec.getDefault().getName(), - Map.of("segment_1", segment_1, "segment_2", segment_2, "segment_3", segment_3) + Map.of("segment_1", segment_1, "segment_2", segment_2, "segment_3", segment_3), + 0L ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); @@ -1974,7 +1977,8 @@ public void testSegmentReplicationCheckpointForRelocatingPrimary() { 1, 5L, Codec.getDefault().getName(), - Map.of("segment_1", segment_1) + Map.of("segment_1", segment_1), + 0L ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); tracker.startReplicationLagTimers(initialCheckpoint); @@ -2033,7 +2037,8 @@ public void testSegmentReplicationCheckpointTrackingInvalidAllocationIDs() { 1, 1L, Codec.getDefault().getName(), - Collections.emptyMap() + Collections.emptyMap(), + 0L ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); tracker.startReplicationLagTimers(initialCheckpoint); diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java index 8b4b3aff701b4..fd26bdbf1068f 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java @@ -177,6 +177,9 @@ public void onFailure(Exception e) { logger.error("Unexpected onFailure", e); Assert.fail(); } + }, (ReplicationCheckpoint checkpoint, ShardId shardId) -> { + assertEquals(repCheckpoint, checkpoint); + assertEquals(shardId, spyIndexShard.shardId()); }); } @@ -230,7 +233,7 @@ public void onFailure(Exception e) { assertEquals(exception, e.getCause().getCause()); segrepTarget.fail(new ReplicationFailedException(e), false); } - }); + }, mock(BiConsumer.class)); } public void testFailureResponse_getSegmentFiles() { @@ -283,7 +286,7 @@ public void onFailure(Exception e) { assertEquals(exception, e.getCause().getCause()); segrepTarget.fail(new ReplicationFailedException(e), false); } - }); + }, mock(BiConsumer.class)); } public void testFailure_finalizeReplication_NonCorruptionException() throws IOException { @@ -330,7 +333,7 @@ public void onFailure(Exception e) { assertEquals(exception, e.getCause()); segrepTarget.fail(new ReplicationFailedException(e), false); } - }); + }, mock(BiConsumer.class)); } public void testFailure_finalizeReplication_IndexFormatException() throws IOException { @@ -376,7 +379,7 @@ public void onFailure(Exception e) { assertEquals(exception, e.getCause()); segrepTarget.fail(new ReplicationFailedException(e), false); } - }); + }, mock(BiConsumer.class)); } public void testFailure_differentSegmentFiles() throws IOException { @@ -429,7 +432,7 @@ public void onFailure(Exception e) { assertTrue(e.getMessage().contains("has local copies of segments that differ from the primary")); segrepTarget.fail(new ReplicationFailedException(e), false); } - }); + }, mock(BiConsumer.class)); } /** @@ -483,7 +486,7 @@ public void onFailure(Exception e) { logger.error("Unexpected onFailure", e); Assert.fail(); } - }); + }, mock(BiConsumer.class)); } /** diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 81ea16c80dd79..eda2ef58f4265 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -9,6 +9,8 @@ package org.opensearch.indices.replication; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.Version; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -20,6 +22,8 @@ import org.opensearch.common.lucene.Lucene; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.ReplicationStats; import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.replication.TestReplicationSource; import org.opensearch.index.shard.IndexShard; @@ -35,6 +39,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -200,6 +205,94 @@ public void getSegmentFiles( closeShards(primary, replica); } + public void testGetSegmentReplicationStats_WhenNoReplication() { + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + ShardId shardId = new ShardId("index", "uuid", 0); + ReplicationCheckpoint replicationCheckpoint = ReplicationCheckpoint.empty(shardId); + ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, replicationCheckpoint); + assertEquals(0, replicationStats.maxReplicationLag); + assertEquals(0, replicationStats.totalBytesBehind); + assertEquals(0, replicationStats.maxBytesBehind); + } + + public void testGetSegmentReplicationStats_WhenOnGoingReplication() { + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + ShardId shardId = new ShardId("index", "uuid", 0); + ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); + + BytesRef bytesRef = new BytesRef(500); + StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, bytesRef); + StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, bytesRef); + Map stringStoreFileMetadataMap = new HashMap<>(); + stringStoreFileMetadataMap.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMap.put("test-2", storeFileMetadata2); + ReplicationCheckpoint secondReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 2, + 2, + 2, + 1000, + "", + stringStoreFileMetadataMap, + System.currentTimeMillis() + ); + + segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); + + ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); + assertEquals(1000, replicationStats.totalBytesBehind); + assertEquals(1000, replicationStats.maxBytesBehind); + // Since we use System.currentTimeMillis() directly inside the method, actual value will vary + assertTrue(replicationStats.maxReplicationLag > 0); + } + + public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefreshedToNewCheckPoint() { + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + ShardId shardId = new ShardId("index", "uuid", 0); + ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); + + BytesRef bytesRef = new BytesRef(500); + StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, bytesRef); + StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, bytesRef); + Map stringStoreFileMetadataMap = new HashMap<>(); + stringStoreFileMetadataMap.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMap.put("test-2", storeFileMetadata2); + ReplicationCheckpoint secondReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 2, + 2, + 2, + 1000, + "", + stringStoreFileMetadataMap, + System.currentTimeMillis() + ); + + StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, bytesRef); + stringStoreFileMetadataMap.put("test-3", storeFileMetadata3); + + ReplicationCheckpoint thirdReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 3, + 3, + 3, + 1200, + "", + stringStoreFileMetadataMap, + System.currentTimeMillis() + ); + + segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); + segmentReplicator.updatePrimaryLastRefreshedCheckpoint(thirdReplicationCheckpoint, shardId); + + ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); + System.out.println(replicationStats); + assertEquals(1200, replicationStats.totalBytesBehind); + assertEquals(1200, replicationStats.maxBytesBehind); + // Since we use System.currentTimeMillis() directly inside the method, actual value will vary + assertTrue(replicationStats.maxReplicationLag > 0); + } + protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { try (final CopyState copyState = new CopyState(primary)) { listener.onResponse( diff --git a/server/src/test/java/org/opensearch/indices/replication/common/CopyStateTests.java b/server/src/test/java/org/opensearch/indices/replication/common/CopyStateTests.java index 0b30486038e3a..3b7c5560f89fb 100644 --- a/server/src/test/java/org/opensearch/indices/replication/common/CopyStateTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/common/CopyStateTests.java @@ -81,7 +81,8 @@ public static IndexShard createMockIndexShard() throws IOException { 0L, 0L, Codec.getDefault().getName(), - SI_SNAPSHOT.asMap() + SI_SNAPSHOT.asMap(), + 0L ); final Tuple, ReplicationCheckpoint> gatedCloseableReplicationCheckpointTuple = new Tuple<>( new GatedCloseable<>(testSegmentInfos, () -> {}), 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 655a9eb7d5d38..66fafc23b4f60 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 @@ -37,6 +37,8 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.junit.Assert; +import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.action.admin.indices.flush.FlushRequest; @@ -86,6 +88,7 @@ import org.opensearch.env.TestEnvironment; import org.opensearch.index.IndexSettings; import org.opensearch.index.MapperTestUtils; +import org.opensearch.index.ReplicationStats; import org.opensearch.index.VersionType; import org.opensearch.index.cache.IndexCache; import org.opensearch.index.cache.query.DisabledQueryCache; @@ -159,7 +162,6 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; -import org.junit.Assert; import java.io.IOException; import java.nio.file.Path; @@ -182,10 +184,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import org.mockito.Mockito; - -import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -194,6 +192,8 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; /** * A base class for unit tests that need to create and shutdown {@link IndexShard} instances easily, @@ -688,6 +688,10 @@ protected IndexShard newShard( } return new InternalTranslogFactory(); }; + //This is fine since we are not testing the node stats now + BiFunction mockReplicationStatsProvider = mock(BiFunction.class); + when(mockReplicationStatsProvider.apply(any(), any())) + .thenReturn(new ReplicationStats(800, 800, 500)); indexShard = new IndexShard( routing, indexSettings, @@ -717,7 +721,8 @@ protected IndexShard newShard( DefaultRecoverySettings.INSTANCE, DefaultRemoteStoreSettings.INSTANCE, false, - discoveryNodes + discoveryNodes, + mockReplicationStatsProvider ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); if (remoteStoreStatsTrackerFactory != null) { From 5d138e370e79e5a85f7abdaed7f281864072ee89 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Sat, 18 Jan 2025 19:11:54 -0800 Subject: [PATCH 02/25] Updated style checks in the test Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/index/shard/IndexShardTestCase.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) 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 66fafc23b4f60..bb91ba258e803 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 @@ -37,8 +37,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.junit.Assert; -import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.action.admin.indices.flush.FlushRequest; @@ -162,6 +160,7 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; +import org.junit.Assert; import java.io.IOException; import java.nio.file.Path; @@ -184,6 +183,10 @@ import java.util.function.Function; import java.util.stream.Collectors; +import org.mockito.Mockito; + +import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -192,8 +195,6 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; /** * A base class for unit tests that need to create and shutdown {@link IndexShard} instances easily, @@ -688,10 +689,9 @@ protected IndexShard newShard( } return new InternalTranslogFactory(); }; - //This is fine since we are not testing the node stats now + // This is fine since we are not testing the node stats now BiFunction mockReplicationStatsProvider = mock(BiFunction.class); - when(mockReplicationStatsProvider.apply(any(), any())) - .thenReturn(new ReplicationStats(800, 800, 500)); + when(mockReplicationStatsProvider.apply(any(), any())).thenReturn(new ReplicationStats(800, 800, 500)); indexShard = new IndexShard( routing, indexSettings, From 18664d2a8fb6ab113e20300d05e78d27aa8a6b2d Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Sat, 18 Jan 2025 19:13:52 -0800 Subject: [PATCH 03/25] Updated changelog Signed-off-by: Vinay Krishna Pudyodu --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 45bc56b505eb3..0fbf6ba3bffa3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Introduce a setting to disable download of full cluster state from remote on term mismatch([#16798](https://github.com/opensearch-project/OpenSearch/pull/16798/)) - Added ability to retrieve value from DocValues in a flat_object filed([#16802](https://github.com/opensearch-project/OpenSearch/pull/16802)) - Introduce framework for auxiliary transports and an experimental gRPC transport plugin ([#16534](https://github.com/opensearch-project/OpenSearch/pull/16534)) +- Implemented computation of segment replication stats at shard level ([#17055](https://github.com/opensearch-project/OpenSearch/pull/17055)) ### Dependencies - Bump `com.google.cloud:google-cloud-core-http` from 2.23.0 to 2.47.0 ([#16504](https://github.com/opensearch-project/OpenSearch/pull/16504)) From 346816497811e0262a8323a7c6cc8dd68b4f537f Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Sat, 18 Jan 2025 19:34:25 -0800 Subject: [PATCH 04/25] fixed style issues Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/indices/replication/SegmentReplicatorTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index eda2ef58f4265..83895800bd05c 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -286,7 +286,6 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefresh segmentReplicator.updatePrimaryLastRefreshedCheckpoint(thirdReplicationCheckpoint, shardId); ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); - System.out.println(replicationStats); assertEquals(1200, replicationStats.totalBytesBehind); assertEquals(1200, replicationStats.maxBytesBehind); // Since we use System.currentTimeMillis() directly inside the method, actual value will vary From 4e693a5db9a4367ea7c97017b35bd8ecdca5af13 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 21 Jan 2025 10:47:50 -0800 Subject: [PATCH 05/25] Fix the failing integration test Signed-off-by: Vinay Krishna Pudyodu --- .../indices/replication/SegmentReplicatorTests.java | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 83895800bd05c..63b5bbd94e503 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; +import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -46,8 +47,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import org.mockito.Mockito; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -242,8 +241,8 @@ public void testGetSegmentReplicationStats_WhenOnGoingReplication() { ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); assertEquals(1000, replicationStats.totalBytesBehind); assertEquals(1000, replicationStats.maxBytesBehind); - // Since we use System.currentTimeMillis() directly inside the method, actual value will vary - assertTrue(replicationStats.maxReplicationLag > 0); + // Since we use System.currentTimeMillis() directly inside the getSegmentReplicationStats method, actual value will vary + // Although there is a way to mock the Clock skipping it here for the simplicity } public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefreshedToNewCheckPoint() { @@ -288,8 +287,8 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefresh ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); assertEquals(1200, replicationStats.totalBytesBehind); assertEquals(1200, replicationStats.maxBytesBehind); - // Since we use System.currentTimeMillis() directly inside the method, actual value will vary - assertTrue(replicationStats.maxReplicationLag > 0); + // Since we use System.currentTimeMillis() directly inside the getSegmentReplicationStats method, actual value will vary + // Although there is a way to mock the Clock skipping it here for the simplicity } protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { From 5a3d1efa0a0fcc667adc50d941c53a75d30ae17b Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 21 Jan 2025 10:52:33 -0800 Subject: [PATCH 06/25] Fix stylecheck Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/indices/replication/SegmentReplicatorTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 63b5bbd94e503..8e4b59a7d869f 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,7 +11,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; -import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -47,6 +46,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; +import org.mockito.Mockito; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; From a94240f61ab4cc3d61257d6ea74226b203a0a62c Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 21 Jan 2025 12:08:22 -0800 Subject: [PATCH 07/25] Fixed the comments for the initial revision Signed-off-by: Vinay Krishna Pudyodu --- .../replication/SegmentReplicationTarget.java | 4 ---- .../indices/replication/SegmentReplicator.java | 18 +++++++++++------- .../checkpoint/ReplicationCheckpoint.java | 10 ++++++++-- 3 files changed, 19 insertions(+), 13 deletions(-) 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 8dfc4f45bc3c9..0c75389a8eb70 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -141,10 +141,6 @@ public ReplicationCheckpoint getCheckpoint() { return this.checkpoint; } - public SegmentReplicationSource getSource() { - return source; - } - @Override public void writeFileChunk( StoreFileMetadata metadata, 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 c540fa37607eb..d7666d9a100ed 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -13,7 +13,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.opensearch.OpenSearchCorruptionException; -import org.opensearch.action.StepListener; import org.opensearch.common.SetOnce; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; @@ -226,12 +225,17 @@ private void fetchPrimaryLastRefreshedCheckpoint(SegmentReplicationTarget target return; } - final StepListener checkpointInfoListener = new StepListener<>(); - target.getSource().getCheckpointMetadata(target.getId(), target.getCheckpoint(), checkpointInfoListener); - checkpointInfoListener.whenComplete( - checkpointInfo -> updatePrimaryLastRefreshedCheckpoint(checkpointInfo.getCheckpoint(), target.indexShard().shardId()), - checkpointInfoListener::onFailure - ); + sourceFactory.get().get(target.indexShard()).getCheckpointMetadata(target.getId(), target.getCheckpoint(), new ActionListener<>() { + @Override + public void onResponse(CheckpointInfoResponse checkpointInfoResponse) { + updatePrimaryLastRefreshedCheckpoint(checkpointInfoResponse.getCheckpoint(), target.indexShard().shardId()); + } + + @Override + public void onFailure(Exception e) { + logger.error("Failed to fetch primary last refreshed checkpoint", e); + } + }); } private boolean isStoreCorrupt(SegmentReplicationTarget target) { diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 009678b7502f8..af8c289214e47 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -100,7 +100,11 @@ public ReplicationCheckpoint(StreamInput in) throws IOException { } else { this.metadataMap = Collections.emptyMap(); } - this.createdTimeStamp = in.readLong(); + if (in.getVersion().onOrAfter(Version.V_2_19_0)) { + this.createdTimeStamp = in.readLong(); + } else { + this.createdTimeStamp = 0; + } } /** @@ -164,7 +168,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); } - out.writeLong(createdTimeStamp); + if (out.getVersion().onOrAfter(Version.V_2_19_0)) { + out.writeLong(createdTimeStamp); + } } @Override From dd0406d60c7868ee626699296bd5de7d5b73497c Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 21 Jan 2025 12:23:52 -0800 Subject: [PATCH 08/25] Updated to use System.nanoTime() for lag calculation Signed-off-by: Vinay Krishna Pudyodu --- .../main/java/org/opensearch/index/shard/IndexShard.java | 2 +- .../opensearch/indices/replication/SegmentReplicator.java | 3 ++- .../replication/checkpoint/ReplicationCheckpoint.java | 4 ++-- .../indices/replication/SegmentReplicatorTests.java | 6 +++--- 4 files changed, 8 insertions(+), 7 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 ca3aefd423984..e931d7944ffe3 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1788,7 +1788,7 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th metadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), getEngine().config().getCodec().getName(), metadataMap, - System.currentTimeMillis() + System.nanoTime() ); logger.trace("Recomputed ReplicationCheckpoint for shard {}", checkpoint); return checkpoint; 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 d7666d9a100ed..1a7fafc4a05cd 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; /** * This class is responsible for managing segment replication events on replicas. @@ -124,7 +125,7 @@ public ReplicationStats getSegmentReplicationStats(ShardId shardId, ReplicationC final ReplicationCheckpoint lastOnGoingReplicationCheckpoint = this.lastOnGoingReplicationCheckpoint.get(shardId); final long replicationLag = lastOnGoingReplicationCheckpoint != null - ? System.currentTimeMillis() - lastOnGoingReplicationCheckpoint.getCreatedTimeStamp() + ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastOnGoingReplicationCheckpoint.getCreatedTimeStamp()) : 0; return new ReplicationStats(bytesBehindSum, bytesBehindSum, bytesBehindSum > 0L ? replicationLag : 0); diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index af8c289214e47..0fc5c0d42d4e2 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -56,11 +56,11 @@ private ReplicationCheckpoint(ShardId shardId, String codec) { length = 0L; this.codec = codec; this.metadataMap = Collections.emptyMap(); - this.createdTimeStamp = System.currentTimeMillis(); + this.createdTimeStamp = System.nanoTime(); } public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen, long segmentInfosVersion, String codec) { - this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap(), System.currentTimeMillis()); + this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap(), System.nanoTime()); } public ReplicationCheckpoint( diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 8e4b59a7d869f..fee5cc90f4df3 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -234,7 +234,7 @@ public void testGetSegmentReplicationStats_WhenOnGoingReplication() { 1000, "", stringStoreFileMetadataMap, - System.currentTimeMillis() + System.nanoTime() ); segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); @@ -265,7 +265,7 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefresh 1000, "", stringStoreFileMetadataMap, - System.currentTimeMillis() + System.nanoTime() ); StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, bytesRef); @@ -279,7 +279,7 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefresh 1200, "", stringStoreFileMetadataMap, - System.currentTimeMillis() + System.nanoTime() ); segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); From 1104c1f294ddbd62215e62884a01cb2b4bee41a8 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 21 Jan 2025 22:33:21 -0800 Subject: [PATCH 09/25] Fixed the integration test for node stats Signed-off-by: Vinay Krishna Pudyodu --- .../replication/SegmentReplicationStatsIT.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java index 89aef6f0be1a6..f5abc66ab90a1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java @@ -404,19 +404,17 @@ public void testSegmentReplicationNodeAndIndexStats() throws Exception { for (NodeStats nodeStats : nodesStatsResponse.getNodes()) { ReplicationStats replicationStats = nodeStats.getIndices().getSegments().getReplicationStats(); - // primary node - should hold replication statistics + // primary node - do not have any replication statistics if (nodeStats.getNode().getName().equals(primaryNode)) { - assertTrue(replicationStats.getMaxBytesBehind() > 0); - assertTrue(replicationStats.getTotalBytesBehind() > 0); - assertTrue(replicationStats.getMaxReplicationLag() > 0); - // 2 replicas so total bytes should be double of max - assertEquals(replicationStats.getMaxBytesBehind() * 2, replicationStats.getTotalBytesBehind()); + assertTrue(replicationStats.getMaxBytesBehind() == 0); + assertTrue(replicationStats.getTotalBytesBehind() == 0); + assertTrue(replicationStats.getMaxReplicationLag() == 0); } // replica nodes - should hold empty replication statistics if (nodeStats.getNode().getName().equals(replicaNode1) || nodeStats.getNode().getName().equals(replicaNode2)) { - assertEquals(0, replicationStats.getMaxBytesBehind()); - assertEquals(0, replicationStats.getTotalBytesBehind()); - assertEquals(0, replicationStats.getMaxReplicationLag()); + assertTrue(replicationStats.getMaxBytesBehind() > 0); + assertTrue(replicationStats.getTotalBytesBehind() > 0); + assertTrue(replicationStats.getMaxReplicationLag() > 0); } } // get replication statistics at index level From 90b96a84d5f3788ce149ff8c7fd23edcf35ec6fc Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Wed, 22 Jan 2025 16:19:01 -0800 Subject: [PATCH 10/25] Modified the version in the ReplicationCheckpoint for backward compatibility Signed-off-by: Vinay Krishna Pudyodu --- .../indices/replication/checkpoint/ReplicationCheckpoint.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 0fc5c0d42d4e2..d609e35373c8f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -100,7 +100,7 @@ public ReplicationCheckpoint(StreamInput in) throws IOException { } else { this.metadataMap = Collections.emptyMap(); } - if (in.getVersion().onOrAfter(Version.V_2_19_0)) { + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { this.createdTimeStamp = in.readLong(); } else { this.createdTimeStamp = 0; @@ -168,7 +168,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); } - if (out.getVersion().onOrAfter(Version.V_2_19_0)) { + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { out.writeLong(createdTimeStamp); } } From 7f465a007eaa9171d8fc38af510d7f6b32eb5638 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 27 Jan 2025 14:21:44 -0800 Subject: [PATCH 11/25] Added precomputation logic for the stats calculation Signed-off-by: Vinay Krishna Pudyodu --- .../SegmentReplicationStatsIT.java | 2 +- .../org/opensearch/index/IndexModule.java | 5 +- .../org/opensearch/index/IndexService.java | 7 +- .../opensearch/index/ReplicationStats.java | 4 + .../opensearch/index/shard/IndexShard.java | 8 +- .../opensearch/indices/IndicesService.java | 4 +- .../RemoteStoreReplicationSource.java | 11 +- .../replication/SegmentReplicationTarget.java | 5 +- .../SegmentReplicationTargetService.java | 2 +- .../replication/SegmentReplicator.java | 117 ++++++++++++------ .../SegmentReplicationTargetTests.java | 9 +- .../replication/SegmentReplicatorTests.java | 102 +++++++-------- .../index/shard/IndexShardTestCase.java | 13 +- 13 files changed, 167 insertions(+), 122 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java index f5abc66ab90a1..5d69799e32647 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationStatsIT.java @@ -410,7 +410,7 @@ public void testSegmentReplicationNodeAndIndexStats() throws Exception { assertTrue(replicationStats.getTotalBytesBehind() == 0); assertTrue(replicationStats.getMaxReplicationLag() == 0); } - // replica nodes - should hold empty replication statistics + // replica nodes - should hold replication statistics if (nodeStats.getNode().getName().equals(replicaNode1) || nodeStats.getNode().getName().equals(replicaNode2)) { assertTrue(replicationStats.getMaxBytesBehind() > 0); assertTrue(replicationStats.getTotalBytesBehind() > 0); diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index c55e5e1c90778..d5843318a675f 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -88,7 +88,6 @@ import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.repositories.RepositoriesService; import org.opensearch.script.ScriptService; @@ -656,7 +655,7 @@ public IndexService newIndexService( recoverySettings, remoteStoreSettings, (s) -> {}, - null + shardId -> ReplicationStats.empty() ); } @@ -683,7 +682,7 @@ public IndexService newIndexService( RecoverySettings recoverySettings, RemoteStoreSettings remoteStoreSettings, Consumer replicator, - BiFunction segmentReplicationStatsProvider + Function segmentReplicationStatsProvider ) throws IOException { final IndexEventListener eventListener = freeze(); Function> readerWrapperFactory = indexReaderWrapper diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 69a950e604ecb..54a74c0115c9b 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -105,7 +105,6 @@ import org.opensearch.indices.mapper.MapperRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.plugins.IndexStorePlugin; @@ -198,7 +197,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final FileCache fileCache; private final CompositeIndexSettings compositeIndexSettings; private final Consumer replicator; - private final BiFunction segmentReplicationStatsProvider; + private final Function segmentReplicationStatsProvider; public IndexService( IndexSettings indexSettings, @@ -238,7 +237,7 @@ public IndexService( FileCache fileCache, CompositeIndexSettings compositeIndexSettings, Consumer replicator, - BiFunction segmentReplicationStatsProvider + Function segmentReplicationStatsProvider ) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; @@ -400,7 +399,7 @@ public IndexService( null, null, s -> {}, - null + (shardId) -> ReplicationStats.empty() ); } diff --git a/server/src/main/java/org/opensearch/index/ReplicationStats.java b/server/src/main/java/org/opensearch/index/ReplicationStats.java index 8987a492e9a90..22628e86d309f 100644 --- a/server/src/main/java/org/opensearch/index/ReplicationStats.java +++ b/server/src/main/java/org/opensearch/index/ReplicationStats.java @@ -42,6 +42,10 @@ public ReplicationStats(StreamInput in) throws IOException { this.maxReplicationLag = in.readVLong(); } + public static ReplicationStats empty() { + return new ReplicationStats(); + } + public ReplicationStats() { } 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 e931d7944ffe3..425a5079138a7 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -361,7 +361,7 @@ Runnable getGlobalCheckpointSyncer() { */ private final ShardMigrationState shardMigrationState; private DiscoveryNodes discoveryNodes; - private final BiFunction segmentReplicationStatsProvider; + private final Function segmentReplicationStatsProvider; public IndexShard( final ShardRouting shardRouting, @@ -393,7 +393,7 @@ public IndexShard( final RemoteStoreSettings remoteStoreSettings, boolean seedRemote, final DiscoveryNodes discoveryNodes, - final BiFunction segmentReplicationStatsProvider + final Function segmentReplicationStatsProvider ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -3233,9 +3233,9 @@ public Set getReplicationStatsForTrackedReplicas() public ReplicationStats getReplicationStats() { if (indexSettings.isSegRepEnabledOrRemoteNode() && !routingEntry().primary()) { - return segmentReplicationStatsProvider.apply(shardId, this.getLatestReplicationCheckpoint()); + return segmentReplicationStatsProvider.apply(shardId); } - return new ReplicationStats(0, 0, 0); + return ReplicationStats.empty(); } /** diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index cbfcfb45d4aa2..fc3e8f2a4555b 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -363,7 +363,7 @@ public class IndicesService extends AbstractLifecycleComponent private final FileCache fileCache; private final CompositeIndexSettings compositeIndexSettings; private final Consumer replicator; - private final BiFunction segmentReplicationStatsProvider; + private final Function segmentReplicationStatsProvider; private volatile int maxSizeInRequestCache; @Override @@ -403,7 +403,7 @@ public IndicesService( FileCache fileCache, CompositeIndexSettings compositeIndexSettings, Consumer replicator, - BiFunction segmentReplicationStatsProvider + Function segmentReplicationStatsProvider ) { this.settings = settings; this.threadPool = threadPool; diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index b06b3e0497cf7..aaeac12ec9d89 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -110,7 +110,7 @@ public void getSegmentFiles( return; } logger.debug("Downloading segment files from remote store {}", filesToFetch); - + slowByStringOperations(16); if (remoteMetadataExists()) { final Directory storeDirectory = indexShard.store().directory(); final Collection directoryFiles = List.of(storeDirectory.listAll()); @@ -136,6 +136,15 @@ public void getSegmentFiles( } } + public static void slowByStringOperations(int intensity) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < intensity * 10000; i++) { + sb.append(String.valueOf(i)); + sb.reverse(); + } + System.out.println("Slow Operation finished"); + } + @Override public void cancel() { this.cancellableThreads.cancel("Canceled by target"); 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 0c75389a8eb70..64bd73ebb4611 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -24,7 +24,6 @@ import org.opensearch.common.util.CancellableThreads; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; @@ -163,7 +162,7 @@ public void writeFileChunk( * * @param listener {@link ActionListener} listener. */ - public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { + public void startReplication(ActionListener listener, BiConsumer checkpointUpdater) { cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { throw new CancellableThreads.ExecutionCancelledException("replication was canceled reason [" + reason + "]"); }); @@ -179,7 +178,7 @@ public void startReplication(ActionListener listener, BiConsumer { - checkpointUpdater.accept(checkpointInfo.getCheckpoint(), this.indexShard.shardId()); + checkpointUpdater.accept(checkpointInfo.getCheckpoint(), this.indexShard); final List filesToFetch = getFiles(checkpointInfo); state.setStage(SegmentReplicationState.Stage.GET_FILES); 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 bd11f1e3d02ec..59be56f2519c8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -294,7 +294,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe return; } updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); - replicator.updatePrimaryLastRefreshedCheckpoint(receivedCheckpoint, replicaShard.shardId()); + replicator.updateReplicationCheckpointStats(receivedCheckpoint, replicaShard); // Checks if replica shard is in the correct STARTED state to process checkpoints (avoids parallel replication events taking place) // This check ensures we do not try to process a received checkpoint while the shard is still recovering, yet we stored the latest // checkpoint to be replayed once the shard is Active. 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 1a7fafc4a05cd..76f96469f8a67 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -31,7 +31,9 @@ import java.io.IOException; import java.util.Map; -import java.util.Objects; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.TimeUnit; /** @@ -47,8 +49,8 @@ public class SegmentReplicator { private final ReplicationCollection onGoingReplications; private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); - private final Map primaryLastRefreshedCheckpoint = ConcurrentCollections.newConcurrentMap(); - private final Map lastOnGoingReplicationCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap latestCheckpoint = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -107,47 +109,87 @@ SegmentReplicationTarget startReplication( return target; } - public ReplicationStats getSegmentReplicationStats(ShardId shardId, ReplicationCheckpoint indexReplicationCheckPoint) { - assert shardId != null : "shardId cannot be null"; - assert indexReplicationCheckPoint != null : "indexReplicationCheckPoint cannot be null"; - ; - final Map indexStoreFileMetadata = indexReplicationCheckPoint.getMetadataMap(); - // If primaryLastRefreshedCheckpoint is null, we will default to indexReplicationCheckPoint - // so that we can avoid any failures - final ReplicationCheckpoint primaryLastRefreshedCheckpoint = Objects.requireNonNullElse( - this.primaryLastRefreshedCheckpoint.get(shardId), - indexReplicationCheckPoint - ); - final Map storeFileMetadata = primaryLastRefreshedCheckpoint.getMetadataMap(); + public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); + if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { + return ReplicationStats.empty(); + } - final Store.RecoveryDiff diff = Store.segmentReplicationDiff(storeFileMetadata, indexStoreFileMetadata); - long bytesBehindSum = diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); + Map.Entry lowestEntry = existingCheckpointStats.firstEntry(); + Map.Entry highestEntry = existingCheckpointStats.lastEntry(); - final ReplicationCheckpoint lastOnGoingReplicationCheckpoint = this.lastOnGoingReplicationCheckpoint.get(shardId); - final long replicationLag = lastOnGoingReplicationCheckpoint != null - ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastOnGoingReplicationCheckpoint.getCreatedTimeStamp()) - : 0; + long bytesBehind = highestEntry.getValue().getBytesBehind(); + long replicationLag = bytesBehind > 0L ? + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) : 0; - return new ReplicationStats(bytesBehindSum, bytesBehindSum, bytesBehindSum > 0L ? replicationLag : 0); + return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); } - public void updatePrimaryLastRefreshedCheckpoint(ReplicationCheckpoint replicationCheckpoint, ShardId shardId) { - updateCheckpointIfAhead(primaryLastRefreshedCheckpoint, replicationCheckpoint, shardId); + public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestCheckPoint, final IndexShard indexShard) { + latestCheckpoint.put(indexShard.shardId(), latestCheckPoint); + ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.computeIfAbsent( + indexShard.shardId(), + k -> new ConcurrentSkipListMap<>() + ); + + long segmentInfosVersion = latestCheckPoint.getSegmentInfosVersion(); + if (!existingCheckpointStats.containsKey(segmentInfosVersion)) { + ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); + long bytesBehind = calculateBytesBehind(latestCheckPoint, replicationCheckpoint); + + if (bytesBehind > 0) { + existingCheckpointStats.put( + segmentInfosVersion, + new ReplicationCheckpointStats(bytesBehind, latestCheckPoint.getCreatedTimeStamp()) + ); + } + } } - public void updateReplicationCheckpoints(ReplicationCheckpoint replicationCheckpoint, ShardId shardId) { - updateCheckpointIfAhead(lastOnGoingReplicationCheckpoint, replicationCheckpoint, shardId); - updatePrimaryLastRefreshedCheckpoint(replicationCheckpoint, shardId); + protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { + ReplicationCheckpoint latestCheckpoint = this.latestCheckpoint.get(indexShard.shardId()); + + ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); + long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(indexShard.shardId()); + + existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); + Map.Entry lastEntry = existingCheckpointStats.lastEntry(); + if(lastEntry != null) { + lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); + } } - private void updateCheckpointIfAhead( - Map checkpointMap, - ReplicationCheckpoint newCheckpoint, - ShardId shardId - ) { - final ReplicationCheckpoint existingCheckpoint = checkpointMap.get(shardId); - if (existingCheckpoint == null || newCheckpoint.isAheadOf(existingCheckpoint)) { - checkpointMap.put(shardId, newCheckpoint); + private long calculateBytesBehind(final ReplicationCheckpoint latestCheckPoint, final ReplicationCheckpoint replicationCheckpoint) { + Store.RecoveryDiff diff = Store.segmentReplicationDiff( + latestCheckPoint.getMetadataMap(), + replicationCheckpoint.getMetadataMap() + ); + + return diff.missing.stream() + .mapToLong(StoreFileMetadata::length) + .sum(); + } + + private static class ReplicationCheckpointStats { + private long bytesBehind; + private final long timestamp; + + public ReplicationCheckpointStats(long bytesBehind, long timestamp) { + this.bytesBehind = bytesBehind; + this.timestamp = timestamp; + } + + public long getBytesBehind() { + return bytesBehind; + } + + public void setBytesBehind(long bytesBehind) { + this.bytesBehind = bytesBehind; + } + + public long getTimestamp() { + return timestamp; } } @@ -187,6 +229,7 @@ private void start(final long replicationId) { @Override 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) { completedReplications.put(target.shardId(), target.state()); @@ -202,7 +245,7 @@ public void onFailure(Exception e) { } onGoingReplications.fail(replicationId, new ReplicationFailedException("Segment Replication failed", e), false); } - }, this::updateReplicationCheckpoints); + }, this::updateReplicationCheckpointStats); } // pkg-private for integration tests @@ -229,7 +272,7 @@ private void fetchPrimaryLastRefreshedCheckpoint(SegmentReplicationTarget target sourceFactory.get().get(target.indexShard()).getCheckpointMetadata(target.getId(), target.getCheckpoint(), new ActionListener<>() { @Override public void onResponse(CheckpointInfoResponse checkpointInfoResponse) { - updatePrimaryLastRefreshedCheckpoint(checkpointInfoResponse.getCheckpoint(), target.indexShard().shardId()); + updateReplicationCheckpointStats(checkpointInfoResponse.getCheckpoint(), target.indexShard()); } @Override diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java index fd26bdbf1068f..c40dc37783ab8 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java @@ -24,6 +24,8 @@ import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.Version; +import org.junit.Assert; +import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.metadata.IndexMetadata; @@ -44,7 +46,6 @@ import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.DummyShardLock; import org.opensearch.test.IndexSettingsModule; -import org.junit.Assert; import java.io.FileNotFoundException; import java.io.IOException; @@ -56,8 +57,6 @@ import java.util.Random; import java.util.function.BiConsumer; -import org.mockito.Mockito; - import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -177,9 +176,9 @@ public void onFailure(Exception e) { logger.error("Unexpected onFailure", e); Assert.fail(); } - }, (ReplicationCheckpoint checkpoint, ShardId shardId) -> { + }, (ReplicationCheckpoint checkpoint, IndexShard indexShard) -> { assertEquals(repCheckpoint, checkpoint); - assertEquals(shardId, spyIndexShard.shardId()); + assertEquals(indexShard, spyIndexShard); }); } diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index fee5cc90f4df3..e0c9ee1f005c3 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; +import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -43,11 +44,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import org.mockito.Mockito; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -208,55 +208,30 @@ public void getSegmentFiles( public void testGetSegmentReplicationStats_WhenNoReplication() { SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); ShardId shardId = new ShardId("index", "uuid", 0); - ReplicationCheckpoint replicationCheckpoint = ReplicationCheckpoint.empty(shardId); - ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, replicationCheckpoint); + ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId); assertEquals(0, replicationStats.maxReplicationLag); assertEquals(0, replicationStats.totalBytesBehind); assertEquals(0, replicationStats.maxBytesBehind); } - public void testGetSegmentReplicationStats_WhenOnGoingReplication() { - SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + public void testGetSegmentReplicationStats_When() { ShardId shardId = new ShardId("index", "uuid", 0); - ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); - - BytesRef bytesRef = new BytesRef(500); - StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, bytesRef); - StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, bytesRef); - Map stringStoreFileMetadataMap = new HashMap<>(); - stringStoreFileMetadataMap.put("test-1", storeFileMetadata1); - stringStoreFileMetadataMap.put("test-2", storeFileMetadata2); - ReplicationCheckpoint secondReplicationCheckpoint = new ReplicationCheckpoint( - shardId, - 2, - 2, - 2, - 1000, - "", - stringStoreFileMetadataMap, - System.nanoTime() - ); - - segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); - - ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); - assertEquals(1000, replicationStats.totalBytesBehind); - assertEquals(1000, replicationStats.maxBytesBehind); - // Since we use System.currentTimeMillis() directly inside the getSegmentReplicationStats method, actual value will vary - // Although there is a way to mock the Clock skipping it here for the simplicity + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId); + assertEquals(0, replicationStats.maxReplicationLag); + assertEquals(0, replicationStats.maxBytesBehind); + assertEquals(0, replicationStats.totalBytesBehind); } - public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefreshedToNewCheckPoint() { - SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefreshedToNewCheckPoint() { ShardId shardId = new ShardId("index", "uuid", 0); ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); - BytesRef bytesRef = new BytesRef(500); - StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, bytesRef); - StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, bytesRef); - Map stringStoreFileMetadataMap = new HashMap<>(); - stringStoreFileMetadataMap.put("test-1", storeFileMetadata1); - stringStoreFileMetadataMap.put("test-2", storeFileMetadata2); + StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, new BytesRef(500)); + StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, new BytesRef(500)); + Map stringStoreFileMetadataMapOne = new HashMap<>(); + stringStoreFileMetadataMapOne.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMapOne.put("test-2", storeFileMetadata2); ReplicationCheckpoint secondReplicationCheckpoint = new ReplicationCheckpoint( shardId, 2, @@ -264,32 +239,51 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationPrimaryRefresh 2, 1000, "", - stringStoreFileMetadataMap, - System.nanoTime() + stringStoreFileMetadataMapOne, + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) ); - StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, bytesRef); - stringStoreFileMetadataMap.put("test-3", storeFileMetadata3); + IndexShard replicaShard = mock(IndexShard.class); + when(replicaShard.shardId()).thenReturn(shardId); + when(replicaShard.getLatestReplicationCheckpoint()) + .thenReturn(firstReplicationCheckpoint) + .thenReturn(firstReplicationCheckpoint) + .thenReturn(firstReplicationCheckpoint) + .thenReturn(secondReplicationCheckpoint); + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + segmentReplicator.updateReplicationCheckpointStats(firstReplicationCheckpoint, replicaShard); + segmentReplicator.updateReplicationCheckpointStats(secondReplicationCheckpoint, replicaShard); + + Map stringStoreFileMetadataMapTwo = new HashMap<>(); + StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, new BytesRef(200)); + stringStoreFileMetadataMapTwo.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMapTwo.put("test-2", storeFileMetadata2); + stringStoreFileMetadataMapTwo.put("test-3", storeFileMetadata3); ReplicationCheckpoint thirdReplicationCheckpoint = new ReplicationCheckpoint( shardId, 3, 3, 3, - 1200, + 200, "", - stringStoreFileMetadataMap, - System.nanoTime() + stringStoreFileMetadataMapTwo, + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) ); - segmentReplicator.updateReplicationCheckpoints(secondReplicationCheckpoint, shardId); - segmentReplicator.updatePrimaryLastRefreshedCheckpoint(thirdReplicationCheckpoint, shardId); + segmentReplicator.updateReplicationCheckpointStats(thirdReplicationCheckpoint, replicaShard); + + ReplicationStats replicationStatsFirst = segmentReplicator.getSegmentReplicationStats(shardId); + assertEquals(1200, replicationStatsFirst.totalBytesBehind); + assertEquals(1200, replicationStatsFirst.maxBytesBehind); + assertTrue(replicationStatsFirst.maxReplicationLag > 0); + + segmentReplicator.pruneCheckpointsUpToLastSync(replicaShard); - ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId, firstReplicationCheckpoint); - assertEquals(1200, replicationStats.totalBytesBehind); - assertEquals(1200, replicationStats.maxBytesBehind); - // Since we use System.currentTimeMillis() directly inside the getSegmentReplicationStats method, actual value will vary - // Although there is a way to mock the Clock skipping it here for the simplicity + ReplicationStats replicationStatsSecond = segmentReplicator.getSegmentReplicationStats(shardId); + assertEquals(200, replicationStatsSecond.totalBytesBehind); + assertEquals(200, replicationStatsSecond.maxBytesBehind); + assertTrue(replicationStatsSecond.maxReplicationLag > 0); } protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { 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 bb91ba258e803..e590285c32959 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 @@ -37,6 +37,8 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.junit.Assert; +import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.action.admin.indices.flush.FlushRequest; @@ -160,7 +162,6 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; -import org.junit.Assert; import java.io.IOException; import java.nio.file.Path; @@ -183,10 +184,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import org.mockito.Mockito; - -import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -195,6 +192,8 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; /** * A base class for unit tests that need to create and shutdown {@link IndexShard} instances easily, @@ -690,8 +689,8 @@ protected IndexShard newShard( return new InternalTranslogFactory(); }; // This is fine since we are not testing the node stats now - BiFunction mockReplicationStatsProvider = mock(BiFunction.class); - when(mockReplicationStatsProvider.apply(any(), any())).thenReturn(new ReplicationStats(800, 800, 500)); + Function mockReplicationStatsProvider = mock(Function.class); + when(mockReplicationStatsProvider.apply(any())).thenReturn(new ReplicationStats(800, 800, 500)); indexShard = new IndexShard( routing, indexSettings, From 28f1cfcb9f3e2eda7a85d87c7b882a0c38dc2805 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 27 Jan 2025 14:31:51 -0800 Subject: [PATCH 12/25] Removed unwanted lines Signed-off-by: Vinay Krishna Pudyodu --- .../replication/RemoteStoreReplicationSource.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index aaeac12ec9d89..30d9c362b6269 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -110,7 +110,6 @@ public void getSegmentFiles( return; } logger.debug("Downloading segment files from remote store {}", filesToFetch); - slowByStringOperations(16); if (remoteMetadataExists()) { final Directory storeDirectory = indexShard.store().directory(); final Collection directoryFiles = List.of(storeDirectory.listAll()); @@ -136,15 +135,6 @@ public void getSegmentFiles( } } - public static void slowByStringOperations(int intensity) { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < intensity * 10000; i++) { - sb.append(String.valueOf(i)); - sb.reverse(); - } - System.out.println("Slow Operation finished"); - } - @Override public void cancel() { this.cancellableThreads.cancel("Canceled by target"); From f80791f14a8b6dcd37bbdc5853998e53d07a544e Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 27 Jan 2025 15:29:32 -0800 Subject: [PATCH 13/25] Clean up the maps when index closed Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/index/shard/IndexShardIT.java | 4 +- .../opensearch/indices/IndicesService.java | 1 - .../SegmentReplicationTargetService.java | 2 +- .../replication/SegmentReplicator.java | 48 ++++++++++--------- .../SegmentReplicationTargetTests.java | 5 +- .../replication/SegmentReplicatorTests.java | 8 ++-- .../index/shard/IndexShardTestCase.java | 9 ++-- 7 files changed, 41 insertions(+), 36 deletions(-) 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 bd959938248bb..2d0918ff6e89a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -114,7 +114,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Stream; @@ -719,7 +719,7 @@ public static final IndexShard newIndexShard( DefaultRemoteStoreSettings.INSTANCE, false, IndexShardTestUtils.getFakeDiscoveryNodes(initializingShardRouting), - mock(BiFunction.class) + mock(Function.class) ); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 1ba1dfded734e..fd65559e86a0a 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -152,7 +152,6 @@ import org.opensearch.indices.recovery.RecoveryListener; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; -import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.Node; 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 59be56f2519c8..0a74954110815 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -293,7 +293,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe logger.trace(() -> "Ignoring checkpoint, Shard is closed"); return; } - updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); + // updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); replicator.updateReplicationCheckpointStats(receivedCheckpoint, replicaShard); // Checks if replica shard is in the correct STARTED state to process checkpoints (avoids parallel replication events taking place) // This check ensures we do not try to process a received checkpoint while the shard is still recovering, yet we stored the latest 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 76f96469f8a67..2586d58811d07 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -49,8 +49,9 @@ public class SegmentReplicator { private final ReplicationCollection onGoingReplications; private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap latestCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap> replicationCheckpointStats = + ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap latestReceivedCheckpoint = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -119,14 +120,15 @@ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { Map.Entry highestEntry = existingCheckpointStats.lastEntry(); long bytesBehind = highestEntry.getValue().getBytesBehind(); - long replicationLag = bytesBehind > 0L ? - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) : 0; + long replicationLag = bytesBehind > 0L + ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) + : 0; return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); } public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestCheckPoint, final IndexShard indexShard) { - latestCheckpoint.put(indexShard.shardId(), latestCheckPoint); + latestReceivedCheckpoint.put(indexShard.shardId(), latestCheckPoint); ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.computeIfAbsent( indexShard.shardId(), k -> new ConcurrentSkipListMap<>() @@ -147,28 +149,28 @@ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestC } protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { - ReplicationCheckpoint latestCheckpoint = this.latestCheckpoint.get(indexShard.shardId()); - - ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); - long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(indexShard.shardId()); - - existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); - Map.Entry lastEntry = existingCheckpointStats.lastEntry(); - if(lastEntry != null) { - lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); + ReplicationCheckpoint latestCheckpoint = this.latestReceivedCheckpoint.get(indexShard.shardId()); + if (latestCheckpoint != null) { + ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); + long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( + indexShard.shardId() + ); + + if (existingCheckpointStats != null && !existingCheckpointStats.isEmpty()) { + existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); + Map.Entry lastEntry = existingCheckpointStats.lastEntry(); + if (lastEntry != null) { + lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); + } + } } } private long calculateBytesBehind(final ReplicationCheckpoint latestCheckPoint, final ReplicationCheckpoint replicationCheckpoint) { - Store.RecoveryDiff diff = Store.segmentReplicationDiff( - latestCheckPoint.getMetadataMap(), - replicationCheckpoint.getMetadataMap() - ); + Store.RecoveryDiff diff = Store.segmentReplicationDiff(latestCheckPoint.getMetadataMap(), replicationCheckpoint.getMetadataMap()); - return diff.missing.stream() - .mapToLong(StoreFileMetadata::length) - .sum(); + return diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); } private static class ReplicationCheckpointStats { @@ -309,6 +311,8 @@ int size() { void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); + replicationCheckpointStats.remove(shardId); + latestReceivedCheckpoint.remove(shardId); } SegmentReplicationTarget get(ShardId shardId) { diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java index c40dc37783ab8..52cb39bebd2b7 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java @@ -24,8 +24,6 @@ import org.apache.lucene.tests.analysis.MockAnalyzer; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.Version; -import org.junit.Assert; -import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.metadata.IndexMetadata; @@ -46,6 +44,7 @@ import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.DummyShardLock; import org.opensearch.test.IndexSettingsModule; +import org.junit.Assert; import java.io.FileNotFoundException; import java.io.IOException; @@ -57,6 +56,8 @@ import java.util.Random; import java.util.function.BiConsumer; +import org.mockito.Mockito; + import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index e0c9ee1f005c3..d02b45492d698 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,7 +11,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; -import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -48,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; +import org.mockito.Mockito; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -245,8 +246,7 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefr IndexShard replicaShard = mock(IndexShard.class); when(replicaShard.shardId()).thenReturn(shardId); - when(replicaShard.getLatestReplicationCheckpoint()) - .thenReturn(firstReplicationCheckpoint) + when(replicaShard.getLatestReplicationCheckpoint()).thenReturn(firstReplicationCheckpoint) .thenReturn(firstReplicationCheckpoint) .thenReturn(firstReplicationCheckpoint) .thenReturn(secondReplicationCheckpoint); @@ -256,7 +256,7 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefr segmentReplicator.updateReplicationCheckpointStats(secondReplicationCheckpoint, replicaShard); Map stringStoreFileMetadataMapTwo = new HashMap<>(); - StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, new BytesRef(200)); + StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, new BytesRef(200)); stringStoreFileMetadataMapTwo.put("test-1", storeFileMetadata1); stringStoreFileMetadataMapTwo.put("test-2", storeFileMetadata2); stringStoreFileMetadataMapTwo.put("test-3", storeFileMetadata3); 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 e590285c32959..bdd4b40e398d5 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 @@ -37,8 +37,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.junit.Assert; -import org.mockito.Mockito; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.action.admin.indices.flush.FlushRequest; @@ -162,6 +160,7 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; +import org.junit.Assert; import java.io.IOException; import java.nio.file.Path; @@ -184,6 +183,10 @@ import java.util.function.Function; import java.util.stream.Collectors; +import org.mockito.Mockito; + +import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -192,8 +195,6 @@ import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; /** * A base class for unit tests that need to create and shutdown {@link IndexShard} instances easily, From 29ffb0132370459ae4ba3f4ac1614308c94e2876 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 27 Jan 2025 17:53:07 -0800 Subject: [PATCH 14/25] Added a null check for the indexshard checkpoint Signed-off-by: Vinay Krishna Pudyodu --- .../SegmentReplicationTargetService.java | 2 +- .../indices/replication/SegmentReplicator.java | 15 ++++++++------- 2 files changed, 9 insertions(+), 8 deletions(-) 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 0a74954110815..59be56f2519c8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -293,7 +293,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe logger.trace(() -> "Ignoring checkpoint, Shard is closed"); return; } - // updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); + updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); replicator.updateReplicationCheckpointStats(receivedCheckpoint, replicaShard); // Checks if replica shard is in the correct STARTED state to process checkpoints (avoids parallel replication events taking place) // This check ensures we do not try to process a received checkpoint while the shard is still recovering, yet we stored the latest 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 2586d58811d07..c3ed1a157ff79 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -137,13 +137,14 @@ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestC long segmentInfosVersion = latestCheckPoint.getSegmentInfosVersion(); if (!existingCheckpointStats.containsKey(segmentInfosVersion)) { ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); - long bytesBehind = calculateBytesBehind(latestCheckPoint, replicationCheckpoint); - - if (bytesBehind > 0) { - existingCheckpointStats.put( - segmentInfosVersion, - new ReplicationCheckpointStats(bytesBehind, latestCheckPoint.getCreatedTimeStamp()) - ); + if(replicationCheckpoint != null) { + long bytesBehind = calculateBytesBehind(latestCheckPoint, replicationCheckpoint); + if (bytesBehind > 0) { + existingCheckpointStats.put( + segmentInfosVersion, + new ReplicationCheckpointStats(bytesBehind, latestCheckPoint.getCreatedTimeStamp()) + ); + } } } } From 4fe2f873b9c0f9923ed327ffe1ba8cd6c19afdf3 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 28 Jan 2025 13:07:37 -0800 Subject: [PATCH 15/25] fix style checks Signed-off-by: Vinay Krishna Pudyodu --- .../org/opensearch/indices/replication/SegmentReplicator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c3ed1a157ff79..498f707cbc1c1 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -137,7 +137,7 @@ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestC long segmentInfosVersion = latestCheckPoint.getSegmentInfosVersion(); if (!existingCheckpointStats.containsKey(segmentInfosVersion)) { ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); - if(replicationCheckpoint != null) { + if (replicationCheckpoint != null) { long bytesBehind = calculateBytesBehind(latestCheckPoint, replicationCheckpoint); if (bytesBehind > 0) { existingCheckpointStats.put( From c83803395efe2586db4ee5cb2f62d2e99403084a Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 4 Feb 2025 13:53:34 -0800 Subject: [PATCH 16/25] Updated version and added bwc for RemoteSegmentMetadata Signed-off-by: Vinay Krishna Pudyodu --- .../io/IndexIOStreamHandlerFactory.java | 25 +++++++++++ .../io/VersionedCodecStreamWrapper.java | 24 ++++++---- .../store/RemoteSegmentStoreDirectory.java | 5 ++- .../metadata/RemoteSegmentMetadata.java | 35 ++++++++++++--- .../RemoteSegmentMetadataHandler.java | 9 +++- .../RemoteSegmentMetadataHandlerFactory.java | 37 ++++++++++++++++ .../transfer/TranslogTransferManager.java | 3 +- ...ranslogTransferMetadataHandlerFactory.java | 36 +++++++++++++++ .../io/VersionedCodecStreamWrapperTests.java | 11 ++++- .../RemoteSegmentStoreDirectoryTests.java | 9 ++-- ...oteSegmentMetadataHandlerFactoryTests.java | 44 +++++++++++++++++++ .../RemoteSegmentMetadataHandlerTests.java | 2 +- ...ogTransferMetadataHandlerFactoryTests.java | 42 ++++++++++++++++++ 13 files changed, 258 insertions(+), 24 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/io/IndexIOStreamHandlerFactory.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java create mode 100644 server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactoryTests.java create mode 100644 server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactoryTests.java diff --git a/server/src/main/java/org/opensearch/common/io/IndexIOStreamHandlerFactory.java b/server/src/main/java/org/opensearch/common/io/IndexIOStreamHandlerFactory.java new file mode 100644 index 0000000000000..a4ad161d0ced3 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/io/IndexIOStreamHandlerFactory.java @@ -0,0 +1,25 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.io; + +/** + * Interface for factory to provide handler implementation for type {@link T} + * @param The type of content to be read/written to stream + * + * @opensearch.internal + */ +public interface IndexIOStreamHandlerFactory { + + /** + * Implements logic to provide handler based on the stream versions + * @param version stream version + * @return Handler for reading/writing content streams to/from - {@link T} + */ + IndexIOStreamHandler getHandler(int version); +} diff --git a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java index 8089d354a2480..586008d39174c 100644 --- a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java +++ b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java @@ -28,18 +28,24 @@ public class VersionedCodecStreamWrapper { private static final Logger logger = LogManager.getLogger(VersionedCodecStreamWrapper.class); - // TODO This can be updated to hold a streamReadWriteHandlerFactory and get relevant handler based on the stream versions - private final IndexIOStreamHandler indexIOStreamHandler; + private final IndexIOStreamHandlerFactory indexIOStreamHandlerFactory; + private final int minVersion; private final int currentVersion; private final String codec; /** - * @param indexIOStreamHandler handler to read/write stream from T + * @param indexIOStreamHandlerFactory factory for providing handler to read/write stream from T * @param currentVersion latest supported version of the stream * @param codec: stream codec */ - public VersionedCodecStreamWrapper(IndexIOStreamHandler indexIOStreamHandler, int currentVersion, String codec) { - this.indexIOStreamHandler = indexIOStreamHandler; + public VersionedCodecStreamWrapper( + IndexIOStreamHandlerFactory indexIOStreamHandlerFactory, + int minVersion, + int currentVersion, + String codec + ) { + this.indexIOStreamHandlerFactory = indexIOStreamHandlerFactory; + this.minVersion = minVersion; this.currentVersion = currentVersion; this.codec = codec; } @@ -56,6 +62,7 @@ public T readStream(IndexInput indexInput) throws IOException { try { CodecUtil.checksumEntireFile(indexInput); int readStreamVersion = checkHeader(indexInput); + logger.info("Read stream version :" + readStreamVersion); return getHandlerForVersion(readStreamVersion).readContent(indexInput); } catch (CorruptIndexException cie) { logger.error( @@ -87,7 +94,7 @@ public void writeStream(IndexOutput indexOutput, T content) throws IOException { */ private int checkHeader(IndexInput indexInput) throws IOException { // TODO Once versioning strategy is decided we'll add support for min/max supported versions - return CodecUtil.checkHeader(indexInput, this.codec, this.currentVersion, this.currentVersion); + return CodecUtil.checkHeader(indexInput, this.codec, minVersion, this.currentVersion); } /** @@ -120,8 +127,7 @@ private void writeFooter(IndexOutput indexOutput) throws IOException { * @param version stream content version */ private IndexIOStreamHandler getHandlerForVersion(int version) { - // TODO implement factory and pick relevant handler based on version. - // It should also take into account min and max supported versions - return this.indexIOStreamHandler; + // TODO factory should also take into account min and max supported versions + return this.indexIOStreamHandlerFactory.getHandler(version); } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 941cf047347f7..b12a60136874a 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -38,7 +38,7 @@ import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; -import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandlerFactory; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.node.remotestore.RemoteStorePinnedTimestampService; import org.opensearch.threadpool.ThreadPool; @@ -104,7 +104,8 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement private Map segmentsUploadedToRemoteStore; private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( - new RemoteSegmentMetadataHandler(), + new RemoteSegmentMetadataHandlerFactory(), + RemoteSegmentMetadata.VERSION_ONE, RemoteSegmentMetadata.CURRENT_VERSION, RemoteSegmentMetadata.METADATA_CODEC ); diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index c04ede4b443a1..0a7078cbcf0cc 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -8,6 +8,8 @@ package org.opensearch.index.store.remote.metadata; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Version; @@ -30,10 +32,15 @@ */ @PublicApi(since = "2.6.0") public class RemoteSegmentMetadata { + + public static final int VERSION_ONE = 1; + + public static final int VERSION_TWO = 2; + /** * Latest supported version of metadata */ - public static final int CURRENT_VERSION = 1; + public static final int CURRENT_VERSION = VERSION_TWO; /** * Metadata codec */ @@ -48,6 +55,8 @@ public class RemoteSegmentMetadata { private final ReplicationCheckpoint replicationCheckpoint; + private static final Logger logger = LogManager.getLogger(RemoteSegmentMetadata.class); + public RemoteSegmentMetadata( Map metadata, byte[] segmentInfosBytes, @@ -106,6 +115,11 @@ public static Map f ); } + /** + * Write always writes with latest version of the RemoteSegmentMetadata + * @param out + * @throws IOException + */ public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); writeCheckpointToIndexOutput(replicationCheckpoint, out); @@ -113,18 +127,27 @@ public void write(IndexOutput out) throws IOException { out.writeBytes(segmentInfosBytes, segmentInfosBytes.length); } - public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { + /** + * Read can happen in the upgraded version of replica which needs to support all version of RemoteSegmentMetadata + * @param indexInput + * @param version + * @return + * @throws IOException + */ + public static RemoteSegmentMetadata read(IndexInput indexInput, int version) throws IOException { Map metadata = indexInput.readMapOfStrings(); final Map uploadedSegmentMetadataMap = RemoteSegmentMetadata .fromMapOfStrings(metadata); - ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, uploadedSegmentMetadataMap); + ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, uploadedSegmentMetadataMap, version); int byteArraySize = (int) indexInput.readLong(); + logger.info("ByteArray Size : " + byteArraySize); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); return new RemoteSegmentMetadata(uploadedSegmentMetadataMap, segmentInfosBytes, replicationCheckpoint); } public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicationCheckpoint, IndexOutput out) throws IOException { + logger.info("----> writeCheckpointToIndexOutput"); ShardId shardId = replicationCheckpoint.getShardId(); // Write ShardId out.writeString(shardId.getIndex().getName()); @@ -141,8 +164,10 @@ public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicatio private static ReplicationCheckpoint readCheckpointFromIndexInput( IndexInput in, - Map uploadedSegmentMetadataMap + Map uploadedSegmentMetadataMap, + int version ) throws IOException { + logger.info("----> readCheckpointFromIndexInput"); return new ReplicationCheckpoint( new ShardId(new Index(in.readString(), in.readString()), in.readVInt()), in.readLong(), @@ -151,7 +176,7 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput( in.readLong(), in.readString(), toStoreFileMetadata(uploadedSegmentMetadataMap), - in.readLong() + version >= CURRENT_VERSION ? in.readLong() : 0 ); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandler.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandler.java index 3077d8c76ddae..9fa76b38d2b07 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandler.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandler.java @@ -20,6 +20,13 @@ * @opensearch.internal */ public class RemoteSegmentMetadataHandler implements IndexIOStreamHandler { + + private final int version; + + public RemoteSegmentMetadataHandler(int version) { + this.version = version; + } + /** * Reads metadata content from metadata file input stream and parsed into {@link RemoteSegmentMetadata} * @param indexInput metadata file input stream with {@link IndexInput#getFilePointer()} pointing to metadata content @@ -27,7 +34,7 @@ public class RemoteSegmentMetadataHandler implements IndexIOStreamHandler { + + private final ConcurrentHashMap> handlers = new ConcurrentHashMap<>(); + + @Override + public IndexIOStreamHandler getHandler(int version) { + return handlers.computeIfAbsent(version, this::createHandler); + } + + private IndexIOStreamHandler createHandler(int version) { + return switch (version) { + case RemoteSegmentMetadata.VERSION_ONE -> new RemoteSegmentMetadataHandler(RemoteSegmentMetadata.VERSION_ONE); + case RemoteSegmentMetadata.VERSION_TWO -> new RemoteSegmentMetadataHandler(RemoteSegmentMetadata.VERSION_TWO); + default -> throw new IllegalArgumentException("Unsupported RemoteSegmentMetadata version: " + version); + }; + } +} diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 1e621d6cb7688..d410f473c71f1 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -75,7 +75,8 @@ public class TranslogTransferManager { private final Logger logger; private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( - new TranslogTransferMetadataHandler(), + new TranslogTransferMetadataHandlerFactory(), + TranslogTransferMetadata.CURRENT_VERSION, TranslogTransferMetadata.CURRENT_VERSION, TranslogTransferMetadata.METADATA_CODEC ); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java new file mode 100644 index 0000000000000..6df16d4f263c7 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog.transfer; + +import org.opensearch.common.io.IndexIOStreamHandler; +import org.opensearch.common.io.IndexIOStreamHandlerFactory; + +import java.util.concurrent.ConcurrentHashMap; + +/** + * {@link TranslogTransferMetadataHandlerFactory} + * + * @opensearch.internal + */ +public class TranslogTransferMetadataHandlerFactory implements IndexIOStreamHandlerFactory { + + private final ConcurrentHashMap> handlers = new ConcurrentHashMap<>(); + + @Override + public IndexIOStreamHandler getHandler(int version) { + return handlers.computeIfAbsent(version, this::createHandler); + } + + private IndexIOStreamHandler createHandler(int version) { + return switch (version) { + case TranslogTransferMetadata.CURRENT_VERSION -> new TranslogTransferMetadataHandler(); + default -> throw new IllegalArgumentException("Unsupported TranslogTransferMetadata version: " + version); + }; + } +} diff --git a/server/src/test/java/org/opensearch/common/io/VersionedCodecStreamWrapperTests.java b/server/src/test/java/org/opensearch/common/io/VersionedCodecStreamWrapperTests.java index 938337fc5146e..a88df528bcb86 100644 --- a/server/src/test/java/org/opensearch/common/io/VersionedCodecStreamWrapperTests.java +++ b/server/src/test/java/org/opensearch/common/io/VersionedCodecStreamWrapperTests.java @@ -38,16 +38,19 @@ public class VersionedCodecStreamWrapperTests extends OpenSearchTestCase { private static final int VERSION = 1; IndexIOStreamHandler ioStreamHandler; + IndexIOStreamHandlerFactory ioStreamHandlerFactory; VersionedCodecStreamWrapper versionedCodecStreamWrapper; @Before public void setup() throws IOException { + ioStreamHandlerFactory = mock(IndexIOStreamHandlerFactory.class); ioStreamHandler = mock(IndexIOStreamHandler.class); - versionedCodecStreamWrapper = new VersionedCodecStreamWrapper(ioStreamHandler, VERSION, CODEC); + versionedCodecStreamWrapper = new VersionedCodecStreamWrapper(ioStreamHandlerFactory, VERSION, VERSION, CODEC); } public void testReadStream() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); DummyObject readData = versionedCodecStreamWrapper.readStream(createHeaderFooterBytes(CODEC, VERSION, true, true)); assertEquals(readData, expectedObject); @@ -55,6 +58,7 @@ public void testReadStream() throws IOException { public void testReadWithOldVersionThrowsException() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); assertThrows( IndexFormatTooOldException.class, @@ -64,6 +68,7 @@ public void testReadWithOldVersionThrowsException() throws IOException { public void testReadWithNewVersionThrowsException() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); assertThrows( IndexFormatTooNewException.class, @@ -73,6 +78,7 @@ public void testReadWithNewVersionThrowsException() throws IOException { public void testReadWithUnexpectedCodecThrowsException() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); assertThrows( CorruptIndexException.class, @@ -82,6 +88,7 @@ public void testReadWithUnexpectedCodecThrowsException() throws IOException { public void testReadWithNoHeaderThrowsException() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); assertThrows( CorruptIndexException.class, @@ -91,6 +98,7 @@ public void testReadWithNoHeaderThrowsException() throws IOException { public void testReadWithNoFooterThrowsException() throws IOException { DummyObject expectedObject = new DummyObject("test read"); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); when(ioStreamHandler.readContent(any())).thenReturn(expectedObject); assertThrows( CorruptIndexException.class, @@ -102,6 +110,7 @@ public void testWriteStream() throws IOException { DummyObject expectedObject = new DummyObject("test read"); BytesStreamOutput output = new BytesStreamOutput(); OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); + when(ioStreamHandlerFactory.getHandler(VERSION)).thenReturn(ioStreamHandler); doAnswer(invocation -> { IndexOutput io = invocation.getArgument(0); io.writeString("test write"); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index df3df81361a12..d673eb49be581 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -37,7 +37,7 @@ import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; -import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandlerFactory; import org.opensearch.test.MockLogAppender; import org.opensearch.test.junit.annotations.TestLogging; import org.opensearch.threadpool.ThreadPool; @@ -696,7 +696,8 @@ public void testUploadMetadataNonEmpty() throws IOException { eq(IOContext.DEFAULT) ); VersionedCodecStreamWrapper streamWrapper = new VersionedCodecStreamWrapper<>( - new RemoteSegmentMetadataHandler(), + new RemoteSegmentMetadataHandlerFactory(), + RemoteSegmentMetadata.CURRENT_VERSION, RemoteSegmentMetadata.CURRENT_VERSION, RemoteSegmentMetadata.METADATA_CODEC ); @@ -840,7 +841,7 @@ public void testHeaderMaxVersionCorruptIndexException() throws IOException { BytesStreamOutput output = new BytesStreamOutput(); OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("segment metadata", "metadata output stream", output, 4096); - CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, 2); + CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, 3); indexOutput.writeMapOfStrings(metadata); CodecUtil.writeFooter(indexOutput); indexOutput.close(); @@ -1115,7 +1116,7 @@ public void testSegmentMetadataCurrentVersion() { If author doesn't want to support old metadata files. Then this can be ignored. After taking appropriate action, fix this test by setting the correct version here */ - assertEquals(RemoteSegmentMetadata.CURRENT_VERSION, 1); + assertEquals(RemoteSegmentMetadata.CURRENT_VERSION, 2); } private void indexDocs(int startDocId, int numberOfDocs) throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactoryTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactoryTests.java new file mode 100644 index 0000000000000..6911b84c58e4d --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactoryTests.java @@ -0,0 +1,44 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.store.remote.metadata; + +import org.opensearch.common.io.IndexIOStreamHandler; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +/** + * Unit tests for {@link org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandlerFactoryTests}. + */ +public class RemoteSegmentMetadataHandlerFactoryTests extends OpenSearchTestCase { + + private RemoteSegmentMetadataHandlerFactory segmentMetadataHandlerFactory; + + @Before + public void setup() { + segmentMetadataHandlerFactory = new RemoteSegmentMetadataHandlerFactory(); + } + + public void testGetHandlerReturnsBasedOnVersion() { + IndexIOStreamHandler versionOneHandler = segmentMetadataHandlerFactory.getHandler(1); + assertTrue(versionOneHandler instanceof RemoteSegmentMetadataHandler); + IndexIOStreamHandler versionTwoHandler = segmentMetadataHandlerFactory.getHandler(2); + assertTrue(versionTwoHandler instanceof RemoteSegmentMetadataHandler); + } + + public void testGetHandlerWhenCalledMultipleTimesReturnsCachedHandler() { + IndexIOStreamHandler versionTwoHandlerOne = segmentMetadataHandlerFactory.getHandler(2); + IndexIOStreamHandler versionTwoHandlerTwo = segmentMetadataHandlerFactory.getHandler(2); + assertEquals(versionTwoHandlerOne, versionTwoHandlerTwo); + } + + public void testGetHandlerWhenHandlerNotProvidedThrowsException() { + Throwable throwable = assertThrows(IllegalArgumentException.class, () -> { segmentMetadataHandlerFactory.getHandler(3); }); + assertEquals("Unsupported RemoteSegmentMetadata version: 3", throwable.getMessage()); + } +} diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index f5d54dc790e76..0a668bba28c74 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -44,7 +44,7 @@ public class RemoteSegmentMetadataHandlerTests extends IndexShardTestCase { @Before public void setup() throws IOException { - remoteSegmentMetadataHandler = new RemoteSegmentMetadataHandler(); + remoteSegmentMetadataHandler = new RemoteSegmentMetadataHandler(2); Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT) diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactoryTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactoryTests.java new file mode 100644 index 0000000000000..767037160980e --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactoryTests.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.translog.transfer; + +import org.opensearch.common.io.IndexIOStreamHandler; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +/** + * Unit tests for {@link org.opensearch.index.translog.transfer.TranslogTransferMetadataHandlerFactoryTests}. + */ +public class TranslogTransferMetadataHandlerFactoryTests extends OpenSearchTestCase { + + private TranslogTransferMetadataHandlerFactory translogTransferMetadataHandlerFactory; + + @Before + public void setup() { + translogTransferMetadataHandlerFactory = new TranslogTransferMetadataHandlerFactory(); + } + + public void testGetHandlerReturnsBasedOnVersion() { + IndexIOStreamHandler versionOneHandler = translogTransferMetadataHandlerFactory.getHandler(1); + assertTrue(versionOneHandler instanceof TranslogTransferMetadataHandler); + } + + public void testGetHandlerWhenCalledMultipleTimesReturnsCachedHandler() { + IndexIOStreamHandler versionTwoHandlerOne = translogTransferMetadataHandlerFactory.getHandler(1); + IndexIOStreamHandler versionTwoHandlerTwo = translogTransferMetadataHandlerFactory.getHandler(1); + assertEquals(versionTwoHandlerOne, versionTwoHandlerTwo); + } + + public void testGetHandlerWhenHandlerNotProvidedThrowsException() { + Throwable throwable = assertThrows(IllegalArgumentException.class, () -> { translogTransferMetadataHandlerFactory.getHandler(2); }); + assertEquals("Unsupported TranslogTransferMetadata version: 2", throwable.getMessage()); + } +} From 1fdd5d248d307ec3b13ba25296b7459730ab6dcb Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 4 Feb 2025 14:28:33 -0800 Subject: [PATCH 17/25] Upated the javadoc comments Signed-off-by: Vinay Krishna Pudyodu --- .../io/VersionedCodecStreamWrapper.java | 1 - .../metadata/RemoteSegmentMetadata.java | 23 +++++++------------ .../RemoteSegmentMetadataHandlerFactory.java | 3 ++- ...ranslogTransferMetadataHandlerFactory.java | 3 ++- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java index 586008d39174c..72969279c2577 100644 --- a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java +++ b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java @@ -62,7 +62,6 @@ public T readStream(IndexInput indexInput) throws IOException { try { CodecUtil.checksumEntireFile(indexInput); int readStreamVersion = checkHeader(indexInput); - logger.info("Read stream version :" + readStreamVersion); return getHandlerForVersion(readStreamVersion).readContent(indexInput); } catch (CorruptIndexException cie) { logger.error( diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 0a7078cbcf0cc..d230c7efb6766 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -8,8 +8,6 @@ package org.opensearch.index.store.remote.metadata; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.Version; @@ -55,8 +53,6 @@ public class RemoteSegmentMetadata { private final ReplicationCheckpoint replicationCheckpoint; - private static final Logger logger = LogManager.getLogger(RemoteSegmentMetadata.class); - public RemoteSegmentMetadata( Map metadata, byte[] segmentInfosBytes, @@ -116,9 +112,9 @@ public static Map f } /** - * Write always writes with latest version of the RemoteSegmentMetadata - * @param out - * @throws IOException + * Write always writes with the latest version of the RemoteSegmentMetadata + * @param out file output stream which will store stream content + * @throws IOException in case there is a problem writing the file */ public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); @@ -128,11 +124,11 @@ public void write(IndexOutput out) throws IOException { } /** - * Read can happen in the upgraded version of replica which needs to support all version of RemoteSegmentMetadata - * @param indexInput - * @param version - * @return - * @throws IOException + * Read can happen in the upgraded version of replica which needs to support all versions of RemoteSegmentMetadata + * @param indexInput file input stream + * @param version version of the RemoteSegmentMetadata + * @return {@code RemoteSegmentMetadata} + * @throws IOException in case there is a problem reading from the file input stream */ public static RemoteSegmentMetadata read(IndexInput indexInput, int version) throws IOException { Map metadata = indexInput.readMapOfStrings(); @@ -140,14 +136,12 @@ public static RemoteSegmentMetadata read(IndexInput indexInput, int version) thr .fromMapOfStrings(metadata); ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, uploadedSegmentMetadataMap, version); int byteArraySize = (int) indexInput.readLong(); - logger.info("ByteArray Size : " + byteArraySize); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); return new RemoteSegmentMetadata(uploadedSegmentMetadataMap, segmentInfosBytes, replicationCheckpoint); } public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicationCheckpoint, IndexOutput out) throws IOException { - logger.info("----> writeCheckpointToIndexOutput"); ShardId shardId = replicationCheckpoint.getShardId(); // Write ShardId out.writeString(shardId.getIndex().getName()); @@ -167,7 +161,6 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput( Map uploadedSegmentMetadataMap, int version ) throws IOException { - logger.info("----> readCheckpointFromIndexInput"); return new ReplicationCheckpoint( new ShardId(new Index(in.readString(), in.readString()), in.readVInt()), in.readLong(), diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java index 9fe779edac912..82c5ffae262fb 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java @@ -14,7 +14,8 @@ import java.util.concurrent.ConcurrentHashMap; /** - * {@link RemoteSegmentMetadataHandlerFactory} + * {@link RemoteSegmentMetadataHandlerFactory} is a factory class to create {@link RemoteSegmentMetadataHandler} + * instances based on the {@link RemoteSegmentMetadata} version * * @opensearch.internal */ diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java index 6df16d4f263c7..8f8e3e816d665 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerFactory.java @@ -14,7 +14,8 @@ import java.util.concurrent.ConcurrentHashMap; /** - * {@link TranslogTransferMetadataHandlerFactory} + * {@link TranslogTransferMetadataHandlerFactory} is a factory class to create {@link TranslogTransferMetadataHandler} + * instances based on the {@link TranslogTransferMetadata} version * * @opensearch.internal */ From 85dd3427a7d2fd048d33c7c067653f88b6fce6ce Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Wed, 19 Feb 2025 15:22:40 -0800 Subject: [PATCH 18/25] Address comments PR Signed-off-by: Vinay Krishna Pudyodu --- .../io/VersionedCodecStreamWrapper.java | 2 +- .../opensearch/index/shard/IndexShard.java | 3 +- .../metadata/RemoteSegmentMetadata.java | 2 +- .../RemoteSegmentMetadataHandlerFactory.java | 14 ++- .../replication/SegmentReplicator.java | 89 ++++++++++++++----- .../checkpoint/ReplicationCheckpoint.java | 19 ++++ .../replication/SegmentReplicatorTests.java | 59 ++++++++++++ 7 files changed, 157 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java index 72969279c2577..b62ae1f1d3956 100644 --- a/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java +++ b/server/src/main/java/org/opensearch/common/io/VersionedCodecStreamWrapper.java @@ -35,6 +35,7 @@ public class VersionedCodecStreamWrapper { /** * @param indexIOStreamHandlerFactory factory for providing handler to read/write stream from T + * @param minVersion earliest supported version of the stream * @param currentVersion latest supported version of the stream * @param codec: stream codec */ @@ -126,7 +127,6 @@ private void writeFooter(IndexOutput indexOutput) throws IOException { * @param version stream content version */ private IndexIOStreamHandler getHandlerForVersion(int version) { - // TODO factory should also take into account min and max supported versions return this.indexIOStreamHandlerFactory.getHandler(version); } } 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 715ba6c31a680..f50d8acaa9010 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1787,8 +1787,7 @@ ReplicationCheckpoint computeReplicationCheckpoint(SegmentInfos segmentInfos) th segmentInfos.getVersion(), metadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), getEngine().config().getCodec().getName(), - metadataMap, - System.nanoTime() + metadataMap ); logger.trace("Recomputed ReplicationCheckpoint for shard {}", checkpoint); return checkpoint; diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index d230c7efb6766..463e08918b3f7 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -169,7 +169,7 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput( in.readLong(), in.readString(), toStoreFileMetadata(uploadedSegmentMetadataMap), - version >= CURRENT_VERSION ? in.readLong() : 0 + version >= VERSION_TWO ? in.readLong() : 0 ); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java index 82c5ffae262fb..116608f669784 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerFactory.java @@ -11,7 +11,7 @@ import org.opensearch.common.io.IndexIOStreamHandler; import org.opensearch.common.io.IndexIOStreamHandlerFactory; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; /** * {@link RemoteSegmentMetadataHandlerFactory} is a factory class to create {@link RemoteSegmentMetadataHandler} @@ -20,12 +20,18 @@ * @opensearch.internal */ public class RemoteSegmentMetadataHandlerFactory implements IndexIOStreamHandlerFactory { - - private final ConcurrentHashMap> handlers = new ConcurrentHashMap<>(); + private final AtomicReference> handlerRef = new AtomicReference<>(); @Override public IndexIOStreamHandler getHandler(int version) { - return handlers.computeIfAbsent(version, this::createHandler); + IndexIOStreamHandler current = handlerRef.get(); + if (current != null) { + return current; + } + + IndexIOStreamHandler newHandler = createHandler(version); + handlerRef.compareAndSet(null, newHandler); + return handlerRef.get(); } private IndexIOStreamHandler createHandler(int version) { 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 498f707cbc1c1..fed18b524879f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -51,7 +51,7 @@ public class SegmentReplicator { private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap latestReceivedCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap latestPrimaryCheckpoint = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -110,6 +110,14 @@ SegmentReplicationTarget startReplication( return target; } + /** + * Retrieves segment replication statistics for a specific shard. + * Its computed based on the last and first entry in the replicationCheckpointStats map. + * The Last entry gives the Bytes behind, and the difference in the first and last entry provides the lag. + * + * @param shardId The shardId to get statistics for + * @return ReplicationStats containing bytes behind and replication lag information + */ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { @@ -127,30 +135,46 @@ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); } - public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestCheckPoint, final IndexShard indexShard) { - latestReceivedCheckpoint.put(indexShard.shardId(), latestCheckPoint); - ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.computeIfAbsent( - indexShard.shardId(), - k -> new ConcurrentSkipListMap<>() - ); - - long segmentInfosVersion = latestCheckPoint.getSegmentInfosVersion(); - if (!existingCheckpointStats.containsKey(segmentInfosVersion)) { - ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); - if (replicationCheckpoint != null) { - long bytesBehind = calculateBytesBehind(latestCheckPoint, replicationCheckpoint); - if (bytesBehind > 0) { - existingCheckpointStats.put( - segmentInfosVersion, - new ReplicationCheckpointStats(bytesBehind, latestCheckPoint.getCreatedTimeStamp()) - ); - } - } + /** + * Updates the latest checkpoint of the primary for the replica shard and then + * calculates checkpoint statistics for the replica shard with the latest checkpoint information. + * This method maintains statistics about how far behind replica shards are from the primary. + * It calculates the bytes behind by comparing the latest-received and current checkpoint in the indexShard, + * and it maintains the bytes behind and timestamp for each segmentInfosVersion of latestCheckPoint. + *
+     * Example:
+     * {
+     *     [replica][0] : {
+     *                       7 : {bytesBehind=0, timestamp=1700220000000}
+     *                       8 : {bytesBehind=100, timestamp=1700330000000}
+     *                       9 : {bytesBehind=150, timestamp=1700440000000}
+     *                    }
+     * }
+     * 
+ * @param latestReceivedCheckPoint The most recent checkpoint from the primary + * @param indexShard The index shard where its updated + */ + synchronized public void updateReplicationCheckpointStats( + final ReplicationCheckpoint latestReceivedCheckPoint, + final IndexShard indexShard + ) { + ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { + this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); } + // Even if we receive any out of order checkpoint, we will still calculate the stats at that point + calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); } - protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { - ReplicationCheckpoint latestCheckpoint = this.latestReceivedCheckpoint.get(indexShard.shardId()); + /** + * Removes checkpoint statistics for all checkpoints up to and including the last successful sync + * and recalculates the bytes behind value for the last replicationCheckpointStats entry. + * This helps maintain only relevant checkpoint information and clean up old data. + * + * @param indexShard The index shard to prune checkpoints for + */ + synchronized protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { + ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); if (latestCheckpoint != null) { ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); @@ -168,6 +192,25 @@ protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { } } + private void calculateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { + ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.computeIfAbsent( + indexShard.shardId(), + k -> new ConcurrentSkipListMap<>() + ); + + long segmentInfosVersion = latestReceivedCheckPoint.getSegmentInfosVersion(); + ReplicationCheckpoint indexShardReplicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); + if (indexShardReplicationCheckpoint != null) { + long bytesBehind = calculateBytesBehind(latestReceivedCheckPoint, indexShardReplicationCheckpoint); + if (bytesBehind > 0) { + existingCheckpointStats.computeIfAbsent( + segmentInfosVersion, + k -> new ReplicationCheckpointStats(bytesBehind, latestReceivedCheckPoint.getCreatedTimeStamp()) + ); + } + } + } + private long calculateBytesBehind(final ReplicationCheckpoint latestCheckPoint, final ReplicationCheckpoint replicationCheckpoint) { Store.RecoveryDiff diff = Store.segmentReplicationDiff(latestCheckPoint.getMetadataMap(), replicationCheckpoint.getMetadataMap()); @@ -313,7 +356,7 @@ int size() { void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); replicationCheckpointStats.remove(shardId); - latestReceivedCheckpoint.remove(shardId); + latestPrimaryCheckpoint.remove(shardId); } SegmentReplicationTarget get(ShardId shardId) { diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index d609e35373c8f..8380187a288ba 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -63,6 +63,25 @@ public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap(), System.nanoTime()); } + public ReplicationCheckpoint( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + String codec, + Map metadataMap + ) { + this.shardId = shardId; + this.primaryTerm = primaryTerm; + this.segmentsGen = segmentsGen; + this.segmentInfosVersion = segmentInfosVersion; + this.length = length; + this.codec = codec; + this.metadataMap = metadataMap; + this.createdTimeStamp = System.nanoTime(); + } + public ReplicationCheckpoint( ShardId shardId, long primaryTerm, diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index d02b45492d698..3e52f7c32fffd 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -286,6 +286,65 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefr assertTrue(replicationStatsSecond.maxReplicationLag > 0); } + public void testGetSegmentReplicationStats_WhenCheckPointReceivedOutOfOrder() { + ShardId shardId = new ShardId("index", "uuid", 0); + ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); + + StoreFileMetadata storeFileMetadata1 = new StoreFileMetadata("test-1", 500, "1", Version.LATEST, new BytesRef(500)); + StoreFileMetadata storeFileMetadata2 = new StoreFileMetadata("test-2", 500, "1", Version.LATEST, new BytesRef(500)); + Map stringStoreFileMetadataMapOne = new HashMap<>(); + stringStoreFileMetadataMapOne.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMapOne.put("test-2", storeFileMetadata2); + ReplicationCheckpoint secondReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 2, + 2, + 2, + 1000, + "", + stringStoreFileMetadataMapOne, + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) + ); + + IndexShard replicaShard = mock(IndexShard.class); + when(replicaShard.shardId()).thenReturn(shardId); + when(replicaShard.getLatestReplicationCheckpoint()).thenReturn(firstReplicationCheckpoint) + .thenReturn(firstReplicationCheckpoint) + .thenReturn(firstReplicationCheckpoint); + + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + segmentReplicator.updateReplicationCheckpointStats(firstReplicationCheckpoint, replicaShard); + + Map stringStoreFileMetadataMapTwo = new HashMap<>(); + StoreFileMetadata storeFileMetadata3 = new StoreFileMetadata("test-3", 200, "1", Version.LATEST, new BytesRef(200)); + stringStoreFileMetadataMapTwo.put("test-1", storeFileMetadata1); + stringStoreFileMetadataMapTwo.put("test-2", storeFileMetadata2); + stringStoreFileMetadataMapTwo.put("test-3", storeFileMetadata3); + ReplicationCheckpoint thirdReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 3, + 3, + 3, + 200, + "", + stringStoreFileMetadataMapTwo, + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) + ); + + segmentReplicator.updateReplicationCheckpointStats(thirdReplicationCheckpoint, replicaShard); + + ReplicationStats replicationStatsFirst = segmentReplicator.getSegmentReplicationStats(shardId); + assertEquals(1200, replicationStatsFirst.totalBytesBehind); + assertEquals(1200, replicationStatsFirst.maxBytesBehind); + assertTrue(replicationStatsFirst.maxReplicationLag > 0); + + segmentReplicator.updateReplicationCheckpointStats(secondReplicationCheckpoint, replicaShard); + ReplicationStats replicationStatsSecond = segmentReplicator.getSegmentReplicationStats(shardId); + assertEquals(1200, replicationStatsSecond.totalBytesBehind); + assertEquals(1200, replicationStatsSecond.maxBytesBehind); + assertTrue(replicationStatsSecond.maxReplicationLag > 0); + } + protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { try (final CopyState copyState = new CopyState(primary)) { listener.onResponse( From c993370734295f1ae3968f30b500b649576787f6 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Thu, 20 Feb 2025 13:19:22 -0800 Subject: [PATCH 19/25] Removed the latestReceivedCheckpoint map from SegmentReplicationTargetService Signed-off-by: Vinay Krishna Pudyodu --- .../SegmentReplicationTargetService.java | 22 +++--------- .../replication/SegmentReplicator.java | 7 ++-- .../SegmentReplicationTargetServiceTests.java | 12 ++----- .../replication/SegmentReplicatorTests.java | 36 ++++++++++++++++++- 4 files changed, 47 insertions(+), 30 deletions(-) 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 59be56f2519c8..124e7e7bdbff2 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -22,7 +22,6 @@ import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.CancellableThreads; -import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.transport.TransportResponse; @@ -49,7 +48,6 @@ import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.Map; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -58,7 +56,8 @@ /** * Service class that handles incoming checkpoints to initiate replication events on replicas. - * + * This is used for the peer to peer replication, not for the remote replication, remote replication uses SegmentReplicator + * Explore this * @opensearch.internal */ public class SegmentReplicationTargetService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { @@ -70,8 +69,6 @@ public class SegmentReplicationTargetService extends AbstractLifecycleComponent private final SegmentReplicationSourceFactory sourceFactory; - protected final Map latestReceivedCheckpoint = ConcurrentCollections.newConcurrentMap(); - private final IndicesService indicesService; private final ClusterService clusterService; private final TransportService transportService; @@ -216,7 +213,6 @@ public void clusterChanged(ClusterChangedEvent event) { public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { if (indexShard != null && indexShard.indexSettings().isSegRepEnabledOrRemoteNode()) { replicator.cancel(indexShard.shardId(), "Shard closing"); - latestReceivedCheckpoint.remove(shardId); } } @@ -241,7 +237,6 @@ public void shardRoutingChanged(IndexShard indexShard, @Nullable ShardRouting ol && oldRouting.primary() == false && newRouting.primary()) { replicator.cancel(indexShard.shardId(), "Shard has been promoted to primary"); - latestReceivedCheckpoint.remove(indexShard.shardId()); } } @@ -294,7 +289,6 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe return; } updateLatestReceivedCheckpoint(receivedCheckpoint, replicaShard); - replicator.updateReplicationCheckpointStats(receivedCheckpoint, replicaShard); // Checks if replica shard is in the correct STARTED state to process checkpoints (avoids parallel replication events taking place) // This check ensures we do not try to process a received checkpoint while the shard is still recovering, yet we stored the latest // checkpoint to be replayed once the shard is Active. @@ -469,7 +463,7 @@ private DiscoveryNode getPrimaryNode(ShardRouting primaryShard) { // visible to tests protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Thread thread) { - final ReplicationCheckpoint latestPublishedCheckpoint = latestReceivedCheckpoint.get(replicaShard.shardId()); + final ReplicationCheckpoint latestPublishedCheckpoint = replicator.getLatestPrimaryCheckpoint(replicaShard.shardId()); if (latestPublishedCheckpoint != null) { logger.trace( () -> new ParameterizedMessage( @@ -482,7 +476,7 @@ protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Threa // if we retry ensure the shard is not in the process of being closed. // it will be removed from indexService's collection before the shard is actually marked as closed. if (indicesService.getShardOrNull(replicaShard.shardId()) != null) { - onNewCheckpoint(latestReceivedCheckpoint.get(replicaShard.shardId()), replicaShard); + onNewCheckpoint(replicator.getLatestPrimaryCheckpoint(replicaShard.shardId()), replicaShard); } }; // Checks if we are using same thread and forks if necessary. @@ -498,13 +492,7 @@ protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Threa // visible to tests protected void updateLatestReceivedCheckpoint(ReplicationCheckpoint receivedCheckpoint, IndexShard replicaShard) { - if (latestReceivedCheckpoint.get(replicaShard.shardId()) != null) { - if (receivedCheckpoint.isAheadOf(latestReceivedCheckpoint.get(replicaShard.shardId()))) { - latestReceivedCheckpoint.replace(replicaShard.shardId(), receivedCheckpoint); - } - } else { - latestReceivedCheckpoint.put(replicaShard.shardId(), receivedCheckpoint); - } + replicator.updateReplicationCheckpointStats(receivedCheckpoint, replicaShard); } /** 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 fed18b524879f..b0069f037b3bf 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -161,9 +161,8 @@ synchronized public void updateReplicationCheckpointStats( ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); + calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); } - // Even if we receive any out of order checkpoint, we will still calculate the stats at that point - calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); } /** @@ -363,6 +362,10 @@ SegmentReplicationTarget get(ShardId shardId) { return onGoingReplications.getOngoingReplicationTarget(shardId); } + ReplicationCheckpoint getLatestPrimaryCheckpoint(ShardId shardId) { + return latestPrimaryCheckpoint.get(shardId); + } + ReplicationCollection.ReplicationRef get(long id) { return onGoingReplications.get(id); } 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 1faaa16ce5628..8a47b87b09f30 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -100,8 +100,6 @@ public class SegmentReplicationTargetServiceTests extends IndexShardTestCase { private SegmentReplicationState state; private ReplicationCheckpoint initialCheckpoint; - private ClusterState clusterState; - private static final long TRANSPORT_TIMEOUT = 30000;// 30sec @Override @@ -140,13 +138,14 @@ public void setUp() throws Exception { indicesService = mock(IndicesService.class); ClusterService clusterService = mock(ClusterService.class); - clusterState = mock(ClusterState.class); + ClusterState clusterState = mock(ClusterState.class); RoutingTable mockRoutingTable = mock(RoutingTable.class); when(clusterService.state()).thenReturn(clusterState); when(clusterState.routingTable()).thenReturn(mockRoutingTable); when(mockRoutingTable.shardRoutingTable(any())).thenReturn(primaryShard.getReplicationGroup().getRoutingTable()); when(clusterState.nodes()).thenReturn(DiscoveryNodes.builder().add(localNode).build()); + sut = prepareForReplication(primaryShard, replicaShard, transportService, indicesService, clusterService); initialCheckpoint = primaryShard.getLatestReplicationCheckpoint(); aheadCheckpoint = new ReplicationCheckpoint( @@ -596,13 +595,6 @@ public void testShardRoutingChanged_DoesNothingForDocRepIndex() throws IOExcepti closeShards(shard); } - public void testUpdateLatestReceivedCheckpoint() { - final SegmentReplicationTargetService spy = spy(sut); - sut.updateLatestReceivedCheckpoint(checkpoint, replicaShard); - sut.updateLatestReceivedCheckpoint(aheadCheckpoint, replicaShard); - assertEquals(sut.latestReceivedCheckpoint.get(replicaShard.shardId()), aheadCheckpoint); - } - public void testForceSegmentSyncHandler() throws Exception { ForceSyncRequest forceSyncRequest = new ForceSyncRequest(1L, 1L, replicaShard.shardId()); when(indicesService.getShardOrNull(forceSyncRequest.getShardId())).thenReturn(replicaShard); diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 3e52f7c32fffd..454923a630fc6 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -345,6 +345,41 @@ public void testGetSegmentReplicationStats_WhenCheckPointReceivedOutOfOrder() { assertTrue(replicationStatsSecond.maxReplicationLag > 0); } + public void testUpdateReplicationCheckpointStatsIgnoresWhenOutOfOrderCheckPointReceived() { + ShardId shardId = new ShardId("index", "uuid", 0); + IndexShard replicaShard = mock(IndexShard.class); + when(replicaShard.shardId()).thenReturn(shardId); + + SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + ReplicationCheckpoint replicationCheckpoint = new ReplicationCheckpoint( + shardId, + 2, + 2, + 2, + 1000, + "", + new HashMap<>(), + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) + ); + segmentReplicator.updateReplicationCheckpointStats(replicationCheckpoint, replicaShard); + + assertEquals(replicationCheckpoint, segmentReplicator.getLatestPrimaryCheckpoint(shardId)); + + ReplicationCheckpoint oldReplicationCheckpoint = new ReplicationCheckpoint( + shardId, + 1, + 1, + 1, + 500, + "", + new HashMap<>(), + System.nanoTime() - TimeUnit.MINUTES.toNanos(1) + ); + segmentReplicator.updateReplicationCheckpointStats(oldReplicationCheckpoint, replicaShard); + + assertEquals(replicationCheckpoint, segmentReplicator.getLatestPrimaryCheckpoint(shardId)); + } + protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { try (final CopyState copyState = new CopyState(primary)) { listener.onResponse( @@ -354,5 +389,4 @@ protected void resolveCheckpointListener(ActionListener throw new UncheckedIOException(e); } } - } From 6879f4c6aeaa986cea694880dc45f93336e7ef8b Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 24 Feb 2025 12:02:18 -0800 Subject: [PATCH 20/25] Added granular locks for the concurrency of stats methods Signed-off-by: Vinay Krishna Pudyodu --- .../replication/SegmentReplicator.java | 99 ++++++++++--------- 1 file changed, 54 insertions(+), 45 deletions(-) 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 b0069f037b3bf..eade6521935ee 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -52,6 +52,8 @@ public class SegmentReplicator { private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap latestPrimaryCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap shardLocks = ConcurrentCollections.newConcurrentMap(); + private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -119,20 +121,22 @@ SegmentReplicationTarget startReplication( * @return ReplicationStats containing bytes behind and replication lag information */ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); - if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { - return ReplicationStats.empty(); - } + synchronized (getLockForShard(shardId)) { + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); + if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { + return ReplicationStats.empty(); + } - Map.Entry lowestEntry = existingCheckpointStats.firstEntry(); - Map.Entry highestEntry = existingCheckpointStats.lastEntry(); + Map.Entry lowestEntry = existingCheckpointStats.firstEntry(); + Map.Entry highestEntry = existingCheckpointStats.lastEntry(); - long bytesBehind = highestEntry.getValue().getBytesBehind(); - long replicationLag = bytesBehind > 0L - ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) - : 0; + long bytesBehind = highestEntry.getValue().getBytesBehind(); + long replicationLag = bytesBehind > 0L + ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) + : 0; - return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); + return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); + } } /** @@ -154,14 +158,13 @@ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { * @param latestReceivedCheckPoint The most recent checkpoint from the primary * @param indexShard The index shard where its updated */ - synchronized public void updateReplicationCheckpointStats( - final ReplicationCheckpoint latestReceivedCheckPoint, - final IndexShard indexShard - ) { - ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); - if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { - this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); - calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); + public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { + synchronized (getLockForShard(indexShard.shardId())) { + ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { + this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); + calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); + } } } @@ -172,44 +175,49 @@ synchronized public void updateReplicationCheckpointStats( * * @param indexShard The index shard to prune checkpoints for */ - synchronized protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { - ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); - if (latestCheckpoint != null) { - ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); - long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( - indexShard.shardId() - ); - - if (existingCheckpointStats != null && !existingCheckpointStats.isEmpty()) { - existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); - Map.Entry lastEntry = existingCheckpointStats.lastEntry(); - if (lastEntry != null) { - lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); + protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { + synchronized (getLockForShard(indexShard.shardId())) { + ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + if (latestCheckpoint != null) { + ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); + long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( + indexShard.shardId() + ); + + if (existingCheckpointStats != null && !existingCheckpointStats.isEmpty()) { + existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); + Map.Entry lastEntry = existingCheckpointStats.lastEntry(); + if (lastEntry != null) { + lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); + } } } } } private void calculateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { - ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.computeIfAbsent( - indexShard.shardId(), - k -> new ConcurrentSkipListMap<>() - ); - - long segmentInfosVersion = latestReceivedCheckPoint.getSegmentInfosVersion(); ReplicationCheckpoint indexShardReplicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); if (indexShardReplicationCheckpoint != null) { - long bytesBehind = calculateBytesBehind(latestReceivedCheckPoint, indexShardReplicationCheckpoint); - if (bytesBehind > 0) { - existingCheckpointStats.computeIfAbsent( - segmentInfosVersion, - k -> new ReplicationCheckpointStats(bytesBehind, latestReceivedCheckPoint.getCreatedTimeStamp()) - ); + synchronized (getLockForShard(indexShard.shardId())) { + long segmentInfosVersion = latestReceivedCheckPoint.getSegmentInfosVersion(); + long bytesBehind = calculateBytesBehind(latestReceivedCheckPoint, indexShardReplicationCheckpoint); + if (bytesBehind > 0) { + ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats + .computeIfAbsent(indexShard.shardId(), k -> new ConcurrentSkipListMap<>()); + existingCheckpointStats.computeIfAbsent( + segmentInfosVersion, + k -> new ReplicationCheckpointStats(bytesBehind, latestReceivedCheckPoint.getCreatedTimeStamp()) + ); + } } } } + private Object getLockForShard(ShardId shardId) { + return shardLocks.computeIfAbsent(shardId, k -> new Object()); + } + private long calculateBytesBehind(final ReplicationCheckpoint latestCheckPoint, final ReplicationCheckpoint replicationCheckpoint) { Store.RecoveryDiff diff = Store.segmentReplicationDiff(latestCheckPoint.getMetadataMap(), replicationCheckpoint.getMetadataMap()); @@ -356,6 +364,7 @@ void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); replicationCheckpointStats.remove(shardId); latestPrimaryCheckpoint.remove(shardId); + shardLocks.remove(shardId); } SegmentReplicationTarget get(ShardId shardId) { From 6ee18080ca48560350b3e3e879af5718c60ed007 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Mon, 24 Feb 2025 12:35:54 -0800 Subject: [PATCH 21/25] Style check fixes Signed-off-by: Vinay Krishna Pudyodu --- .../org/opensearch/indices/replication/SegmentReplicator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 eade6521935ee..8ab5947c52290 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -122,7 +122,9 @@ SegmentReplicationTarget startReplication( */ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { synchronized (getLockForShard(shardId)) { - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( + shardId + ); if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { return ReplicationStats.empty(); } From e928e80ef7b875bd5c5758e666dc77195679685e Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 25 Feb 2025 10:16:40 -0800 Subject: [PATCH 22/25] Changes to maintain atomicity Signed-off-by: Vinay Krishna Pudyodu --- .../SegmentReplicationTargetService.java | 1 + .../replication/SegmentReplicator.java | 85 +++++++++---------- .../replication/SegmentReplicatorTests.java | 5 +- 3 files changed, 42 insertions(+), 49 deletions(-) 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 124e7e7bdbff2..d4c4fdf54f48e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -223,6 +223,7 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh @Override public void afterIndexShardStarted(IndexShard indexShard) { if (indexShard.indexSettings().isSegRepEnabledOrRemoteNode() && indexShard.routingEntry().primary() == false) { + replicator.initializeStats(indexShard.shardId()); processLatestReceivedCheckpoint(indexShard, Thread.currentThread()); } } 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 8ab5947c52290..436493aba359d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -52,7 +52,6 @@ public class SegmentReplicator { private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap latestPrimaryCheckpoint = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap shardLocks = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -121,24 +120,20 @@ SegmentReplicationTarget startReplication( * @return ReplicationStats containing bytes behind and replication lag information */ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { - synchronized (getLockForShard(shardId)) { - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( - shardId - ); - if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { - return ReplicationStats.empty(); - } + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get(shardId); + if (existingCheckpointStats == null || existingCheckpointStats.isEmpty()) { + return ReplicationStats.empty(); + } - Map.Entry lowestEntry = existingCheckpointStats.firstEntry(); - Map.Entry highestEntry = existingCheckpointStats.lastEntry(); + Map.Entry lowestEntry = existingCheckpointStats.firstEntry(); + Map.Entry highestEntry = existingCheckpointStats.lastEntry(); - long bytesBehind = highestEntry.getValue().getBytesBehind(); - long replicationLag = bytesBehind > 0L - ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) - : 0; + long bytesBehind = highestEntry.getValue().getBytesBehind(); + long replicationLag = bytesBehind > 0L + ? TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lowestEntry.getValue().getTimestamp()) + : 0; - return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); - } + return new ReplicationStats(bytesBehind, bytesBehind, replicationLag); } /** @@ -161,12 +156,10 @@ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { * @param indexShard The index shard where its updated */ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { - synchronized (getLockForShard(indexShard.shardId())) { - ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); - if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { - this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); - calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); - } + ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { + this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); + calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); } } @@ -178,21 +171,19 @@ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestR * @param indexShard The index shard to prune checkpoints for */ protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { - synchronized (getLockForShard(indexShard.shardId())) { - ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); - if (latestCheckpoint != null) { - ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); - long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); - final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( - indexShard.shardId() - ); + ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + if (latestCheckpoint != null) { + ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); + long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); + final ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( + indexShard.shardId() + ); - if (existingCheckpointStats != null && !existingCheckpointStats.isEmpty()) { - existingCheckpointStats.headMap(segmentInfoVersion, true).clear(); - Map.Entry lastEntry = existingCheckpointStats.lastEntry(); - if (lastEntry != null) { - lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); - } + if (existingCheckpointStats != null && !existingCheckpointStats.isEmpty()) { + existingCheckpointStats.keySet().removeIf(key -> key < segmentInfoVersion); + Map.Entry lastEntry = existingCheckpointStats.lastEntry(); + if (lastEntry != null) { + lastEntry.getValue().setBytesBehind(calculateBytesBehind(latestCheckpoint, indexReplicationCheckPoint)); } } } @@ -201,12 +192,13 @@ protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { private void calculateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { ReplicationCheckpoint indexShardReplicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); if (indexShardReplicationCheckpoint != null) { - synchronized (getLockForShard(indexShard.shardId())) { - long segmentInfosVersion = latestReceivedCheckPoint.getSegmentInfosVersion(); - long bytesBehind = calculateBytesBehind(latestReceivedCheckPoint, indexShardReplicationCheckpoint); - if (bytesBehind > 0) { - ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats - .computeIfAbsent(indexShard.shardId(), k -> new ConcurrentSkipListMap<>()); + long segmentInfosVersion = latestReceivedCheckPoint.getSegmentInfosVersion(); + long bytesBehind = calculateBytesBehind(latestReceivedCheckPoint, indexShardReplicationCheckpoint); + if (bytesBehind > 0) { + ConcurrentNavigableMap existingCheckpointStats = replicationCheckpointStats.get( + indexShard.shardId() + ); + if (existingCheckpointStats != null) { existingCheckpointStats.computeIfAbsent( segmentInfosVersion, k -> new ReplicationCheckpointStats(bytesBehind, latestReceivedCheckPoint.getCreatedTimeStamp()) @@ -216,16 +208,16 @@ private void calculateReplicationCheckpointStats(final ReplicationCheckpoint lat } } - private Object getLockForShard(ShardId shardId) { - return shardLocks.computeIfAbsent(shardId, k -> new Object()); - } - private long calculateBytesBehind(final ReplicationCheckpoint latestCheckPoint, final ReplicationCheckpoint replicationCheckpoint) { Store.RecoveryDiff diff = Store.segmentReplicationDiff(latestCheckPoint.getMetadataMap(), replicationCheckpoint.getMetadataMap()); return diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); } + public void initializeStats(ShardId shardId) { + replicationCheckpointStats.computeIfAbsent(shardId, k -> new ConcurrentSkipListMap<>()); + } + private static class ReplicationCheckpointStats { private long bytesBehind; private final long timestamp; @@ -366,7 +358,6 @@ void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); replicationCheckpointStats.remove(shardId); latestPrimaryCheckpoint.remove(shardId); - shardLocks.remove(shardId); } SegmentReplicationTarget get(ShardId shardId) { diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 454923a630fc6..e980341a7626b 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; +import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -47,8 +48,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import org.mockito.Mockito; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -252,6 +251,7 @@ public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefr .thenReturn(secondReplicationCheckpoint); SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + segmentReplicator.initializeStats(shardId); segmentReplicator.updateReplicationCheckpointStats(firstReplicationCheckpoint, replicaShard); segmentReplicator.updateReplicationCheckpointStats(secondReplicationCheckpoint, replicaShard); @@ -313,6 +313,7 @@ public void testGetSegmentReplicationStats_WhenCheckPointReceivedOutOfOrder() { .thenReturn(firstReplicationCheckpoint); SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); + segmentReplicator.initializeStats(shardId); segmentReplicator.updateReplicationCheckpointStats(firstReplicationCheckpoint, replicaShard); Map stringStoreFileMetadataMapTwo = new HashMap<>(); From 3d030d5549c827540e6251e881676965f86b64e8 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Tue, 25 Feb 2025 10:32:10 -0800 Subject: [PATCH 23/25] spotlessApply Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/indices/replication/SegmentReplicatorTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index e980341a7626b..19d684595a480 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,7 +11,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; -import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -48,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; +import org.mockito.Mockito; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; From fcc57bf4265ca957b3452186e20810a6b547fcd4 Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Wed, 26 Feb 2025 12:19:33 -0800 Subject: [PATCH 24/25] removed querying the remotestore when replication is in progress Signed-off-by: Vinay Krishna Pudyodu --- .../SegmentReplicationTargetService.java | 6 ++-- .../replication/SegmentReplicator.java | 36 +++++-------------- .../replication/SegmentReplicatorTests.java | 16 ++------- 3 files changed, 13 insertions(+), 45 deletions(-) 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 d4c4fdf54f48e..d57f35a5079fc 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -56,8 +56,6 @@ /** * Service class that handles incoming checkpoints to initiate replication events on replicas. - * This is used for the peer to peer replication, not for the remote replication, remote replication uses SegmentReplicator - * Explore this * @opensearch.internal */ public class SegmentReplicationTargetService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { @@ -464,7 +462,7 @@ private DiscoveryNode getPrimaryNode(ShardRouting primaryShard) { // visible to tests protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Thread thread) { - final ReplicationCheckpoint latestPublishedCheckpoint = replicator.getLatestPrimaryCheckpoint(replicaShard.shardId()); + final ReplicationCheckpoint latestPublishedCheckpoint = replicator.getPrimaryCheckpoint(replicaShard.shardId()); if (latestPublishedCheckpoint != null) { logger.trace( () -> new ParameterizedMessage( @@ -477,7 +475,7 @@ protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Threa // if we retry ensure the shard is not in the process of being closed. // it will be removed from indexService's collection before the shard is actually marked as closed. if (indicesService.getShardOrNull(replicaShard.shardId()) != null) { - onNewCheckpoint(replicator.getLatestPrimaryCheckpoint(replicaShard.shardId()), replicaShard); + onNewCheckpoint(replicator.getPrimaryCheckpoint(replicaShard.shardId()), replicaShard); } }; // Checks if we are using same thread and forks if necessary. 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 436493aba359d..b8a5774c21c1f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -51,7 +51,7 @@ public class SegmentReplicator { private final Map completedReplications = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap> replicationCheckpointStats = ConcurrentCollections.newConcurrentMap(); - private final ConcurrentMap latestPrimaryCheckpoint = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap primaryCheckpoint = ConcurrentCollections.newConcurrentMap(); private final ThreadPool threadPool; private final SetOnce sourceFactory; @@ -156,9 +156,9 @@ public ReplicationStats getSegmentReplicationStats(final ShardId shardId) { * @param indexShard The index shard where its updated */ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestReceivedCheckPoint, final IndexShard indexShard) { - ReplicationCheckpoint latestPrimaryCheckPoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); - if (latestPrimaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(latestPrimaryCheckPoint)) { - this.latestPrimaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); + ReplicationCheckpoint primaryCheckPoint = this.primaryCheckpoint.get(indexShard.shardId()); + if (primaryCheckPoint == null || latestReceivedCheckPoint.isAheadOf(primaryCheckPoint)) { + this.primaryCheckpoint.put(indexShard.shardId(), latestReceivedCheckPoint); calculateReplicationCheckpointStats(latestReceivedCheckPoint, indexShard); } } @@ -171,7 +171,7 @@ public void updateReplicationCheckpointStats(final ReplicationCheckpoint latestR * @param indexShard The index shard to prune checkpoints for */ protected void pruneCheckpointsUpToLastSync(final IndexShard indexShard) { - ReplicationCheckpoint latestCheckpoint = this.latestPrimaryCheckpoint.get(indexShard.shardId()); + ReplicationCheckpoint latestCheckpoint = this.primaryCheckpoint.get(indexShard.shardId()); if (latestCheckpoint != null) { ReplicationCheckpoint indexReplicationCheckPoint = indexShard.getLatestReplicationCheckpoint(); long segmentInfoVersion = indexReplicationCheckPoint.getSegmentInfosVersion(); @@ -302,7 +302,6 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) replicationId = onGoingReplications.startSafe(target, timeout); } catch (ReplicationFailedException e) { // replication already running for shard. - fetchPrimaryLastRefreshedCheckpoint(target); target.fail(e, false); return; } @@ -310,25 +309,6 @@ void startReplication(final SegmentReplicationTarget target, TimeValue timeout) threadPool.generic().execute(new ReplicationRunner(replicationId)); } - private void fetchPrimaryLastRefreshedCheckpoint(SegmentReplicationTarget target) { - // Only process search-only shards - if (!target.indexShard().routingEntry().isSearchOnly()) { - return; - } - - sourceFactory.get().get(target.indexShard()).getCheckpointMetadata(target.getId(), target.getCheckpoint(), new ActionListener<>() { - @Override - public void onResponse(CheckpointInfoResponse checkpointInfoResponse) { - updateReplicationCheckpointStats(checkpointInfoResponse.getCheckpoint(), target.indexShard()); - } - - @Override - public void onFailure(Exception e) { - logger.error("Failed to fetch primary last refreshed checkpoint", e); - } - }); - } - private boolean isStoreCorrupt(SegmentReplicationTarget target) { // ensure target is not already closed. In that case // we can assume the store is not corrupt and that the replication @@ -357,15 +337,15 @@ int size() { void cancel(ShardId shardId, String reason) { onGoingReplications.cancelForShard(shardId, reason); replicationCheckpointStats.remove(shardId); - latestPrimaryCheckpoint.remove(shardId); + primaryCheckpoint.remove(shardId); } SegmentReplicationTarget get(ShardId shardId) { return onGoingReplications.getOngoingReplicationTarget(shardId); } - ReplicationCheckpoint getLatestPrimaryCheckpoint(ShardId shardId) { - return latestPrimaryCheckpoint.get(shardId); + ReplicationCheckpoint getPrimaryCheckpoint(ShardId shardId) { + return primaryCheckpoint.getOrDefault(shardId, ReplicationCheckpoint.empty(shardId)); } ReplicationCollection.ReplicationRef get(long id) { diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 19d684595a480..6d6d9e4c0b553 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; +import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -47,8 +48,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import org.mockito.Mockito; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -215,15 +214,6 @@ public void testGetSegmentReplicationStats_WhenNoReplication() { assertEquals(0, replicationStats.maxBytesBehind); } - public void testGetSegmentReplicationStats_When() { - ShardId shardId = new ShardId("index", "uuid", 0); - SegmentReplicator segmentReplicator = new SegmentReplicator(threadPool); - ReplicationStats replicationStats = segmentReplicator.getSegmentReplicationStats(shardId); - assertEquals(0, replicationStats.maxReplicationLag); - assertEquals(0, replicationStats.maxBytesBehind); - assertEquals(0, replicationStats.totalBytesBehind); - } - public void testGetSegmentReplicationStats_WhileOnGoingReplicationAndPrimaryRefreshedToNewCheckPoint() { ShardId shardId = new ShardId("index", "uuid", 0); ReplicationCheckpoint firstReplicationCheckpoint = ReplicationCheckpoint.empty(shardId); @@ -365,7 +355,7 @@ public void testUpdateReplicationCheckpointStatsIgnoresWhenOutOfOrderCheckPointR ); segmentReplicator.updateReplicationCheckpointStats(replicationCheckpoint, replicaShard); - assertEquals(replicationCheckpoint, segmentReplicator.getLatestPrimaryCheckpoint(shardId)); + assertEquals(replicationCheckpoint, segmentReplicator.getPrimaryCheckpoint(shardId)); ReplicationCheckpoint oldReplicationCheckpoint = new ReplicationCheckpoint( shardId, @@ -379,7 +369,7 @@ public void testUpdateReplicationCheckpointStatsIgnoresWhenOutOfOrderCheckPointR ); segmentReplicator.updateReplicationCheckpointStats(oldReplicationCheckpoint, replicaShard); - assertEquals(replicationCheckpoint, segmentReplicator.getLatestPrimaryCheckpoint(shardId)); + assertEquals(replicationCheckpoint, segmentReplicator.getPrimaryCheckpoint(shardId)); } protected void resolveCheckpointListener(ActionListener listener, IndexShard primary) { From d8585f7d3b391cde03f3884542e363c6075bc96e Mon Sep 17 00:00:00 2001 From: Vinay Krishna Pudyodu Date: Wed, 26 Feb 2025 14:21:43 -0800 Subject: [PATCH 25/25] spotlessApply Signed-off-by: Vinay Krishna Pudyodu --- .../opensearch/indices/replication/SegmentReplicatorTests.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java index 6d6d9e4c0b553..38f1c59bd5b68 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicatorTests.java @@ -11,7 +11,6 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Version; -import org.mockito.Mockito; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -48,6 +47,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; +import org.mockito.Mockito; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when;