From cc0979c76a027ae333ecfd98c4b0ed55a48b2dd2 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Thu, 10 Mar 2022 15:18:32 -0800 Subject: [PATCH 1/8] Further refactoring of GatedAutoCloseable Since this class wraps two subclasses of RefCounted, and has the same shutdown hook for both (decRef), I've made the functionality of the class less generic to increase code convergence. Signed-off-by: Kartik Ganesh --- .../concurrent/AutoCloseableRefCounted.java | 42 ++++++++++++++++++ .../common/concurrent/GatedAutoCloseable.java | 43 ------------------- .../common/concurrent/GatedCloseable.java | 2 +- .../recovery/PeerRecoveryTargetService.java | 25 ++++++----- .../recovery/RecoveriesCollection.java | 9 ++-- .../SegmentReplicationReplicaService.java | 5 ++- .../copy/PrimaryShardReplicationSource.java | 4 +- .../copy/ReplicationCollection.java | 11 ++--- ...java => AutoCloseableRefCountedTests.java} | 21 +++++---- .../recovery/RecoveriesCollectionTests.java | 15 ++++--- 10 files changed, 93 insertions(+), 84 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java delete mode 100644 server/src/main/java/org/opensearch/common/concurrent/GatedAutoCloseable.java rename server/src/test/java/org/opensearch/common/concurrent/{GatedAutoCloseableTests.java => AutoCloseableRefCountedTests.java} (50%) diff --git a/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java b/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java new file mode 100644 index 0000000000000..86bcdd3838e23 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.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. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.common.concurrent; + +import org.opensearch.common.util.concurrent.RefCounted; + +/** + * Adapter class that enables a {@link RefCounted} implementation to function like an {@link AutoCloseable}. + * The {@link #close()} API invokes {@link RefCounted#decRef()} and ensures idempotency using a {@link OneWayGate}. + */ +public class AutoCloseableRefCounted implements AutoCloseable { + + private final RefCounted ref; + private final OneWayGate gate; + + public AutoCloseableRefCounted(RefCounted ref) { + this.ref = ref; + gate = new OneWayGate(); + } + + public T get(Class returnType) { + return (T) ref; + } + + @Override + public void close() { + if (gate.close()) { + ref.decRef(); + } + } +} diff --git a/server/src/main/java/org/opensearch/common/concurrent/GatedAutoCloseable.java b/server/src/main/java/org/opensearch/common/concurrent/GatedAutoCloseable.java deleted file mode 100644 index cb819c0320e91..0000000000000 --- a/server/src/main/java/org/opensearch/common/concurrent/GatedAutoCloseable.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -/* - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -package org.opensearch.common.concurrent; - -/** - * Decorator class that wraps an object reference with a {@link Runnable} that is - * invoked when {@link #close()} is called. The internal {@link OneWayGate} instance ensures - * that this is invoked only once. See also {@link GatedCloseable} - */ -public class GatedAutoCloseable implements AutoCloseable { - - private final T ref; - private final Runnable onClose; - private final OneWayGate gate; - - public GatedAutoCloseable(T ref, Runnable onClose) { - this.ref = ref; - this.onClose = onClose; - gate = new OneWayGate(); - } - - public T get() { - return ref; - } - - @Override - public void close() { - if (gate.close()) { - onClose.run(); - } - } -} diff --git a/server/src/main/java/org/opensearch/common/concurrent/GatedCloseable.java b/server/src/main/java/org/opensearch/common/concurrent/GatedCloseable.java index d98e4cca8d561..467b5e4cfb3ea 100644 --- a/server/src/main/java/org/opensearch/common/concurrent/GatedCloseable.java +++ b/server/src/main/java/org/opensearch/common/concurrent/GatedCloseable.java @@ -21,7 +21,7 @@ /** * Decorator class that wraps an object reference with a {@link CheckedRunnable} that is * invoked when {@link #close()} is called. The internal {@link OneWayGate} instance ensures - * that this is invoked only once. See also {@link GatedAutoCloseable} + * that this is invoked only once. See also {@link AutoCloseableRefCounted} */ public class GatedCloseable implements Closeable { diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 30b1f058ea72e..25564f23ab34b 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -222,7 +222,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi logger.trace("not running recovery with id [{}] - can not find it (probably finished)", recoveryId); return; } - final RecoveryTarget recoveryTarget = recoveryRef.get(); + final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); timer = recoveryTarget.state().getTimer(); cancellableThreads = recoveryTarget.cancellableThreads(); if (preExistingRequest == null) { @@ -362,8 +362,7 @@ public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, if (listener == null) { return; } - - recoveryRef.get().prepareForTranslogOperations(request.totalTranslogOps(), listener); + recoveryRef.get(RecoveryTarget.class).prepareForTranslogOperations(request.totalTranslogOps(), listener); } } } @@ -377,8 +376,7 @@ public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportCh if (listener == null) { return; } - - recoveryRef.get().finalizeRecovery(request.globalCheckpoint(), request.trimAboveSeqNo(), listener); + recoveryRef.get(RecoveryTarget.class).finalizeRecovery(request.globalCheckpoint(), request.trimAboveSeqNo(), listener); } } } @@ -389,7 +387,7 @@ class HandoffPrimaryContextRequestHandler implements TransportRequestHandler listener = createOrFinishListener( recoveryRef, channel, @@ -423,7 +421,7 @@ private void performTranslogOps( final ActionListener listener, final RecoveryRef recoveryRef ) { - final RecoveryTarget recoveryTarget = recoveryRef.get(); + final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()); final Consumer retryOnMappingException = exception -> { @@ -488,7 +486,7 @@ public void messageReceived(RecoveryFilesInfoRequest request, TransportChannel c return; } - recoveryRef.get() + recoveryRef.get(RecoveryTarget.class) .receiveFileInfo( request.phase1FileNames, request.phase1FileSizes, @@ -511,7 +509,7 @@ public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel return; } - recoveryRef.get() + recoveryRef.get(RecoveryTarget.class) .cleanFiles(request.totalTranslogOps(), request.getGlobalCheckpoint(), request.sourceMetaSnapshot(), listener); } } @@ -525,7 +523,7 @@ class FileChunkTransportRequestHandler implements TransportRequestHandler listener = createOrFinishListener(recoveryRef, channel, Actions.FILE_CHUNK, request); if (listener == null) { return; @@ -575,7 +573,7 @@ private ActionListener createOrFinishListener( final RecoveryTransportRequest request, final CheckedFunction responseFn ) { - final RecoveryTarget recoveryTarget = recoveryRef.get(); + final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); final ActionListener voidListener = ActionListener.map(channelListener, responseFn); @@ -609,9 +607,10 @@ public void onFailure(Exception e) { try (RecoveryRef recoveryRef = onGoingRecoveries.getRecovery(recoveryId)) { if (recoveryRef != null) { logger.error(() -> new ParameterizedMessage("unexpected error during recovery [{}], failing shard", recoveryId), e); + RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); onGoingRecoveries.failRecovery( recoveryId, - new RecoveryFailedException(recoveryRef.get().state(), "unexpected error", e), + new RecoveryFailedException(recoveryTarget.state(), "unexpected error", e), true // be safe ); } else { diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java index 3c197a8e33eb6..48af0b47b1bf6 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java @@ -36,7 +36,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchTimeoutException; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.concurrent.GatedAutoCloseable; +import org.opensearch.common.concurrent.AutoCloseableRefCounted; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ConcurrentCollections; @@ -178,7 +178,8 @@ public RecoveryRef getRecoverySafe(long id, ShardId shardId) { if (recoveryRef == null) { throw new IndexShardClosedException(shardId); } - assert recoveryRef.get().shardId().equals(shardId); + RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + assert recoveryTarget.shardId().equals(shardId); return recoveryRef; } @@ -273,14 +274,14 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { * causes {@link RecoveryTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link RecoveryRef#close()} is called. */ - public static class RecoveryRef extends GatedAutoCloseable { + public static class RecoveryRef extends AutoCloseableRefCounted { /** * Important: {@link RecoveryTarget#tryIncRef()} should * be *successfully* called on status before */ public RecoveryRef(RecoveryTarget status) { - super(status, status::decRef); + super(status); status.setLastAccessTime(); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index a2e403f563dfa..0f9d165c64613 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -178,7 +178,7 @@ private void doReplication(final long replicationId) { logger.trace("not running replication with id [{}] - can not find it (probably finished)", replicationId); return; } - final ReplicationTarget replicationTarget = replicationRef.get(); + final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); timer = replicationTarget.state().getTimer(); final IndexShard indexShard = replicationTarget.getIndexShard(); @@ -215,9 +215,10 @@ public void onFailure(Exception e) { () -> new ParameterizedMessage("unexpected error during replication [{}], failing shard", replicationId), e ); + ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); onGoingReplications.failReplication( replicationId, - new ReplicationFailedException(replicationRef.get().getIndexShard(), "unexpected error", e), + new ReplicationFailedException(replicationTarget.getIndexShard(), "unexpected error", e), true // be safe ); } else { diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java index 3be08e6c10ea9..e934ac2ffc427 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java @@ -231,7 +231,7 @@ public void messageReceived(final ReplicationFileChunkRequest request, Transport ReplicationCollection.ReplicationRef replicationRef = segmentReplicationReplicaService.getOnGoingReplications() .getReplicationSafe(request.getReplicationId(), request.shardId()) ) { - final ReplicationTarget replicationTarget = replicationRef.get(); + final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); final ActionListener listener = createOrFinishListener(replicationRef, channel, Actions.FILE_CHUNK, request); if (listener == null) { return; @@ -274,7 +274,7 @@ private ActionListener createOrFinishListener( final ReplicationFileChunkRequest request, final CheckedFunction responseFn ) { - final ReplicationTarget replicationTarget = replicationRef.get(); + final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); final ActionListener voidListener = ActionListener.map(channelListener, responseFn); diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java index 248d8d8f833c6..f0eec86f6e3cf 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -35,7 +35,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.OpenSearchTimeoutException; -import org.opensearch.common.concurrent.GatedAutoCloseable; +import org.opensearch.common.concurrent.AutoCloseableRefCounted; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ConcurrentCollections; @@ -125,7 +125,8 @@ public ReplicationRef getReplicationSafe(long id, ShardId shardId) { if (replicationRef == null) { throw new IndexShardClosedException(shardId); } - assert replicationRef.get().getIndexShard().shardId().equals(shardId); + ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + assert replicationTarget.getIndexShard().shardId().equals(shardId); return replicationRef; } @@ -221,14 +222,14 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { * causes {@link ReplicationTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link ReplicationRef#close()} is called. */ - public static class ReplicationRef extends GatedAutoCloseable { + public static class ReplicationRef extends AutoCloseableRefCounted { /** * Important: {@link ReplicationTarget#tryIncRef()} should - * be *successfully* called on status before + * be *successfully* called on target before */ public ReplicationRef(ReplicationTarget target) { - super(target, target::decRef); + super(target); target.setLastAccessTime(); } } diff --git a/server/src/test/java/org/opensearch/common/concurrent/GatedAutoCloseableTests.java b/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java similarity index 50% rename from server/src/test/java/org/opensearch/common/concurrent/GatedAutoCloseableTests.java rename to server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java index 63058da8f163a..d0604680e5443 100644 --- a/server/src/test/java/org/opensearch/common/concurrent/GatedAutoCloseableTests.java +++ b/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java @@ -14,33 +14,36 @@ package org.opensearch.common.concurrent; import org.junit.Before; +import org.opensearch.common.util.concurrent.RefCounted; import org.opensearch.test.OpenSearchTestCase; -import java.util.concurrent.atomic.AtomicInteger; +import static org.mockito.Mockito.atMostOnce; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; -public class GatedAutoCloseableTests extends OpenSearchTestCase { +public class AutoCloseableRefCountedTests extends OpenSearchTestCase { - private AtomicInteger testRef; - private GatedAutoCloseable testObject; + private RefCounted mockRefCounted; + private AutoCloseableRefCounted testObject; @Before public void setup() { - testRef = new AtomicInteger(0); - testObject = new GatedAutoCloseable<>(testRef, testRef::incrementAndGet); + mockRefCounted = mock(RefCounted.class); + testObject = new AutoCloseableRefCounted(mockRefCounted); } public void testGet() { - assertEquals(0, testObject.get().get()); + assertEquals(mockRefCounted, testObject.get(RefCounted.class)); } public void testClose() { testObject.close(); - assertEquals(1, testObject.get().get()); + verify(mockRefCounted, atMostOnce()).decRef(); } public void testIdempotent() { testObject.close(); testObject.close(); - assertEquals(1, testObject.get().get()); + verify(mockRefCounted, atMostOnce()).decRef(); } } diff --git a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java index 6a08f5115d1e2..66505826a8ea7 100644 --- a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java @@ -69,10 +69,14 @@ public void testLastAccessTimeUpdate() throws Exception { final RecoveriesCollection collection = new RecoveriesCollection(logger, threadPool); final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); try (RecoveriesCollection.RecoveryRef status = collection.getRecovery(recoveryId)) { - final long lastSeenTime = status.get().lastAccessTime(); + final long lastSeenTime = status.get(RecoveryTarget.class).lastAccessTime(); assertBusy(() -> { try (RecoveriesCollection.RecoveryRef currentStatus = collection.getRecovery(recoveryId)) { - assertThat("access time failed to update", lastSeenTime, lessThan(currentStatus.get().lastAccessTime())); + assertThat( + "access time failed to update", + lastSeenTime, + lessThan(currentStatus.get(RecoveryTarget.class).lastAccessTime()) + ); } }); } finally { @@ -120,7 +124,7 @@ public void testRecoveryCancellation() throws Exception { final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); final long recoveryId2 = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); try (RecoveriesCollection.RecoveryRef recoveryRef = collection.getRecovery(recoveryId)) { - ShardId shardId = recoveryRef.get().shardId(); + ShardId shardId = recoveryRef.get(RecoveryTarget.class).shardId(); assertTrue("failed to cancel recoveries", collection.cancelRecoveriesForShard(shardId, "test")); assertThat("all recoveries should be cancelled", collection.size(), equalTo(0)); } finally { @@ -160,8 +164,9 @@ public void testResetRecovery() throws Exception { assertEquals(currentAsTarget, shard.recoveryStats().currentAsTarget()); try (RecoveriesCollection.RecoveryRef newRecoveryRef = collection.getRecovery(resetRecoveryId)) { shards.recoverReplica(shard, (s, n) -> { - assertSame(s, newRecoveryRef.get().indexShard()); - return newRecoveryRef.get(); + RecoveryTarget newRecoveryTarget = newRecoveryRef.get(RecoveryTarget.class); + assertSame(s, newRecoveryTarget.indexShard()); + return newRecoveryTarget; }, false); } shards.assertAllEqual(numDocs); From 76f09bb441805b93376020e2e5e9184712d3e430 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Fri, 11 Mar 2022 12:34:32 -0800 Subject: [PATCH 2/8] Starting to merge common replication functionality into parent classes Parent class names are currently temporary placeholders. They will renamed in a separate commit to avoid a convoluted commit diff. A large part of duplicate functionality in the two *Target classes has been moved into the parent class. The two *Listeners are also very similar and now implement a common interface. The usec-ase specific methods in the listeners will eventually be replaced by their generic equivalents. Finally, the State parent class is currently a placeholder and will soon hold more logic. Some other parts of the code were also changed to make call patterns more streamlined. Signed-off-by: Kartik Ganesh --- .../recovery/PeerRecoveryTargetService.java | 15 +- .../recovery/RecoveriesCollection.java | 7 +- .../indices/recovery/RecoveryState.java | 3 +- .../indices/recovery/RecoveryTarget.java | 209 +++++------------- .../SegmentReplicationReplicaService.java | 20 +- .../indices/replication/common/RListener.java | 18 ++ .../indices/replication/common/RState.java | 14 ++ .../indices/replication/common/RTarget.java | 183 +++++++++++++++ .../copy/ReplicationCollection.java | 16 +- .../replication/copy/ReplicationState.java | 3 +- .../replication/copy/ReplicationTarget.java | 197 ++++------------- 11 files changed, 360 insertions(+), 325 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/replication/common/RListener.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/common/RState.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/common/RTarget.java diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 25564f23ab34b..f95ff43759cb0 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -70,6 +70,8 @@ import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogCorruptedException; import org.opensearch.indices.recovery.RecoveriesCollection.RecoveryRef; +import org.opensearch.indices.replication.common.RListener; +import org.opensearch.indices.replication.common.RState; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.ConnectTransportException; @@ -347,7 +349,18 @@ public static StartRecoveryRequest getStartRecoveryRequest( return request; } - public interface RecoveryListener { + public interface RecoveryListener extends RListener { + + @Override + default void onDone(RState state) { + onRecoveryDone((RecoveryState) state); + } + + @Override + default void onFailure(RState state, OpenSearchException e, boolean sendShardFailure) { + onRecoveryFailure((RecoveryState) state, (RecoveryFailedException) e, sendShardFailure); + } + void onRecoveryDone(RecoveryState state); void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java index 48af0b47b1bf6..8779e805a8476 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java @@ -120,7 +120,12 @@ public RecoveryTarget resetRecovery(final long recoveryId, final TimeValue activ return null; } - newRecoveryTarget = oldRecoveryTarget.retryCopy(); + // Copy the RecoveryTarget to retry recovery from the same source node onto the same shard and using the same listener. + newRecoveryTarget = new RecoveryTarget( + oldRecoveryTarget.indexShard(), + oldRecoveryTarget.sourceNode(), + oldRecoveryTarget.getListener() + ); startRecoveryInternal(newRecoveryTarget, activityTimeout); } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java index 1a11345e8e47d..802deb577eff7 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java @@ -50,6 +50,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.index.store.StoreStats; +import org.opensearch.indices.replication.common.RState; import java.io.IOException; import java.util.ArrayList; @@ -63,7 +64,7 @@ /** * Keeps track of state related to shard recovery. */ -public class RecoveryState implements ToXContentFragment, Writeable { +public class RecoveryState implements ToXContentFragment, Writeable, RState { public enum Stage { INIT((byte) 0), diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java index 3ea7cad528e82..070825a196dec 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java @@ -32,22 +32,18 @@ package org.opensearch.indices.recovery; -import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; import org.opensearch.Assertions; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.logging.Loggers; import org.opensearch.common.lucene.Lucene; import org.opensearch.common.util.CancellableThreads; -import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.MapperException; import org.opensearch.index.seqno.ReplicationTracker; @@ -60,42 +56,23 @@ import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.index.translog.Translog; +import org.opensearch.indices.replication.common.RTarget; import java.io.IOException; import java.nio.file.Path; import java.util.List; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; /** * Represents a recovery where the current node is the target node of the recovery. To track recoveries in a central place, instances of * this class are created through {@link RecoveriesCollection}. */ -public class RecoveryTarget extends AbstractRefCounted implements RecoveryTargetHandler { - - private final Logger logger; - - private static final AtomicLong idGenerator = new AtomicLong(); +public class RecoveryTarget extends RTarget implements RecoveryTargetHandler { private static final String RECOVERY_PREFIX = "recovery."; - private final ShardId shardId; - private final long recoveryId; - private final IndexShard indexShard; private final DiscoveryNode sourceNode; - private final MultiFileWriter multiFileWriter; - private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker(); - private final Store store; - private final PeerRecoveryTargetService.RecoveryListener listener; - - private final AtomicBoolean finished = new AtomicBoolean(); - private final CancellableThreads cancellableThreads; - - // last time this status was accessed - private volatile long lastAccessTime = System.nanoTime(); - // latch that can be used to blockingly wait for RecoveryTarget to be closed private final CountDownLatch closedLatch = new CountDownLatch(1); @@ -107,79 +84,72 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget * @param listener called when recovery is completed/failed */ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener) { - super("recovery_status"); + super("recovery_status", indexShard, indexShard.recoveryState().getIndex(), listener); this.cancellableThreads = new CancellableThreads(); - this.recoveryId = idGenerator.incrementAndGet(); - this.listener = listener; - this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); - this.indexShard = indexShard; this.sourceNode = sourceNode; - this.shardId = indexShard.shardId(); - final String tempFilePrefix = RECOVERY_PREFIX + UUIDs.randomBase64UUID() + "."; - this.multiFileWriter = new MultiFileWriter( - indexShard.store(), - indexShard.recoveryState().getIndex(), - tempFilePrefix, - logger, - this::ensureRefCount - ); - this.store = indexShard.store(); - // make sure the store is not released until we are done. - store.incRef(); indexShard.recoveryStats().incCurrentAsTarget(); } - /** - * Returns a fresh recovery target to retry recovery from the same source node onto the same shard and using the same listener. - * - * @return a copy of this recovery target - */ - public RecoveryTarget retryCopy() { - return new RecoveryTarget(indexShard, sourceNode, listener); + @Override + protected String getPrefix() { + return RECOVERY_PREFIX; } - public ActionListener markRequestReceivedAndCreateListener(long requestSeqNo, ActionListener listener) { - return requestTracker.markReceivedAndCreateListener(requestSeqNo, listener); + @Override + protected void onDone() { + indexShard.postRecovery("peer recovery done"); } - public long recoveryId() { - return recoveryId; + @Override + public RecoveryState state() { + return indexShard.recoveryState(); } - public ShardId shardId() { - return shardId; + /** + * if {@link #cancellableThreads()} was used, the threads will be interrupted. + */ + @Override + protected void onCancel(String reason) { + cancellableThreads.cancel(reason); } - public IndexShard indexShard() { - ensureRefCount(); - return indexShard; + @Override + protected void onFail(OpenSearchException e, boolean sendShardFailure) { + cancellableThreads.cancel("failed recovery [" + ExceptionsHelper.stackTrace(e) + "]"); } - public DiscoveryNode sourceNode() { - return this.sourceNode; + public void fail(RecoveryFailedException e, boolean sendShardFailure) { + super.fail(e, sendShardFailure); } - public RecoveryState state() { - return indexShard.recoveryState(); + @Override + protected void closeInternal() { + try { + super.closeInternal(); + } finally { + indexShard.recoveryStats().decCurrentAsTarget(); + closedLatch.countDown(); + } } - public CancellableThreads cancellableThreads() { - return cancellableThreads; + public PeerRecoveryTargetService.RecoveryListener getListener() { + return (PeerRecoveryTargetService.RecoveryListener) listener; } - /** return the last time this RecoveryStatus was used (based on System.nanoTime() */ - public long lastAccessTime() { - return lastAccessTime; + public long recoveryId() { + return getId(); } - /** sets the lasAccessTime flag to now */ - public void setLastAccessTime() { - lastAccessTime = System.nanoTime(); + public ShardId shardId() { + return indexShard.shardId(); } - public Store store() { - ensureRefCount(); - return store; + public DiscoveryNode sourceNode() { + return this.sourceNode; + } + + public CancellableThreads cancellableThreads() { + return cancellableThreads; } /** @@ -187,9 +157,10 @@ public Store store() { * Returns true if resetting the recovery was successful, false if the recovery target is already cancelled / failed or marked as done. */ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOException { + final long recoveryId = getId(); if (finished.compareAndSet(false, true)) { try { - logger.debug("reset of recovery with shard {} and id [{}]", shardId, recoveryId); + logger.debug("reset of recovery with shard {} and id [{}]", shardId(), recoveryId); } finally { // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now. decRef(); @@ -199,7 +170,7 @@ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOE } catch (CancellableThreads.ExecutionCancelledException e) { logger.trace( "new recovery target cancelled for shard {} while waiting on old recovery target with id [{}] to close", - shardId, + shardId(), recoveryId ); return false; @@ -219,89 +190,13 @@ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOE return false; } - /** - * cancel the recovery. calling this method will clean temporary files and release the store - * unless this object is in use (in which case it will be cleaned once all ongoing users call - * {@link #decRef()} - *

- * if {@link #cancellableThreads()} was used, the threads will be interrupted. - */ - public void cancel(String reason) { - if (finished.compareAndSet(false, true)) { - try { - logger.debug("recovery canceled (reason: [{}])", reason); - cancellableThreads.cancel(reason); - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - } - } - - /** - * fail the recovery and call listener - * - * @param e exception that encapsulating the failure - * @param sendShardFailure indicates whether to notify the master of the shard failure - */ - public void fail(RecoveryFailedException e, boolean sendShardFailure) { - if (finished.compareAndSet(false, true)) { - try { - notifyListener(e, sendShardFailure); - } finally { - try { - cancellableThreads.cancel("failed recovery [" + ExceptionsHelper.stackTrace(e) + "]"); - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - } - } - } - public void notifyListener(RecoveryFailedException e, boolean sendShardFailure) { - listener.onRecoveryFailure(state(), e, sendShardFailure); - } - - /** mark the current recovery as done */ - public void markAsDone() { - if (finished.compareAndSet(false, true)) { - assert multiFileWriter.tempFileNames.isEmpty() : "not all temporary files are renamed"; - try { - // this might still throw an exception ie. if the shard is CLOSED due to some other event. - // it's safer to decrement the reference in a try finally here. - indexShard.postRecovery("peer recovery done"); - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - listener.onRecoveryDone(state()); - } - } - - @Override - protected void closeInternal() { - try { - multiFileWriter.close(); - } finally { - // free store. increment happens in constructor - store.decRef(); - indexShard.recoveryStats().decCurrentAsTarget(); - closedLatch.countDown(); - } + listener.onFailure(state(), e, sendShardFailure); } @Override public String toString() { - return shardId + " [" + recoveryId + "]"; - } - - private void ensureRefCount() { - if (refCount() <= 0) { - throw new OpenSearchException( - "RecoveryStatus is used but it's refcount is 0. Probably a mismatch between incRef/decRef " + "calls" - ); - } + return shardId() + " [" + getId() + "]"; } /*** Implementation of {@link RecoveryTargetHandler } */ @@ -371,7 +266,7 @@ public void indexTranslogOperations( translog.totalOperations(totalTranslogOps); assert indexShard().recoveryState() == state(); if (indexShard().state() != IndexShardState.RECOVERING) { - throw new IndexShardNotRecoveringException(shardId, indexShard().state()); + throw new IndexShardNotRecoveringException(shardId(), indexShard().state()); } /* * The maxSeenAutoIdTimestampOnPrimary received from the primary is at least the highest auto_id_timestamp from any operation @@ -457,7 +352,7 @@ public void cleanFiles( final String translogUUID = Translog.createEmptyTranslog( indexShard.shardPath().resolveTranslog(), globalCheckpoint, - shardId, + shardId(), indexShard.getPendingPrimaryTerm() ); store.associateIndexWithNewTranslog(translogUUID); @@ -510,6 +405,8 @@ public void writeFileChunk( int totalTranslogOps, ActionListener listener ) { + state().getTranslog().totalOperations(totalTranslogOps); + this.writeFileChunk(fileMetadata, position, content, lastChunk, listener); try { state().getTranslog().totalOperations(totalTranslogOps); multiFileWriter.writeFileChunk(fileMetadata, position, content, lastChunk); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index 0f9d165c64613..0bf56f1f8474c 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.support.RetryableAction; @@ -35,6 +36,8 @@ import org.opensearch.indices.recovery.DelayRecoveryException; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.Timer; +import org.opensearch.indices.replication.common.RListener; +import org.opensearch.indices.replication.common.RState; import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.indices.replication.copy.ReplicationCheckpoint; import org.opensearch.indices.replication.copy.ReplicationCollection; @@ -180,7 +183,7 @@ private void doReplication(final long replicationId) { } final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); timer = replicationTarget.state().getTimer(); - final IndexShard indexShard = replicationTarget.getIndexShard(); + final IndexShard indexShard = replicationTarget.indexShard(); try { logger.trace("{} preparing shard for replication", indexShard.shardId()); @@ -218,7 +221,7 @@ public void onFailure(Exception e) { ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); onGoingReplications.failReplication( replicationId, - new ReplicationFailedException(replicationTarget.getIndexShard(), "unexpected error", e), + new ReplicationFailedException(replicationTarget.indexShard(), "unexpected error", e), true // be safe ); } else { @@ -285,7 +288,18 @@ public void onFailure(Exception e) { } } - public interface ReplicationListener { + public interface ReplicationListener extends RListener { + + @Override + default void onDone(RState state) { + onReplicationDone((ReplicationState) state); + } + + @Override + default void onFailure(RState state, OpenSearchException e, boolean sendShardFailure) { + onReplicationFailure((ReplicationState) state, (ReplicationFailedException) e, sendShardFailure); + } + void onReplicationDone(ReplicationState state); void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure); diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RListener.java b/server/src/main/java/org/opensearch/indices/replication/common/RListener.java new file mode 100644 index 0000000000000..44d813d952bfb --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/common/RListener.java @@ -0,0 +1,18 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.common; + +import org.opensearch.OpenSearchException; + +public interface RListener { + + void onDone(RState state); + + void onFailure(RState state, OpenSearchException e, boolean sendShardFailure); +} diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RState.java b/server/src/main/java/org/opensearch/indices/replication/common/RState.java new file mode 100644 index 0000000000000..bfc57d9b6686f --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/common/RState.java @@ -0,0 +1,14 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.common; + +public interface RState { + + // TODO Add APIs here +} diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java new file mode 100644 index 0000000000000..2976f8ecbf427 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java @@ -0,0 +1,183 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.common; + +import org.apache.logging.log4j.Logger; +import org.opensearch.OpenSearchException; +import org.opensearch.action.ActionListener; +import org.opensearch.common.UUIDs; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.logging.Loggers; +import org.opensearch.common.util.concurrent.AbstractRefCounted; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.MultiFileWriter; +import org.opensearch.indices.recovery.RecoveryRequestTracker; +import org.opensearch.indices.recovery.RecoveryState; + +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +public abstract class RTarget extends AbstractRefCounted { + + // TODO will this cause issues because its shared between subclasses? + private static final AtomicLong ID_GENERATOR = new AtomicLong(); + + // last time the target/status was accessed + private volatile long lastAccessTime = System.nanoTime(); + private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker(); + private final long id; + + protected final AtomicBoolean finished = new AtomicBoolean(); + protected final IndexShard indexShard; + protected final Store store; + protected final RListener listener; + protected final MultiFileWriter multiFileWriter; + protected final Logger logger; + + protected abstract String getPrefix(); + + protected abstract void onDone(); + + protected abstract void onCancel(String reason); + + protected abstract void onFail(OpenSearchException e, boolean sendShardFailure); + + public abstract RState state(); + + public RTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryIndex, RListener listener) { + super(name); + this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); + this.listener = listener; + this.id = ID_GENERATOR.incrementAndGet(); + this.indexShard = indexShard; + this.store = indexShard.store(); + final String tempFilePrefix = getPrefix() + UUIDs.randomBase64UUID() + "."; + this.multiFileWriter = new MultiFileWriter(indexShard.store(), recoveryIndex, tempFilePrefix, logger, this::ensureRefCount); + // make sure the store is not released until we are done. + store.incRef(); + } + + public long getId() { + return id; + } + + /** + * return the last time this RecoveryStatus was used (based on System.nanoTime() + */ + public long lastAccessTime() { + return lastAccessTime; + } + + /** + * sets the lasAccessTime flag to now + */ + public void setLastAccessTime() { + lastAccessTime = System.nanoTime(); + } + + public ActionListener markRequestReceivedAndCreateListener(long requestSeqNo, ActionListener listener) { + return requestTracker.markReceivedAndCreateListener(requestSeqNo, listener); + } + + public IndexShard indexShard() { + ensureRefCount(); + return indexShard; + } + + public Store store() { + ensureRefCount(); + return store; + } + + public void writeFileChunk( + StoreFileMetadata fileMetadata, + long position, + BytesReference content, + boolean lastChunk, + ActionListener actionListener + ) { + try { + multiFileWriter.writeFileChunk(fileMetadata, position, content, lastChunk); + actionListener.onResponse(null); + } catch (Exception e) { + actionListener.onFailure(e); + } + } + + /** + * mark the current recovery as done + */ + public void markAsDone() { + if (finished.compareAndSet(false, true)) { + try { + onDone(); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + listener.onDone(state()); + } + } + + /** + * cancel the recovery. calling this method will clean temporary files and release the store + * unless this object is in use (in which case it will be cleaned once all ongoing users call + * {@link #decRef()} + */ + public void cancel(String reason) { + if (finished.compareAndSet(false, true)) { + try { + logger.debug("recovery canceled (reason: [{}])", reason); + onCancel(reason); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + } + } + + /** + * fail the recovery and call listener + * + * @param e exception that encapsulating the failure + * @param sendShardFailure indicates whether to notify the master of the shard failure + */ + public void fail(OpenSearchException e, boolean sendShardFailure) { + if (finished.compareAndSet(false, true)) { + try { + listener.onFailure(state(), e, sendShardFailure); + } finally { + try { + onFail(e, sendShardFailure); + } finally { + // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now + decRef(); + } + } + } + } + + protected void ensureRefCount() { + if (refCount() <= 0) { + throw new OpenSearchException("RecoveryStatus is used but it's refcount is 0. Probably a mismatch between incRef/decRef calls"); + } + } + + @Override + protected void closeInternal() { + try { + multiFileWriter.close(); + } finally { + // free store. increment happens in constructor + store.decRef(); + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java index f0eec86f6e3cf..9286de0b80996 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -88,7 +88,7 @@ private void startReplicationInternal(ReplicationTarget replicationTarget, TimeV assert existingTarget == null : "found two ReplicationStatus instances with the same id"; logger.trace( "{} started segment replication id [{}]", - replicationTarget.getIndexShard().shardId(), + replicationTarget.indexShard().shardId(), replicationTarget.getReplicationId() ); threadPool.schedule( @@ -126,7 +126,7 @@ public ReplicationRef getReplicationSafe(long id, ShardId shardId) { throw new IndexShardClosedException(shardId); } ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); - assert replicationTarget.getIndexShard().shardId().equals(shardId); + assert replicationTarget.indexShard().shardId().equals(shardId); return replicationRef; } @@ -137,7 +137,7 @@ public boolean cancelReplication(long id, String reason) { if (removed != null) { logger.trace( "{} canceled replication, id [{}] (reason [{}])", - removed.getIndexShard().shardId(), + removed.indexShard().shardId(), removed.getReplicationId(), reason ); @@ -159,7 +159,7 @@ public void failReplication(long id, ReplicationFailedException e, boolean sendS if (removed != null) { logger.trace( "{} failing segment replication id [{}]. Send shard failure: [{}]", - removed.getIndexShard().shardId(), + removed.indexShard().shardId(), removed.getReplicationId(), sendShardFailure ); @@ -173,7 +173,7 @@ public void failReplication(long id, ReplicationFailedException e, boolean sendS public void markReplicationAsDone(long id) { ReplicationTarget removed = onGoingReplications.remove(id); if (removed != null) { - logger.trace("{} marking replication as done, id [{}]", removed.getIndexShard().shardId(), removed.getReplicationId()); + logger.trace("{} marking replication as done, id [{}]", removed.indexShard().shardId(), removed.getReplicationId()); removed.markAsDone(); } } @@ -198,7 +198,7 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { synchronized (onGoingReplications) { for (Iterator it = onGoingReplications.values().iterator(); it.hasNext();) { ReplicationTarget status = it.next(); - if (status.getIndexShard().shardId().equals(shardId)) { + if (status.indexShard().shardId().equals(shardId)) { matchedRecoveries.add(status); it.remove(); } @@ -207,7 +207,7 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { for (ReplicationTarget removed : matchedRecoveries) { logger.trace( "{} canceled segment replication id [{}] (reason [{}])", - removed.getIndexShard().shardId(), + removed.indexShard().shardId(), removed.getReplicationId(), reason ); @@ -263,7 +263,7 @@ protected void doRun() throws Exception { String message = "no activity after [" + checkInterval + "]"; failReplication( replicationId, - new ReplicationFailedException(replicationTarget.getIndexShard(), message, new OpenSearchTimeoutException(message)), + new ReplicationFailedException(replicationTarget.indexShard(), message, new OpenSearchTimeoutException(message)), true // to be safe, we don't know what go stuck ); return; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java index f90f7f5eadffe..0856e5bad5dc6 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java @@ -10,8 +10,9 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.Timer; +import org.opensearch.indices.replication.common.RState; -public class ReplicationState { +public class ReplicationState implements RState { private Timer timer; private RecoveryState.Index index; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java index a06af1b697ca7..f3bc7093ea11e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java @@ -32,7 +32,6 @@ package org.opensearch.indices.replication.copy; -import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; @@ -41,52 +40,31 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.StepListener; import org.opensearch.action.support.replication.ReplicationResponse; -import org.opensearch.common.UUIDs; -import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.logging.Loggers; import org.opensearch.common.lucene.Lucene; -import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; -import org.opensearch.indices.recovery.MultiFileWriter; -import org.opensearch.indices.recovery.RecoveryRequestTracker; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.SegmentReplicationReplicaService; import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; +import org.opensearch.indices.replication.common.RTarget; import java.io.IOException; import java.util.Arrays; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; /** * Orchestrates a replication event for a replica shard. */ -public class ReplicationTarget extends AbstractRefCounted { +public class ReplicationTarget extends RTarget { - public ReplicationCheckpoint getCheckpoint() { - return checkpoint; - } + private static final String REPLICATION_PREFIX = "replication."; private final ReplicationCheckpoint checkpoint; - private static final AtomicLong idGenerator = new AtomicLong(); - private final AtomicBoolean finished = new AtomicBoolean(); - private final long replicationId; - private final IndexShard indexShard; - private final Logger logger; private final PrimaryShardReplicationSource source; - private final SegmentReplicationReplicaService.ReplicationListener listener; - private final Store store; - private final MultiFileWriter multiFileWriter; - private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker(); private final ReplicationState state; - private volatile long lastAccessTime = System.nanoTime(); - - private static final String REPLICATION_PREFIX = "replication."; /** * Creates a new replication target object that represents a replication to the provided source. @@ -101,20 +79,47 @@ public ReplicationTarget( PrimaryShardReplicationSource source, SegmentReplicationReplicaService.ReplicationListener listener ) { - super("replication_status"); + super("replication_status", indexShard, new RecoveryState.Index(), listener); this.checkpoint = checkpoint; - this.indexShard = indexShard; - this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); - this.replicationId = idGenerator.incrementAndGet(); this.source = source; - this.listener = listener; - this.store = indexShard.store(); - final String tempFilePrefix = REPLICATION_PREFIX + UUIDs.randomBase64UUID() + "."; state = new ReplicationState(new RecoveryState.Index()); - this.multiFileWriter = new MultiFileWriter(indexShard.store(), state.getIndex(), tempFilePrefix, logger, this::ensureRefCount); - ; - // make sure the store is not released until we are done. - store.incRef(); + } + + @Override + protected String getPrefix() { + return REPLICATION_PREFIX; + } + + @Override + protected void onDone() { + // might need to do something on index shard here. + } + + @Override + protected void onCancel(String reason) { + // TBD + } + + @Override + protected void onFail(OpenSearchException e, boolean sendShardFailure) { + // TBD + } + + /** + * Wrapper method around {@link #fail(OpenSearchException, boolean)} + * to enforce stronger typing of the input exception instance. + */ + public void fail(ReplicationFailedException e, boolean sendShardFailure) { + super.fail(e, sendShardFailure); + } + + @Override + public ReplicationState state() { + return state; + } + + public long getReplicationId() { + return getId(); } public void startReplication(ActionListener listener) { @@ -123,7 +128,7 @@ public void startReplication(ActionListener listener) { final StepListener finalizeListener = new StepListener<>(); // Get list of files to copy from this checkpoint. - source.getCheckpointInfo(replicationId, checkpoint, checkpointInfoListener); + source.getCheckpointInfo(getId(), checkpoint, checkpointInfoListener); checkpointInfoListener.whenComplete(checkpointInfo -> getFiles(checkpointInfo, getFilesListener), listener::onFailure); getFilesListener.whenComplete( @@ -133,11 +138,6 @@ public void startReplication(ActionListener listener) { finalizeListener.whenComplete(r -> listener.onResponse(new ReplicationResponse()), listener::onFailure); } - public Store store() { - ensureRefCount(); - return store; - } - public void finalizeReplication(TransportCheckpointInfoResponse checkpointInfo, ActionListener listener) { ActionListener.completeWith(listener, () -> { // first, we go and move files that were created with the recovery id suffix to @@ -192,23 +192,6 @@ public void finalizeReplication(TransportCheckpointInfoResponse checkpointInfo, }); } - public long getReplicationId() { - return replicationId; - } - - public IndexShard getIndexShard() { - return indexShard; - } - - @Override - protected void closeInternal() { - store.decRef(); - } - - public ReplicationState state() { - return state; - } - private void getFiles(TransportCheckpointInfoResponse checkpointInfo, StepListener getFilesListener) throws IOException { final Store.MetadataSnapshot snapshot = checkpointInfo.getSnapshot(); @@ -223,7 +206,7 @@ private void getFiles(TransportCheckpointInfoResponse checkpointInfo, StepListen if (filesToFetch.isEmpty()) { getFilesListener.onResponse(new GetFilesResponse()); } - source.getFiles(replicationId, checkpointInfo.getCheckpoint(), filesToFetch, getFilesListener); + source.getFiles(getId(), checkpointInfo.getCheckpoint(), filesToFetch, getFilesListener); } private Store.MetadataSnapshot getMetadataSnapshot() throws IOException { @@ -232,98 +215,4 @@ private Store.MetadataSnapshot getMetadataSnapshot() throws IOException { } return store.getMetadata(indexShard.getLatestSegmentInfos()); } - - public ActionListener markRequestReceivedAndCreateListener(long requestSeqNo, ActionListener listener) { - return requestTracker.markReceivedAndCreateListener(requestSeqNo, listener); - } - - public void writeFileChunk( - StoreFileMetadata fileMetadata, - long position, - BytesReference content, - boolean lastChunk, - ActionListener listener - ) { - try { - multiFileWriter.writeFileChunk(fileMetadata, position, content, lastChunk); - listener.onResponse(null); - } catch (Exception e) { - listener.onFailure(e); - } - } - - /** - * return the last time this RecoveryStatus was used (based on System.nanoTime() - */ - public long lastAccessTime() { - return lastAccessTime; - } - - /** - * sets the lasAccessTime flag to now - */ - public void setLastAccessTime() { - lastAccessTime = System.nanoTime(); - } - - private void ensureRefCount() { - if (refCount() <= 0) { - throw new OpenSearchException( - "RecoveryStatus is used but it's refcount is 0. Probably a mismatch between incRef/decRef " + "calls" - ); - } - } - - /** - * mark the current recovery as done - */ - public void markAsDone() { - if (finished.compareAndSet(false, true)) { - try { - // might need to do something on index shard here. - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - listener.onReplicationDone(state()); - } - } - - /** - * fail the recovery and call listener - * - * @param e exception that encapsulating the failure - * @param sendShardFailure indicates whether to notify the master of the shard failure - */ - public void fail(ReplicationFailedException e, boolean sendShardFailure) { - if (finished.compareAndSet(false, true)) { - try { - listener.onReplicationFailure(state(), e, sendShardFailure); - } finally { - try { - // cancellableThreads.cancel("failed recovery [" + ExceptionsHelper.stackTrace(e) + "]"); - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - } - } - } - - /** - * cancel the recovery. calling this method will clean temporary files and release the store - * unless this object is in use (in which case it will be cleaned once all ongoing users call - * {@link #decRef()} - */ - public void cancel(String reason) { - if (finished.compareAndSet(false, true)) { - try { - logger.debug("recovery canceled (reason: [{}])", reason); - // cancellableThreads.cancel(reason); - } finally { - // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now - decRef(); - } - } - } } From 5cdd077476cb781073e7b7494d2e801e8a5ea29d Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Fri, 11 Mar 2022 16:54:58 -0800 Subject: [PATCH 3/8] Bugfix for segrep integration tests Signed-off-by: Kartik Ganesh --- .../org/opensearch/indices/replication/common/RTarget.java | 6 ++++-- .../indices/replication/copy/ReplicationTarget.java | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java index 2976f8ecbf427..79654214d6a67 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java @@ -41,6 +41,7 @@ public abstract class RTarget extends AbstractRefCounted { protected final RListener listener; protected final MultiFileWriter multiFileWriter; protected final Logger logger; + protected final RecoveryState.Index recoveryStateIndex; protected abstract String getPrefix(); @@ -52,15 +53,16 @@ public abstract class RTarget extends AbstractRefCounted { public abstract RState state(); - public RTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryIndex, RListener listener) { + public RTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, RListener listener) { super(name); this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.listener = listener; this.id = ID_GENERATOR.incrementAndGet(); + this.recoveryStateIndex = recoveryStateIndex; this.indexShard = indexShard; this.store = indexShard.store(); final String tempFilePrefix = getPrefix() + UUIDs.randomBase64UUID() + "."; - this.multiFileWriter = new MultiFileWriter(indexShard.store(), recoveryIndex, tempFilePrefix, logger, this::ensureRefCount); + this.multiFileWriter = new MultiFileWriter(indexShard.store(), recoveryStateIndex, tempFilePrefix, logger, this::ensureRefCount); // make sure the store is not released until we are done. store.incRef(); } diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java index f3bc7093ea11e..536ac2efd4a23 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java @@ -82,7 +82,7 @@ public ReplicationTarget( super("replication_status", indexShard, new RecoveryState.Index(), listener); this.checkpoint = checkpoint; this.source = source; - state = new ReplicationState(new RecoveryState.Index()); + state = new ReplicationState(recoveryStateIndex); } @Override From 97d02fdecdfc8811a6d9cd5a2688bd147bc59cb7 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Fri, 11 Mar 2022 22:30:54 -0800 Subject: [PATCH 4/8] Renaming the temporary RTarget class to ReplicationTarget The existing ReplicationTarget has been renamed to SegmentReplicationTarget. Alongside this change, unnecessary wrapper methods have been removed and replaced with more direct invocations. Signed-off-by: Kartik Ganesh --- .../recovery/PeerRecoveryTargetService.java | 16 +++-- .../recovery/RecoveriesCollection.java | 47 +++++++------ .../indices/recovery/RecoveryTarget.java | 52 +++++--------- .../SegmentReplicationReplicaService.java | 6 +- .../{RTarget.java => ReplicationTarget.java} | 4 +- .../copy/PrimaryShardReplicationSource.java | 4 +- .../copy/ReplicationCollection.java | 70 ++++++++----------- ...get.java => SegmentReplicationTarget.java} | 10 +-- .../recovery/RecoveriesCollectionTests.java | 4 +- 9 files changed, 94 insertions(+), 119 deletions(-) rename server/src/main/java/org/opensearch/indices/replication/common/{RTarget.java => ReplicationTarget.java} (96%) rename server/src/main/java/org/opensearch/indices/replication/copy/{ReplicationTarget.java => SegmentReplicationTarget.java} (97%) diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index f95ff43759cb0..d19069634218d 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -84,6 +84,7 @@ import org.opensearch.transport.TransportService; import java.io.IOException; +import java.nio.file.Path; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; @@ -203,7 +204,7 @@ protected void retryRecovery(final long recoveryId, final String reason, TimeVal private void retryRecovery(final long recoveryId, final TimeValue retryAfter, final TimeValue activityTimeout) { RecoveryTarget newTarget = onGoingRecoveries.resetRecovery(recoveryId, activityTimeout); if (newTarget != null) { - threadPool.scheduleUnlessShuttingDown(retryAfter, ThreadPool.Names.GENERIC, new RecoveryRunner(newTarget.recoveryId())); + threadPool.scheduleUnlessShuttingDown(retryAfter, ThreadPool.Names.GENERIC, new RecoveryRunner(newTarget.getId())); } } @@ -232,7 +233,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi final IndexShard indexShard = recoveryTarget.indexShard(); indexShard.preRecovery(); assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node"; - logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); + logger.trace("{} preparing shard for peer recovery", recoveryTarget.indexShard().shardId()); indexShard.prepareForIndexRecovery(); final long startingSeqNo = indexShard.recoverLocallyUpToGlobalCheckpoint(); assert startingSeqNo == UNASSIGNED_SEQ_NO || recoveryTarget.state().getStage() == RecoveryState.Stage.TRANSLOG @@ -292,7 +293,7 @@ public static StartRecoveryRequest getStartRecoveryRequest( long startingSeqNo ) { final StartRecoveryRequest request; - logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + logger.trace("{} collecting local files for [{}]", recoveryTarget.indexShard().shardId(), recoveryTarget.sourceNode()); Store.MetadataSnapshot metadataSnapshot; try { @@ -300,7 +301,8 @@ public static StartRecoveryRequest getStartRecoveryRequest( // Make sure that the current translog is consistent with the Lucene index; otherwise, we have to throw away the Lucene index. try { final String expectedTranslogUUID = metadataSnapshot.getCommitUserData().get(Translog.TRANSLOG_UUID_KEY); - final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation(), expectedTranslogUUID); + Path translogPath = recoveryTarget.indexShard().shardPath().resolveTranslog(); + final long globalCheckpoint = Translog.readGlobalCheckpoint(translogPath, expectedTranslogUUID); assert globalCheckpoint + 1 >= startingSeqNo : "invalid startingSeqNo " + startingSeqNo + " >= " + globalCheckpoint; } catch (IOException | TranslogCorruptedException e) { logger.warn( @@ -335,15 +337,15 @@ public static StartRecoveryRequest getStartRecoveryRequest( } metadataSnapshot = Store.MetadataSnapshot.EMPTY; } - logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size()); + logger.trace("{} local file count [{}]", recoveryTarget.indexShard().shardId(), metadataSnapshot.size()); request = new StartRecoveryRequest( - recoveryTarget.shardId(), + recoveryTarget.indexShard().shardId(), recoveryTarget.indexShard().routingEntry().allocationId().getId(), recoveryTarget.sourceNode(), localNode, metadataSnapshot, recoveryTarget.state().getPrimary(), - recoveryTarget.recoveryId(), + recoveryTarget.getId(), startingSeqNo ); return request; diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java index 8779e805a8476..569970471611d 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java @@ -82,20 +82,20 @@ public long startRecovery( ) { RecoveryTarget recoveryTarget = new RecoveryTarget(indexShard, sourceNode, listener); startRecoveryInternal(recoveryTarget, activityTimeout); - return recoveryTarget.recoveryId(); + return recoveryTarget.getId(); } private void startRecoveryInternal(RecoveryTarget recoveryTarget, TimeValue activityTimeout) { - RecoveryTarget existingTarget = onGoingRecoveries.putIfAbsent(recoveryTarget.recoveryId(), recoveryTarget); + RecoveryTarget existingTarget = onGoingRecoveries.putIfAbsent(recoveryTarget.getId(), recoveryTarget); assert existingTarget == null : "found two RecoveryStatus instances with the same id"; logger.trace( "{} started recovery from {}, id [{}]", - recoveryTarget.shardId(), + recoveryTarget.indexShard().shardId(), recoveryTarget.sourceNode(), - recoveryTarget.recoveryId() + recoveryTarget.getId() ); threadPool.schedule( - new RecoveryMonitor(recoveryTarget.recoveryId(), recoveryTarget.lastAccessTime(), activityTimeout), + new RecoveryMonitor(recoveryTarget.getId(), recoveryTarget.lastAccessTime(), activityTimeout), activityTimeout, ThreadPool.Names.GENERIC ); @@ -134,21 +134,21 @@ public RecoveryTarget resetRecovery(final long recoveryId, final TimeValue activ if (successfulReset) { logger.trace( "{} restarted recovery from {}, id [{}], previous id [{}]", - newRecoveryTarget.shardId(), + newRecoveryTarget.indexShard().shardId(), newRecoveryTarget.sourceNode(), - newRecoveryTarget.recoveryId(), - oldRecoveryTarget.recoveryId() + newRecoveryTarget.getId(), + oldRecoveryTarget.getId() ); return newRecoveryTarget; } else { logger.trace( "{} recovery could not be reset as it is already cancelled, recovery from {}, id [{}], previous id [{}]", - newRecoveryTarget.shardId(), + newRecoveryTarget.indexShard().shardId(), newRecoveryTarget.sourceNode(), - newRecoveryTarget.recoveryId(), - oldRecoveryTarget.recoveryId() + newRecoveryTarget.getId(), + oldRecoveryTarget.getId() ); - cancelRecovery(newRecoveryTarget.recoveryId(), "recovery cancelled during reset"); + cancelRecovery(newRecoveryTarget.getId(), "recovery cancelled during reset"); return null; } } catch (Exception e) { @@ -184,7 +184,7 @@ public RecoveryRef getRecoverySafe(long id, ShardId shardId) { throw new IndexShardClosedException(shardId); } RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); - assert recoveryTarget.shardId().equals(shardId); + assert recoveryTarget.indexShard().shardId().equals(shardId); return recoveryRef; } @@ -195,9 +195,9 @@ public boolean cancelRecovery(long id, String reason) { if (removed != null) { logger.trace( "{} canceled recovery from {}, id [{}] (reason [{}])", - removed.shardId(), + removed.indexShard().shardId(), removed.sourceNode(), - removed.recoveryId(), + removed.getId(), reason ); removed.cancel(reason); @@ -218,9 +218,9 @@ public void failRecovery(long id, RecoveryFailedException e, boolean sendShardFa if (removed != null) { logger.trace( "{} failing recovery from {}, id [{}]. Send shard failure: [{}]", - removed.shardId(), + removed.indexShard().shardId(), removed.sourceNode(), - removed.recoveryId(), + removed.getId(), sendShardFailure ); removed.fail(e, sendShardFailure); @@ -231,7 +231,12 @@ public void failRecovery(long id, RecoveryFailedException e, boolean sendShardFa public void markRecoveryAsDone(long id) { RecoveryTarget removed = onGoingRecoveries.remove(id); if (removed != null) { - logger.trace("{} marking recovery from {} as done, id [{}]", removed.shardId(), removed.sourceNode(), removed.recoveryId()); + logger.trace( + "{} marking recovery from {} as done, id [{}]", + removed.indexShard().shardId(), + removed.sourceNode(), + removed.getId() + ); removed.markAsDone(); } } @@ -254,7 +259,7 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { synchronized (onGoingRecoveries) { for (Iterator it = onGoingRecoveries.values().iterator(); it.hasNext();) { RecoveryTarget status = it.next(); - if (status.shardId().equals(shardId)) { + if (status.indexShard().shardId().equals(shardId)) { matchedRecoveries.add(status); it.remove(); } @@ -263,9 +268,9 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { for (RecoveryTarget removed : matchedRecoveries) { logger.trace( "{} canceled recovery from {}, id [{}] (reason [{}])", - removed.shardId(), + removed.indexShard().shardId(), removed.sourceNode(), - removed.recoveryId(), + removed.getId(), reason ); removed.cancel(reason); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java index 070825a196dec..79a9f5da8a69f 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java @@ -52,14 +52,12 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardNotRecoveringException; import org.opensearch.index.shard.IndexShardState; -import org.opensearch.index.shard.ShardId; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.index.translog.Translog; -import org.opensearch.indices.replication.common.RTarget; +import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; -import java.nio.file.Path; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -67,7 +65,7 @@ * Represents a recovery where the current node is the target node of the recovery. To track recoveries in a central place, instances of * this class are created through {@link RecoveriesCollection}. */ -public class RecoveryTarget extends RTarget implements RecoveryTargetHandler { +public class RecoveryTarget extends ReplicationTarget implements RecoveryTargetHandler { private static final String RECOVERY_PREFIX = "recovery."; @@ -90,6 +88,11 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecov indexShard.recoveryStats().incCurrentAsTarget(); } + @Override + public String toString() { + return indexShard.shardId() + " [" + getId() + "]"; + } + @Override protected String getPrefix() { return RECOVERY_PREFIX; @@ -132,18 +135,14 @@ protected void closeInternal() { } } + /** + * Wrapper method around the internal {@link #listener} object + * to enforce stronger typing on the return type. + */ public PeerRecoveryTargetService.RecoveryListener getListener() { return (PeerRecoveryTargetService.RecoveryListener) listener; } - public long recoveryId() { - return getId(); - } - - public ShardId shardId() { - return indexShard.shardId(); - } - public DiscoveryNode sourceNode() { return this.sourceNode; } @@ -160,7 +159,7 @@ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOE final long recoveryId = getId(); if (finished.compareAndSet(false, true)) { try { - logger.debug("reset of recovery with shard {} and id [{}]", shardId(), recoveryId); + logger.debug("reset of recovery with shard {} and id [{}]", indexShard.shardId(), recoveryId); } finally { // release the initial reference. recovery files will be cleaned as soon as ref count goes to zero, potentially now. decRef(); @@ -170,7 +169,7 @@ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOE } catch (CancellableThreads.ExecutionCancelledException e) { logger.trace( "new recovery target cancelled for shard {} while waiting on old recovery target with id [{}] to close", - shardId(), + indexShard.shardId(), recoveryId ); return false; @@ -194,11 +193,6 @@ public void notifyListener(RecoveryFailedException e, boolean sendShardFailure) listener.onFailure(state(), e, sendShardFailure); } - @Override - public String toString() { - return shardId() + " [" + getId() + "]"; - } - /*** Implementation of {@link RecoveryTargetHandler } */ @Override @@ -266,7 +260,7 @@ public void indexTranslogOperations( translog.totalOperations(totalTranslogOps); assert indexShard().recoveryState() == state(); if (indexShard().state() != IndexShardState.RECOVERING) { - throw new IndexShardNotRecoveringException(shardId(), indexShard().state()); + throw new IndexShardNotRecoveringException(indexShard.shardId(), indexShard().state()); } /* * The maxSeenAutoIdTimestampOnPrimary received from the primary is at least the highest auto_id_timestamp from any operation @@ -352,7 +346,7 @@ public void cleanFiles( final String translogUUID = Translog.createEmptyTranslog( indexShard.shardPath().resolveTranslog(), globalCheckpoint, - shardId(), + indexShard.shardId(), indexShard.getPendingPrimaryTerm() ); store.associateIndexWithNewTranslog(translogUUID); @@ -407,21 +401,13 @@ public void writeFileChunk( ) { state().getTranslog().totalOperations(totalTranslogOps); this.writeFileChunk(fileMetadata, position, content, lastChunk, listener); - try { - state().getTranslog().totalOperations(totalTranslogOps); - multiFileWriter.writeFileChunk(fileMetadata, position, content, lastChunk); - listener.onResponse(null); - } catch (Exception e) { - listener.onFailure(e); - } } - /** Get a temporary name for the provided file name. */ + /** + * Get a temporary name for the provided file name. + * This is only used in tests. + */ public String getTempNameForFile(String origFile) { return multiFileWriter.getTempNameForFile(origFile); } - - Path translogLocation() { - return indexShard().shardPath().resolveTranslog(); - } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index 0bf56f1f8474c..7483f07165bb0 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -43,7 +43,7 @@ import org.opensearch.indices.replication.copy.ReplicationCollection; import org.opensearch.indices.replication.copy.ReplicationFailedException; import org.opensearch.indices.replication.copy.ReplicationState; -import org.opensearch.indices.replication.copy.ReplicationTarget; +import org.opensearch.indices.replication.copy.SegmentReplicationTarget; import org.opensearch.indices.replication.copy.SegmentReplicationPrimaryService; import org.opensearch.indices.replication.copy.TrackShardRequest; import org.opensearch.indices.replication.copy.TrackShardResponse; @@ -181,7 +181,7 @@ private void doReplication(final long replicationId) { logger.trace("not running replication with id [{}] - can not find it (probably finished)", replicationId); return; } - final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); timer = replicationTarget.state().getTimer(); final IndexShard indexShard = replicationTarget.indexShard(); @@ -218,7 +218,7 @@ public void onFailure(Exception e) { () -> new ParameterizedMessage("unexpected error during replication [{}], failing shard", replicationId), e ); - ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); onGoingReplications.failReplication( replicationId, new ReplicationFailedException(replicationTarget.indexShard(), "unexpected error", e), diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java similarity index 96% rename from server/src/main/java/org/opensearch/indices/replication/common/RTarget.java rename to server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index 79654214d6a67..3e18239ea0aaf 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/RTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -25,7 +25,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -public abstract class RTarget extends AbstractRefCounted { +public abstract class ReplicationTarget extends AbstractRefCounted { // TODO will this cause issues because its shared between subclasses? private static final AtomicLong ID_GENERATOR = new AtomicLong(); @@ -53,7 +53,7 @@ public abstract class RTarget extends AbstractRefCounted { public abstract RState state(); - public RTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, RListener listener) { + public ReplicationTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, RListener listener) { super(name); this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.listener = listener; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java index e934ac2ffc427..d51cfad35145b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java @@ -231,7 +231,7 @@ public void messageReceived(final ReplicationFileChunkRequest request, Transport ReplicationCollection.ReplicationRef replicationRef = segmentReplicationReplicaService.getOnGoingReplications() .getReplicationSafe(request.getReplicationId(), request.shardId()) ) { - final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); final ActionListener listener = createOrFinishListener(replicationRef, channel, Actions.FILE_CHUNK, request); if (listener == null) { return; @@ -274,7 +274,7 @@ private ActionListener createOrFinishListener( final ReplicationFileChunkRequest request, final CheckedFunction responseFn ) { - final ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); final ActionListener voidListener = ActionListener.map(channelListener, responseFn); diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java index 9286de0b80996..d24d077843f6b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -61,7 +61,7 @@ public class ReplicationCollection { /** * This is the single source of truth for ongoing recoveries. If it's not here, it was canceled or done */ - private final ConcurrentMap onGoingReplications = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap onGoingReplications = ConcurrentCollections.newConcurrentMap(); private final Logger logger; private final ThreadPool threadPool; @@ -78,39 +78,35 @@ public long startReplication( SegmentReplicationReplicaService.ReplicationListener listener, TimeValue activityTimeout ) { - ReplicationTarget replicationTarget = new ReplicationTarget(checkpoint, indexShard, source, listener); + SegmentReplicationTarget replicationTarget = new SegmentReplicationTarget(checkpoint, indexShard, source, listener); startReplicationInternal(replicationTarget, activityTimeout); - return replicationTarget.getReplicationId(); + return replicationTarget.getId(); } - private void startReplicationInternal(ReplicationTarget replicationTarget, TimeValue activityTimeout) { - ReplicationTarget existingTarget = onGoingReplications.putIfAbsent(replicationTarget.getReplicationId(), replicationTarget); + private void startReplicationInternal(SegmentReplicationTarget replicationTarget, TimeValue activityTimeout) { + SegmentReplicationTarget existingTarget = onGoingReplications.putIfAbsent(replicationTarget.getId(), replicationTarget); assert existingTarget == null : "found two ReplicationStatus instances with the same id"; - logger.trace( - "{} started segment replication id [{}]", - replicationTarget.indexShard().shardId(), - replicationTarget.getReplicationId() - ); + logger.trace("{} started segment replication id [{}]", replicationTarget.indexShard().shardId(), replicationTarget.getId()); threadPool.schedule( - new ReplicationMonitor(replicationTarget.getReplicationId(), replicationTarget.lastAccessTime(), activityTimeout), + new ReplicationMonitor(replicationTarget.getId(), replicationTarget.lastAccessTime(), activityTimeout), activityTimeout, ThreadPool.Names.GENERIC ); } - public ReplicationTarget getReplicationTarget(long id) { + public SegmentReplicationTarget getReplicationTarget(long id) { return onGoingReplications.get(id); } /** - * gets the {@link ReplicationTarget } for a given id. The ReplicationStatus returned has it's ref count already incremented - * to make sure it's safe to use. However, you must call {@link ReplicationTarget#decRef()} when you are done with it, typically + * gets the {@link SegmentReplicationTarget } for a given id. The ReplicationStatus returned has it's ref count already incremented + * to make sure it's safe to use. However, you must call {@link SegmentReplicationTarget#decRef()} when you are done with it, typically * by using this method in a try-with-resources clause. *

* Returns null if replication is not found */ public ReplicationRef getReplication(long id) { - ReplicationTarget status = onGoingReplications.get(id); + SegmentReplicationTarget status = onGoingReplications.get(id); if (status != null && status.tryIncRef()) { return new ReplicationRef(status); } @@ -125,22 +121,17 @@ public ReplicationRef getReplicationSafe(long id, ShardId shardId) { if (replicationRef == null) { throw new IndexShardClosedException(shardId); } - ReplicationTarget replicationTarget = replicationRef.get(ReplicationTarget.class); + SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); assert replicationTarget.indexShard().shardId().equals(shardId); return replicationRef; } /** cancel the replication with the given id (if found) and remove it from the replication collection */ public boolean cancelReplication(long id, String reason) { - ReplicationTarget removed = onGoingReplications.remove(id); + SegmentReplicationTarget removed = onGoingReplications.remove(id); boolean cancelled = false; if (removed != null) { - logger.trace( - "{} canceled replication, id [{}] (reason [{}])", - removed.indexShard().shardId(), - removed.getReplicationId(), - reason - ); + logger.trace("{} canceled replication, id [{}] (reason [{}])", removed.indexShard().shardId(), removed.getId(), reason); removed.cancel(reason); cancelled = true; } @@ -155,12 +146,12 @@ public boolean cancelReplication(long id, String reason) { * @param sendShardFailure true a shard failed message should be sent to the master */ public void failReplication(long id, ReplicationFailedException e, boolean sendShardFailure) { - ReplicationTarget removed = onGoingReplications.remove(id); + SegmentReplicationTarget removed = onGoingReplications.remove(id); if (removed != null) { logger.trace( "{} failing segment replication id [{}]. Send shard failure: [{}]", removed.indexShard().shardId(), - removed.getReplicationId(), + removed.getId(), sendShardFailure ); removed.fail(e, sendShardFailure); @@ -171,9 +162,9 @@ public void failReplication(long id, ReplicationFailedException e, boolean sendS * mark the replication with the given id as done (if found) */ public void markReplicationAsDone(long id) { - ReplicationTarget removed = onGoingReplications.remove(id); + SegmentReplicationTarget removed = onGoingReplications.remove(id); if (removed != null) { - logger.trace("{} marking replication as done, id [{}]", removed.indexShard().shardId(), removed.getReplicationId()); + logger.trace("{} marking replication as done, id [{}]", removed.indexShard().shardId(), removed.getId()); removed.markAsDone(); } } @@ -194,23 +185,18 @@ public int size() { */ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { boolean cancelled = false; - List matchedRecoveries = new ArrayList<>(); + List matchedRecoveries = new ArrayList<>(); synchronized (onGoingReplications) { - for (Iterator it = onGoingReplications.values().iterator(); it.hasNext();) { - ReplicationTarget status = it.next(); + for (Iterator it = onGoingReplications.values().iterator(); it.hasNext();) { + SegmentReplicationTarget status = it.next(); if (status.indexShard().shardId().equals(shardId)) { matchedRecoveries.add(status); it.remove(); } } } - for (ReplicationTarget removed : matchedRecoveries) { - logger.trace( - "{} canceled segment replication id [{}] (reason [{}])", - removed.indexShard().shardId(), - removed.getReplicationId(), - reason - ); + for (SegmentReplicationTarget removed : matchedRecoveries) { + logger.trace("{} canceled segment replication id [{}] (reason [{}])", removed.indexShard().shardId(), removed.getId(), reason); removed.cancel(reason); cancelled = true; } @@ -218,17 +204,17 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { } /** - * a reference to {@link ReplicationTarget}, which implements {@link AutoCloseable}. closing the reference - * causes {@link ReplicationTarget#decRef()} to be called. This makes sure that the underlying resources + * a reference to {@link SegmentReplicationTarget}, which implements {@link AutoCloseable}. closing the reference + * causes {@link SegmentReplicationTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link ReplicationRef#close()} is called. */ public static class ReplicationRef extends AutoCloseableRefCounted { /** - * Important: {@link ReplicationTarget#tryIncRef()} should + * Important: {@link SegmentReplicationTarget#tryIncRef()} should * be *successfully* called on target before */ - public ReplicationRef(ReplicationTarget target) { + public ReplicationRef(SegmentReplicationTarget target) { super(target); target.setLastAccessTime(); } @@ -253,7 +239,7 @@ public void onFailure(Exception e) { @Override protected void doRun() throws Exception { - ReplicationTarget replicationTarget = onGoingReplications.get(replicationId); + SegmentReplicationTarget replicationTarget = onGoingReplications.get(replicationId); if (replicationTarget == null) { logger.trace("[monitor] no replicationTarget found for [{}], shutting down", replicationId); return; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java similarity index 97% rename from server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java rename to server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java index 536ac2efd4a23..5fbc8fd8f82bd 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java @@ -47,7 +47,7 @@ import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.SegmentReplicationReplicaService; import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; -import org.opensearch.indices.replication.common.RTarget; +import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; import java.util.Arrays; @@ -58,7 +58,7 @@ /** * Orchestrates a replication event for a replica shard. */ -public class ReplicationTarget extends RTarget { +public class SegmentReplicationTarget extends ReplicationTarget { private static final String REPLICATION_PREFIX = "replication."; @@ -73,7 +73,7 @@ public class ReplicationTarget extends RTarget { * @param source source of the recovery where we recover from * @param listener called when recovery is completed/failed */ - public ReplicationTarget( + public SegmentReplicationTarget( ReplicationCheckpoint checkpoint, IndexShard indexShard, PrimaryShardReplicationSource source, @@ -118,10 +118,6 @@ public ReplicationState state() { return state; } - public long getReplicationId() { - return getId(); - } - public void startReplication(ActionListener listener) { final StepListener checkpointInfoListener = new StepListener<>(); final StepListener getFilesListener = new StepListener<>(); diff --git a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java index 66505826a8ea7..e6be95a1eba58 100644 --- a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java @@ -124,7 +124,7 @@ public void testRecoveryCancellation() throws Exception { final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); final long recoveryId2 = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); try (RecoveriesCollection.RecoveryRef recoveryRef = collection.getRecovery(recoveryId)) { - ShardId shardId = recoveryRef.get(RecoveryTarget.class).shardId(); + ShardId shardId = recoveryRef.get(RecoveryTarget.class).indexShard().shardId(); assertTrue("failed to cancel recoveries", collection.cancelRecoveriesForShard(shardId, "test")); assertThat("all recoveries should be cancelled", collection.size(), equalTo(0)); } finally { @@ -149,7 +149,7 @@ public void testResetRecovery() throws Exception { Store store = recoveryTarget.store(); String tempFileName = recoveryTarget.getTempNameForFile("foobar"); RecoveryTarget resetRecovery = collection.resetRecovery(recoveryId, TimeValue.timeValueMinutes(60)); - final long resetRecoveryId = resetRecovery.recoveryId(); + final long resetRecoveryId = resetRecovery.getId(); assertNotSame(recoveryTarget, resetRecovery); assertNotSame(recoveryTarget.cancellableThreads(), resetRecovery.cancellableThreads()); assertSame(indexShard, resetRecovery.indexShard()); From 7f2012871caa81a8a821855adde3e0aea6a02354 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Mon, 14 Mar 2022 10:25:53 -0700 Subject: [PATCH 5/8] Renaming replication listener classes The ReplicationListener class has been renamed to SegmentReplicationListener, and the parent class now takes the more-generic ReplicationListener name. The inner classes in IndicesClusterStateService have also been renamed to ShardRouting* to reflect their purpose. Signed-off-by: Kartik Ganesh --- .../org/opensearch/index/shard/IndexShard.java | 8 ++++---- .../org/opensearch/indices/IndicesService.java | 4 ++-- .../cluster/IndicesClusterStateService.java | 14 +++++++------- .../recovery/PeerRecoveryTargetService.java | 4 ++-- .../SegmentReplicationReplicaService.java | 6 +++--- .../{RListener.java => ReplicationListener.java} | 2 +- .../replication/common/ReplicationTarget.java | 4 ++-- .../replication/copy/ReplicationCollection.java | 2 +- .../replication/copy/SegmentReplicationTarget.java | 2 +- ...AbstractIndicesClusterStateServiceTestCase.java | 2 +- 10 files changed, 24 insertions(+), 24 deletions(-) rename server/src/main/java/org/opensearch/indices/replication/common/{RListener.java => ReplicationListener.java} (91%) 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 7720103085669..ed2558dee2fef 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -3040,7 +3040,7 @@ protected Engine getEngineOrNull() { public void startRecovery( RecoveryState recoveryState, SegmentReplicationReplicaService segmentReplicationReplicaService, - SegmentReplicationReplicaService.ReplicationListener replicationListener, + SegmentReplicationReplicaService.SegmentReplicationListener segRepListener, PrimaryShardReplicationSource replicationSource, PeerRecoveryTargetService peerRecoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, @@ -3082,7 +3082,7 @@ public void startRecovery( new ActionListener() { @Override public void onResponse(TrackShardResponse unused) { - replicationListener.onReplicationDone(replicationState); + segRepListener.onReplicationDone(replicationState); recoveryState.getIndex().setFileDetailsComplete(); finalizeRecovery(); postRecovery("Shard setup complete."); @@ -3090,7 +3090,7 @@ public void onResponse(TrackShardResponse unused) { @Override public void onFailure(Exception e) { - replicationListener.onReplicationFailure( + segRepListener.onReplicationFailure( replicationState, new ReplicationFailedException(indexShard, e), true @@ -3748,7 +3748,7 @@ public synchronized void onNewCheckpoint( checkpoint, this, source, - new SegmentReplicationReplicaService.ReplicationListener() { + new SegmentReplicationReplicaService.SegmentReplicationListener() { @Override public void onReplicationDone(ReplicationState state) { markReplicationComplete(); diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 2fb399b84153a..6f05de5aa3609 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -838,7 +838,7 @@ public synchronized void verifyIndexMetadata(IndexMetadata metadata, IndexMetada public IndexShard createShard( final ShardRouting shardRouting, final SegmentReplicationReplicaService segmentReplicationReplicaService, - final SegmentReplicationReplicaService.ReplicationListener replicationListener, + final SegmentReplicationReplicaService.SegmentReplicationListener segRepListener, final PrimaryShardReplicationSource replicationSource, final PeerRecoveryTargetService recoveryTargetService, final PeerRecoveryTargetService.RecoveryListener recoveryListener, @@ -859,7 +859,7 @@ public IndexShard createShard( indexShard.startRecovery( recoveryState, segmentReplicationReplicaService, - replicationListener, + segRepListener, replicationSource, recoveryTargetService, recoveryListener, diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index b28488b9f554d..88c60010ddfdb 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -630,10 +630,10 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR indicesService.createShard( shardRouting, segmentReplicationReplicaService, - new ReplicationListener(shardRouting, primaryTerm), + new ShardRoutingReplicationListener(shardRouting, primaryTerm), replicationSource, recoveryTargetService, - new RecoveryListener(shardRouting, primaryTerm), + new ShardRoutingRecoveryListener(shardRouting, primaryTerm), repositoriesService, failedShardHandler, globalCheckpointSyncer, @@ -748,7 +748,7 @@ private static DiscoveryNode findSourceNodeForPeerRecovery( return sourceNode; } - private class ReplicationListener implements SegmentReplicationReplicaService.ReplicationListener { + private class ShardRoutingReplicationListener implements SegmentReplicationReplicaService.SegmentReplicationListener { /** * ShardRouting with which the shard was created @@ -760,7 +760,7 @@ private class ReplicationListener implements SegmentReplicationReplicaService.Re */ private final long primaryTerm; - private ReplicationListener(final ShardRouting shardRouting, final long primaryTerm) { + private ShardRoutingReplicationListener(final ShardRouting shardRouting, final long primaryTerm) { this.shardRouting = shardRouting; this.primaryTerm = primaryTerm; } @@ -778,7 +778,7 @@ public void onReplicationFailure(ReplicationState state, ReplicationFailedExcept } } - private class RecoveryListener implements PeerRecoveryTargetService.RecoveryListener { + private class ShardRoutingRecoveryListener implements PeerRecoveryTargetService.RecoveryListener { /** * ShardRouting with which the shard was created @@ -790,7 +790,7 @@ private class RecoveryListener implements PeerRecoveryTargetService.RecoveryList */ private final long primaryTerm; - private RecoveryListener(final ShardRouting shardRouting, final long primaryTerm) { + private ShardRoutingRecoveryListener(final ShardRouting shardRouting, final long primaryTerm) { this.shardRouting = shardRouting; this.primaryTerm = primaryTerm; } @@ -1028,7 +1028,7 @@ U createIndex(IndexMetadata indexMetadata, List builtInIndex T createShard( ShardRouting shardRouting, SegmentReplicationReplicaService replicaService, - SegmentReplicationReplicaService.ReplicationListener replicationListener, + SegmentReplicationReplicaService.SegmentReplicationListener segRepListener, PrimaryShardReplicationSource replicationSource, PeerRecoveryTargetService recoveryTargetService, PeerRecoveryTargetService.RecoveryListener recoveryListener, diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index d19069634218d..59246d03beda5 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -70,7 +70,7 @@ import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogCorruptedException; import org.opensearch.indices.recovery.RecoveriesCollection.RecoveryRef; -import org.opensearch.indices.replication.common.RListener; +import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.RState; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; @@ -351,7 +351,7 @@ public static StartRecoveryRequest getStartRecoveryRequest( return request; } - public interface RecoveryListener extends RListener { + public interface RecoveryListener extends ReplicationListener { @Override default void onDone(RState state) { diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index 7483f07165bb0..aa8df620f1412 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -36,7 +36,7 @@ import org.opensearch.indices.recovery.DelayRecoveryException; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.Timer; -import org.opensearch.indices.replication.common.RListener; +import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.RState; import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.indices.replication.copy.ReplicationCheckpoint; @@ -161,7 +161,7 @@ public void startReplication( final ReplicationCheckpoint checkpoint, final IndexShard indexShard, PrimaryShardReplicationSource source, - final ReplicationListener listener + final SegmentReplicationListener listener ) { final long replicationId = onGoingReplications.startReplication( checkpoint, @@ -288,7 +288,7 @@ public void onFailure(Exception e) { } } - public interface ReplicationListener extends RListener { + public interface SegmentReplicationListener extends ReplicationListener { @Override default void onDone(RState state) { diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RListener.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java similarity index 91% rename from server/src/main/java/org/opensearch/indices/replication/common/RListener.java rename to server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java index 44d813d952bfb..4c5d20580d113 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/RListener.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java @@ -10,7 +10,7 @@ import org.opensearch.OpenSearchException; -public interface RListener { +public interface ReplicationListener { void onDone(RState state); diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index 3e18239ea0aaf..9a4b26300d16c 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -38,7 +38,7 @@ public abstract class ReplicationTarget extends AbstractRefCounted { protected final AtomicBoolean finished = new AtomicBoolean(); protected final IndexShard indexShard; protected final Store store; - protected final RListener listener; + protected final ReplicationListener listener; protected final MultiFileWriter multiFileWriter; protected final Logger logger; protected final RecoveryState.Index recoveryStateIndex; @@ -53,7 +53,7 @@ public abstract class ReplicationTarget extends AbstractRefCounted { public abstract RState state(); - public ReplicationTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, RListener listener) { + public ReplicationTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, ReplicationListener listener) { super(name); this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.listener = listener; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java index d24d077843f6b..76094cd1672af 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -75,7 +75,7 @@ public long startReplication( ReplicationCheckpoint checkpoint, IndexShard indexShard, PrimaryShardReplicationSource source, - SegmentReplicationReplicaService.ReplicationListener listener, + SegmentReplicationReplicaService.SegmentReplicationListener listener, TimeValue activityTimeout ) { SegmentReplicationTarget replicationTarget = new SegmentReplicationTarget(checkpoint, indexShard, source, listener); diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java index 5fbc8fd8f82bd..613125b3aee1a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java @@ -77,7 +77,7 @@ public SegmentReplicationTarget( ReplicationCheckpoint checkpoint, IndexShard indexShard, PrimaryShardReplicationSource source, - SegmentReplicationReplicaService.ReplicationListener listener + SegmentReplicationReplicaService.SegmentReplicationListener listener ) { super("replication_status", indexShard, new RecoveryState.Index(), listener); this.checkpoint = checkpoint; diff --git a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 2c88b99e4ecea..278c8c3408adc 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -255,7 +255,7 @@ public MockIndexService indexService(Index index) { public MockIndexShard createShard( final ShardRouting shardRouting, final SegmentReplicationReplicaService replicaService, - final SegmentReplicationReplicaService.ReplicationListener replicationListener, + final SegmentReplicationReplicaService.SegmentReplicationListener segRepListener, final PrimaryShardReplicationSource replicationSource, final PeerRecoveryTargetService recoveryTargetService, final PeerRecoveryTargetService.RecoveryListener recoveryListener, From a5b519999ce1311d003c76c4754fec9a0e00f3d9 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Mon, 14 Mar 2022 11:44:36 -0700 Subject: [PATCH 6/8] Moved RecoveryState.Index from an inner class to a top-level class This class is currently across both recovery and segment replication. It has two dependent inner classes - RecoveryFilesDetails and FileDetail - which have been moved from RecoveryState to be inner classes in RecoveryIndex Signed-off-by: Kartik Ganesh --- .../gateway/RecoveryFromGatewayIT.java | 5 +- .../indices/recovery/IndexRecoveryIT.java | 2 +- .../opensearch/index/shard/StoreRecovery.java | 13 +- .../indices/recovery/MultiFileWriter.java | 4 +- .../recovery/PeerRecoveryTargetService.java | 4 +- .../indices/recovery/RecoveryIndex.java | 509 ++++++++++++++++++ .../indices/recovery/RecoveryState.java | 483 +---------------- .../indices/recovery/RecoveryTarget.java | 2 +- .../replication/common/ReplicationTarget.java | 6 +- .../replication/copy/ReplicationState.java | 35 +- .../copy/SegmentReplicationTarget.java | 3 +- .../index/shard/IndexShardTests.java | 3 +- .../index/shard/StoreRecoveryTests.java | 8 +- .../recovery/RecoverySourceHandlerTests.java | 4 +- .../indices/recovery/RecoveryTargetTests.java | 37 +- .../repositories/fs/FsRepositoryTests.java | 5 +- .../action/cat/RestRecoveryActionTests.java | 3 +- 17 files changed, 578 insertions(+), 548 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/recovery/RecoveryIndex.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 612abee7dbf5b..4c791d1fb3a2d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -62,6 +62,7 @@ import org.opensearch.index.shard.ShardId; import org.opensearch.index.shard.ShardPath; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; @@ -555,7 +556,7 @@ public void testReuseInFileBasedPeerRecovery() throws Exception { final Set files = new HashSet<>(); for (final RecoveryState recoveryState : initialRecoveryReponse.shardRecoveryStates().get("test")) { if (recoveryState.getTargetNode().getName().equals(replicaNode)) { - for (final RecoveryState.FileDetail file : recoveryState.getIndex().fileDetails()) { + for (final RecoveryIndex.FileDetail file : recoveryState.getIndex().fileDetails()) { files.add(file.name()); } break; @@ -615,7 +616,7 @@ public Settings onNodeStopped(String nodeName) throws Exception { long reused = 0; int filesRecovered = 0; int filesReused = 0; - for (final RecoveryState.FileDetail file : recoveryState.getIndex().fileDetails()) { + for (final RecoveryIndex.FileDetail file : recoveryState.getIndex().fileDetails()) { if (files.contains(file.name()) == false) { recovered += file.length(); filesRecovered++; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java index a7dc77e024d5c..c8ff3ad7dc8c6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java @@ -836,7 +836,7 @@ private IndicesStatsResponse createAndPopulateIndex(String name, int nodeCount, return client().admin().indices().prepareStats(name).execute().actionGet(); } - private void validateIndexRecoveryState(RecoveryState.Index indexState) { + private void validateIndexRecoveryState(RecoveryIndex indexState) { assertThat(indexState.time(), greaterThanOrEqualTo(0L)); assertThat(indexState.recoveredFilesPercent(), greaterThanOrEqualTo(0.0f)); assertThat(indexState.recoveredFilesPercent(), lessThanOrEqualTo(100.0f)); diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 6cf6ad645ca00..97b40a5e746c5 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -62,6 +62,7 @@ import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.Repository; @@ -177,7 +178,7 @@ void recoverFromLocalShards( } void addIndices( - final RecoveryState.Index indexRecoveryStats, + final RecoveryIndex indexRecoveryStats, final Directory target, final Sort indexSort, final Directory[] sources, @@ -232,9 +233,9 @@ void addIndices( * Directory wrapper that records copy process for recovery statistics */ static final class StatsDirectoryWrapper extends FilterDirectory { - private final RecoveryState.Index index; + private final RecoveryIndex index; - StatsDirectoryWrapper(Directory in, RecoveryState.Index indexRecoveryStats) { + StatsDirectoryWrapper(Directory in, RecoveryIndex indexRecoveryStats) { super(in); this.index = indexRecoveryStats; } @@ -355,7 +356,7 @@ private ActionListener recoveryListener(IndexShard indexShard, ActionLi + "]"; if (logger.isTraceEnabled()) { - RecoveryState.Index index = recoveryState.getIndex(); + RecoveryIndex index = recoveryState.getIndex(); StringBuilder sb = new StringBuilder(); sb.append(" index : files [") .append(index.totalFileCount()) @@ -472,7 +473,7 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe writeEmptyRetentionLeasesFile(indexShard); } // since we recover from local, just fill the files and size - final RecoveryState.Index index = recoveryState.getIndex(); + final RecoveryIndex index = recoveryState.getIndex(); try { if (si != null) { addRecoveredFileDetails(si, store, index); @@ -510,7 +511,7 @@ private static void writeEmptyRetentionLeasesFile(IndexShard indexShard) throws assert indexShard.loadRetentionLeases().leases().isEmpty(); } - private void addRecoveredFileDetails(SegmentInfos si, Store store, RecoveryState.Index index) throws IOException { + private void addRecoveredFileDetails(SegmentInfos si, Store store, RecoveryIndex index) throws IOException { final Directory directory = store.directory(); for (String name : Lucene.files(si)) { long length = directory.fileLength(name); diff --git a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java index e88d123f50679..415847dec1af4 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java @@ -58,7 +58,7 @@ public class MultiFileWriter extends AbstractRefCounted implements Releasable { - public MultiFileWriter(Store store, RecoveryState.Index indexState, String tempFilePrefix, Logger logger, Runnable ensureOpen) { + public MultiFileWriter(Store store, RecoveryIndex indexState, String tempFilePrefix, Logger logger, Runnable ensureOpen) { super("multi_file_writer"); this.store = store; this.indexState = indexState; @@ -71,7 +71,7 @@ public MultiFileWriter(Store store, RecoveryState.Index indexState, String tempF private final AtomicBoolean closed = new AtomicBoolean(false); private final Logger logger; private final Store store; - private final RecoveryState.Index indexState; + private final RecoveryIndex indexState; private final String tempFilePrefix; private final ConcurrentMap openIndexOutputs = ConcurrentCollections.newConcurrentMap(); diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 59246d03beda5..03ba6cf2aeb47 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -544,8 +544,8 @@ public void messageReceived(final RecoveryFileChunkRequest request, TransportCha return; } - final RecoveryState.Index indexState = recoveryTarget.state().getIndex(); - if (request.sourceThrottleTimeInNanos() != RecoveryState.Index.UNKNOWN) { + final RecoveryIndex indexState = recoveryTarget.state().getIndex(); + if (request.sourceThrottleTimeInNanos() != RecoveryIndex.UNKNOWN) { indexState.addSourceThrottling(request.sourceThrottleTimeInNanos()); } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryIndex.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryIndex.java new file mode 100644 index 0000000000000..3a790d9170cc5 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryIndex.java @@ -0,0 +1,509 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.recovery; + +import org.opensearch.common.Strings; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.ToXContentFragment; +import org.opensearch.common.xcontent.ToXContentObject; +import org.opensearch.common.xcontent.XContentBuilder; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.index.store.StoreStats; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +public class RecoveryIndex extends Timer implements ToXContentFragment, Writeable { + private final RecoveryFilesDetails fileDetails; + + public static final long UNKNOWN = -1L; + + private long sourceThrottlingInNanos = UNKNOWN; + private long targetThrottleTimeInNanos = UNKNOWN; + + public RecoveryIndex() { + this(new RecoveryFilesDetails()); + } + + public RecoveryIndex(RecoveryFilesDetails recoveryFilesDetails) { + this.fileDetails = recoveryFilesDetails; + } + + public RecoveryIndex(StreamInput in) throws IOException { + super(in); + fileDetails = new RecoveryFilesDetails(in); + sourceThrottlingInNanos = in.readLong(); + targetThrottleTimeInNanos = in.readLong(); + } + + @Override + public synchronized void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + fileDetails.writeTo(out); + out.writeLong(sourceThrottlingInNanos); + out.writeLong(targetThrottleTimeInNanos); + } + + public synchronized List fileDetails() { + return Collections.unmodifiableList(new ArrayList<>(fileDetails.values())); + } + + public synchronized void reset() { + super.reset(); + fileDetails.clear(); + sourceThrottlingInNanos = UNKNOWN; + targetThrottleTimeInNanos = UNKNOWN; + } + + public synchronized void addFileDetail(String name, long length, boolean reused) { + fileDetails.addFileDetails(name, length, reused); + } + + public synchronized void setFileDetailsComplete() { + fileDetails.setComplete(); + } + + public synchronized void addRecoveredBytesToFile(String name, long bytes) { + fileDetails.addRecoveredBytesToFile(name, bytes); + } + + public synchronized void addSourceThrottling(long timeInNanos) { + if (sourceThrottlingInNanos == UNKNOWN) { + sourceThrottlingInNanos = timeInNanos; + } else { + sourceThrottlingInNanos += timeInNanos; + } + } + + public synchronized void addTargetThrottling(long timeInNanos) { + if (targetThrottleTimeInNanos == UNKNOWN) { + targetThrottleTimeInNanos = timeInNanos; + } else { + targetThrottleTimeInNanos += timeInNanos; + } + } + + public synchronized TimeValue sourceThrottling() { + return TimeValue.timeValueNanos(sourceThrottlingInNanos); + } + + public synchronized TimeValue targetThrottling() { + return TimeValue.timeValueNanos(targetThrottleTimeInNanos); + } + + /** + * total number of files that are part of this recovery, both re-used and recovered + */ + public synchronized int totalFileCount() { + return fileDetails.size(); + } + + /** + * total number of files to be recovered (potentially not yet done) + */ + public synchronized int totalRecoverFiles() { + int total = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused() == false) { + total++; + } + } + return total; + } + + /** + * number of file that were recovered (excluding on ongoing files) + */ + public synchronized int recoveredFileCount() { + int count = 0; + for (FileDetail file : fileDetails.values()) { + if (file.fullyRecovered()) { + count++; + } + } + return count; + } + + /** + * percent of recovered (i.e., not reused) files out of the total files to be recovered + */ + public synchronized float recoveredFilesPercent() { + int total = 0; + int recovered = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused() == false) { + total++; + if (file.fullyRecovered()) { + recovered++; + } + } + } + if (total == 0 && fileDetails.size() == 0) { // indicates we are still in init phase + return 0.0f; + } + if (total == recovered) { + return 100.0f; + } else { + float result = 100.0f * (recovered / (float) total); + return result; + } + } + + /** + * total number of bytes in th shard + */ + public synchronized long totalBytes() { + long total = 0; + for (FileDetail file : fileDetails.values()) { + total += file.length(); + } + return total; + } + + /** + * total number of bytes recovered so far, including both existing and reused + */ + public synchronized long recoveredBytes() { + long recovered = 0; + for (FileDetail file : fileDetails.values()) { + recovered += file.recovered(); + } + return recovered; + } + + /** + * total bytes of files to be recovered (potentially not yet done) + */ + public synchronized long totalRecoverBytes() { + long total = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused() == false) { + total += file.length(); + } + } + return total; + } + + /** + * @return number of bytes still to recover, i.e. {@link RecoveryIndex#totalRecoverBytes()} minus {@link RecoveryIndex#recoveredBytes()}, or + * {@code -1} if the full set of files to recover is not yet known + */ + public synchronized long bytesStillToRecover() { + if (fileDetails.isComplete() == false) { + return -1L; + } + long total = 0L; + for (FileDetail file : fileDetails.values()) { + if (file.reused() == false) { + total += file.length() - file.recovered(); + } + } + return total; + } + + /** + * percent of bytes recovered out of total files bytes *to be* recovered + */ + public synchronized float recoveredBytesPercent() { + long total = 0; + long recovered = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused() == false) { + total += file.length(); + recovered += file.recovered(); + } + } + if (total == 0 && fileDetails.size() == 0) { + // indicates we are still in init phase + return 0.0f; + } + if (total == recovered) { + return 100.0f; + } else { + return 100.0f * recovered / total; + } + } + + public synchronized int reusedFileCount() { + int reused = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused()) { + reused++; + } + } + return reused; + } + + public synchronized long reusedBytes() { + long reused = 0; + for (FileDetail file : fileDetails.values()) { + if (file.reused()) { + reused += file.length(); + } + } + return reused; + } + + @Override + public synchronized XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + // stream size first, as it matters more and the files section can be long + builder.startObject(RecoveryState.Fields.SIZE); + builder.humanReadableField(RecoveryState.Fields.TOTAL_IN_BYTES, RecoveryState.Fields.TOTAL, new ByteSizeValue(totalBytes())); + builder.humanReadableField(RecoveryState.Fields.REUSED_IN_BYTES, RecoveryState.Fields.REUSED, new ByteSizeValue(reusedBytes())); + builder.humanReadableField( + RecoveryState.Fields.RECOVERED_IN_BYTES, + RecoveryState.Fields.RECOVERED, + new ByteSizeValue(recoveredBytes()) + ); + builder.field(RecoveryState.Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredBytesPercent())); + builder.endObject(); + + builder.startObject(RecoveryState.Fields.FILES); + builder.field(RecoveryState.Fields.TOTAL, totalFileCount()); + builder.field(RecoveryState.Fields.REUSED, reusedFileCount()); + builder.field(RecoveryState.Fields.RECOVERED, recoveredFileCount()); + builder.field(RecoveryState.Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredFilesPercent())); + fileDetails.toXContent(builder, params); + builder.endObject(); + builder.humanReadableField(RecoveryState.Fields.TOTAL_TIME_IN_MILLIS, RecoveryState.Fields.TOTAL_TIME, new TimeValue(time())); + builder.humanReadableField( + RecoveryState.Fields.SOURCE_THROTTLE_TIME_IN_MILLIS, + RecoveryState.Fields.SOURCE_THROTTLE_TIME, + sourceThrottling() + ); + builder.humanReadableField( + RecoveryState.Fields.TARGET_THROTTLE_TIME_IN_MILLIS, + RecoveryState.Fields.TARGET_THROTTLE_TIME, + targetThrottling() + ); + return builder; + } + + @Override + public synchronized String toString() { + try { + XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); + builder.startObject(); + toXContent(builder, EMPTY_PARAMS); + builder.endObject(); + return Strings.toString(builder); + } catch (IOException e) { + return "{ \"error\" : \"" + e.getMessage() + "\"}"; + } + } + + public synchronized FileDetail getFileDetails(String dest) { + return fileDetails.get(dest); + } + + public static class FileDetail implements ToXContentObject, Writeable { + private String name; + private long length; + private long recovered; + private boolean reused; + + public FileDetail(String name, long length, boolean reused) { + assert name != null; + this.name = name; + this.length = length; + this.reused = reused; + } + + public FileDetail(StreamInput in) throws IOException { + name = in.readString(); + length = in.readVLong(); + recovered = in.readVLong(); + reused = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + out.writeVLong(length); + out.writeVLong(recovered); + out.writeBoolean(reused); + } + + void addRecoveredBytes(long bytes) { + assert reused == false : "file is marked as reused, can't update recovered bytes"; + assert bytes >= 0 : "can't recovered negative bytes. got [" + bytes + "]"; + recovered += bytes; + } + + /** + * file name * + */ + public String name() { + return name; + } + + /** + * file length * + */ + public long length() { + return length; + } + + /** + * number of bytes recovered for this file (so far). 0 if the file is reused * + */ + public long recovered() { + return recovered; + } + + /** + * returns true if the file is reused from a local copy + */ + public boolean reused() { + return reused; + } + + boolean fullyRecovered() { + return reused == false && length == recovered; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(RecoveryState.Fields.NAME, name); + builder.humanReadableField(RecoveryState.Fields.LENGTH_IN_BYTES, RecoveryState.Fields.LENGTH, new ByteSizeValue(length)); + builder.field(RecoveryState.Fields.REUSED, reused); + builder.humanReadableField( + RecoveryState.Fields.RECOVERED_IN_BYTES, + RecoveryState.Fields.RECOVERED, + new ByteSizeValue(recovered) + ); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof FileDetail) { + FileDetail other = (FileDetail) obj; + return name.equals(other.name) && length == other.length() && reused == other.reused() && recovered == other.recovered(); + } + return false; + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + Long.hashCode(length); + result = 31 * result + Long.hashCode(recovered); + result = 31 * result + (reused ? 1 : 0); + return result; + } + + @Override + public String toString() { + return "file (name [" + name + "], reused [" + reused + "], length [" + length + "], recovered [" + recovered + "])"; + } + } + + public static class RecoveryFilesDetails implements ToXContentFragment, Writeable { + protected final Map fileDetails = new HashMap<>(); + protected boolean complete; + + public RecoveryFilesDetails() {} + + RecoveryFilesDetails(StreamInput in) throws IOException { + int size = in.readVInt(); + for (int i = 0; i < size; i++) { + FileDetail file = new FileDetail(in); + fileDetails.put(file.name, file); + } + if (in.getVersion().onOrAfter(StoreStats.RESERVED_BYTES_VERSION)) { + complete = in.readBoolean(); + } else { + // This flag is used by disk-based allocation to decide whether the remaining bytes measurement is accurate or not; if not + // then it falls back on an estimate. There's only a very short window in which the file details are present but incomplete + // so this is a reasonable approximation, and the stats reported to the disk-based allocator don't hit this code path + // anyway since they always use IndexShard#getRecoveryState which is never transported over the wire. + complete = fileDetails.isEmpty() == false; + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + final FileDetail[] files = values().toArray(new FileDetail[0]); + out.writeVInt(files.length); + for (FileDetail file : files) { + file.writeTo(out); + } + if (out.getVersion().onOrAfter(StoreStats.RESERVED_BYTES_VERSION)) { + out.writeBoolean(complete); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + if (params.paramAsBoolean("detailed", false)) { + builder.startArray(RecoveryState.Fields.DETAILS); + for (FileDetail file : values()) { + file.toXContent(builder, params); + } + builder.endArray(); + } + + return builder; + } + + public void addFileDetails(String name, long length, boolean reused) { + assert complete == false : "addFileDetail for [" + name + "] when file details are already complete"; + FileDetail existing = fileDetails.put(name, new FileDetail(name, length, reused)); + assert existing == null : "file [" + name + "] is already reported"; + } + + public void addRecoveredBytesToFile(String name, long bytes) { + FileDetail file = fileDetails.get(name); + assert file != null : "file [" + name + "] hasn't been reported"; + file.addRecoveredBytes(bytes); + } + + public FileDetail get(String name) { + return fileDetails.get(name); + } + + public void setComplete() { + complete = true; + } + + public int size() { + return fileDetails.size(); + } + + public boolean isEmpty() { + return fileDetails.isEmpty(); + } + + public void clear() { + fileDetails.clear(); + complete = false; + } + + public Collection values() { + return fileDetails.values(); + } + + public boolean isComplete() { + return complete; + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java index 802deb577eff7..19d3bb24be96b 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java @@ -37,29 +37,18 @@ import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; -import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.ToXContentFragment; -import org.opensearch.common.xcontent.ToXContentObject; import org.opensearch.common.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; -import org.opensearch.index.store.StoreStats; import org.opensearch.indices.replication.common.RState; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; import java.util.Locale; -import java.util.Map; /** * Keeps track of state related to shard recovery. @@ -120,7 +109,7 @@ public static Stage fromId(byte id) { private Stage stage; - private final Index index; + private final RecoveryIndex index; private final Translog translog; private final VerifyIndex verifyIndex; private final Timer timer; @@ -133,10 +122,10 @@ public static Stage fromId(byte id) { private boolean primary; public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode) { - this(shardRouting, targetNode, sourceNode, new Index()); + this(shardRouting, targetNode, sourceNode, new RecoveryIndex()); } - public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode, Index index) { + public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode, RecoveryIndex index) { assert shardRouting.initializing() : "only allow initializing shard routing to be recovered: " + shardRouting; RecoverySource recoverySource = shardRouting.recoverySource(); assert (recoverySource.getType() == RecoverySource.Type.PEER) == (sourceNode != null) @@ -161,7 +150,7 @@ public RecoveryState(StreamInput in) throws IOException { recoverySource = RecoverySource.readFrom(in); targetNode = new DiscoveryNode(in); sourceNode = in.readOptionalWriteable(DiscoveryNode::new); - index = new Index(in); + index = new RecoveryIndex(in); translog = new Translog(in); verifyIndex = new VerifyIndex(in); primary = in.readBoolean(); @@ -245,7 +234,7 @@ public synchronized RecoveryState setStage(Stage stage) { return this; } - public Index getIndex() { + public RecoveryIndex getIndex() { return index; } @@ -558,466 +547,4 @@ public synchronized XContentBuilder toXContent(XContentBuilder builder, Params p } } - public static class FileDetail implements ToXContentObject, Writeable { - private String name; - private long length; - private long recovered; - private boolean reused; - - public FileDetail(String name, long length, boolean reused) { - assert name != null; - this.name = name; - this.length = length; - this.reused = reused; - } - - public FileDetail(StreamInput in) throws IOException { - name = in.readString(); - length = in.readVLong(); - recovered = in.readVLong(); - reused = in.readBoolean(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeString(name); - out.writeVLong(length); - out.writeVLong(recovered); - out.writeBoolean(reused); - } - - void addRecoveredBytes(long bytes) { - assert reused == false : "file is marked as reused, can't update recovered bytes"; - assert bytes >= 0 : "can't recovered negative bytes. got [" + bytes + "]"; - recovered += bytes; - } - - /** - * file name * - */ - public String name() { - return name; - } - - /** - * file length * - */ - public long length() { - return length; - } - - /** - * number of bytes recovered for this file (so far). 0 if the file is reused * - */ - public long recovered() { - return recovered; - } - - /** - * returns true if the file is reused from a local copy - */ - public boolean reused() { - return reused; - } - - boolean fullyRecovered() { - return reused == false && length == recovered; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.field(Fields.NAME, name); - builder.humanReadableField(Fields.LENGTH_IN_BYTES, Fields.LENGTH, new ByteSizeValue(length)); - builder.field(Fields.REUSED, reused); - builder.humanReadableField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, new ByteSizeValue(recovered)); - builder.endObject(); - return builder; - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof FileDetail) { - FileDetail other = (FileDetail) obj; - return name.equals(other.name) && length == other.length() && reused == other.reused() && recovered == other.recovered(); - } - return false; - } - - @Override - public int hashCode() { - int result = name.hashCode(); - result = 31 * result + Long.hashCode(length); - result = 31 * result + Long.hashCode(recovered); - result = 31 * result + (reused ? 1 : 0); - return result; - } - - @Override - public String toString() { - return "file (name [" + name + "], reused [" + reused + "], length [" + length + "], recovered [" + recovered + "])"; - } - } - - public static class RecoveryFilesDetails implements ToXContentFragment, Writeable { - protected final Map fileDetails = new HashMap<>(); - protected boolean complete; - - public RecoveryFilesDetails() {} - - RecoveryFilesDetails(StreamInput in) throws IOException { - int size = in.readVInt(); - for (int i = 0; i < size; i++) { - FileDetail file = new FileDetail(in); - fileDetails.put(file.name, file); - } - if (in.getVersion().onOrAfter(StoreStats.RESERVED_BYTES_VERSION)) { - complete = in.readBoolean(); - } else { - // This flag is used by disk-based allocation to decide whether the remaining bytes measurement is accurate or not; if not - // then it falls back on an estimate. There's only a very short window in which the file details are present but incomplete - // so this is a reasonable approximation, and the stats reported to the disk-based allocator don't hit this code path - // anyway since they always use IndexShard#getRecoveryState which is never transported over the wire. - complete = fileDetails.isEmpty() == false; - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - final FileDetail[] files = values().toArray(new FileDetail[0]); - out.writeVInt(files.length); - for (FileDetail file : files) { - file.writeTo(out); - } - if (out.getVersion().onOrAfter(StoreStats.RESERVED_BYTES_VERSION)) { - out.writeBoolean(complete); - } - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - if (params.paramAsBoolean("detailed", false)) { - builder.startArray(Fields.DETAILS); - for (FileDetail file : values()) { - file.toXContent(builder, params); - } - builder.endArray(); - } - - return builder; - } - - public void addFileDetails(String name, long length, boolean reused) { - assert complete == false : "addFileDetail for [" + name + "] when file details are already complete"; - FileDetail existing = fileDetails.put(name, new FileDetail(name, length, reused)); - assert existing == null : "file [" + name + "] is already reported"; - } - - public void addRecoveredBytesToFile(String name, long bytes) { - FileDetail file = fileDetails.get(name); - assert file != null : "file [" + name + "] hasn't been reported"; - file.addRecoveredBytes(bytes); - } - - public FileDetail get(String name) { - return fileDetails.get(name); - } - - public void setComplete() { - complete = true; - } - - public int size() { - return fileDetails.size(); - } - - public boolean isEmpty() { - return fileDetails.isEmpty(); - } - - public void clear() { - fileDetails.clear(); - complete = false; - } - - public Collection values() { - return fileDetails.values(); - } - - public boolean isComplete() { - return complete; - } - } - - public static class Index extends Timer implements ToXContentFragment, Writeable { - private final RecoveryFilesDetails fileDetails; - - public static final long UNKNOWN = -1L; - - private long sourceThrottlingInNanos = UNKNOWN; - private long targetThrottleTimeInNanos = UNKNOWN; - - public Index() { - this(new RecoveryFilesDetails()); - } - - public Index(RecoveryFilesDetails recoveryFilesDetails) { - this.fileDetails = recoveryFilesDetails; - } - - public Index(StreamInput in) throws IOException { - super(in); - fileDetails = new RecoveryFilesDetails(in); - sourceThrottlingInNanos = in.readLong(); - targetThrottleTimeInNanos = in.readLong(); - } - - @Override - public synchronized void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - fileDetails.writeTo(out); - out.writeLong(sourceThrottlingInNanos); - out.writeLong(targetThrottleTimeInNanos); - } - - public synchronized List fileDetails() { - return Collections.unmodifiableList(new ArrayList<>(fileDetails.values())); - } - - public synchronized void reset() { - super.reset(); - fileDetails.clear(); - sourceThrottlingInNanos = UNKNOWN; - targetThrottleTimeInNanos = UNKNOWN; - } - - public synchronized void addFileDetail(String name, long length, boolean reused) { - fileDetails.addFileDetails(name, length, reused); - } - - public synchronized void setFileDetailsComplete() { - fileDetails.setComplete(); - } - - public synchronized void addRecoveredBytesToFile(String name, long bytes) { - fileDetails.addRecoveredBytesToFile(name, bytes); - } - - public synchronized void addSourceThrottling(long timeInNanos) { - if (sourceThrottlingInNanos == UNKNOWN) { - sourceThrottlingInNanos = timeInNanos; - } else { - sourceThrottlingInNanos += timeInNanos; - } - } - - public synchronized void addTargetThrottling(long timeInNanos) { - if (targetThrottleTimeInNanos == UNKNOWN) { - targetThrottleTimeInNanos = timeInNanos; - } else { - targetThrottleTimeInNanos += timeInNanos; - } - } - - public synchronized TimeValue sourceThrottling() { - return TimeValue.timeValueNanos(sourceThrottlingInNanos); - } - - public synchronized TimeValue targetThrottling() { - return TimeValue.timeValueNanos(targetThrottleTimeInNanos); - } - - /** - * total number of files that are part of this recovery, both re-used and recovered - */ - public synchronized int totalFileCount() { - return fileDetails.size(); - } - - /** - * total number of files to be recovered (potentially not yet done) - */ - public synchronized int totalRecoverFiles() { - int total = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused() == false) { - total++; - } - } - return total; - } - - /** - * number of file that were recovered (excluding on ongoing files) - */ - public synchronized int recoveredFileCount() { - int count = 0; - for (FileDetail file : fileDetails.values()) { - if (file.fullyRecovered()) { - count++; - } - } - return count; - } - - /** - * percent of recovered (i.e., not reused) files out of the total files to be recovered - */ - public synchronized float recoveredFilesPercent() { - int total = 0; - int recovered = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused() == false) { - total++; - if (file.fullyRecovered()) { - recovered++; - } - } - } - if (total == 0 && fileDetails.size() == 0) { // indicates we are still in init phase - return 0.0f; - } - if (total == recovered) { - return 100.0f; - } else { - float result = 100.0f * (recovered / (float) total); - return result; - } - } - - /** - * total number of bytes in th shard - */ - public synchronized long totalBytes() { - long total = 0; - for (FileDetail file : fileDetails.values()) { - total += file.length(); - } - return total; - } - - /** - * total number of bytes recovered so far, including both existing and reused - */ - public synchronized long recoveredBytes() { - long recovered = 0; - for (FileDetail file : fileDetails.values()) { - recovered += file.recovered(); - } - return recovered; - } - - /** - * total bytes of files to be recovered (potentially not yet done) - */ - public synchronized long totalRecoverBytes() { - long total = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused() == false) { - total += file.length(); - } - } - return total; - } - - /** - * @return number of bytes still to recover, i.e. {@link Index#totalRecoverBytes()} minus {@link Index#recoveredBytes()}, or - * {@code -1} if the full set of files to recover is not yet known - */ - public synchronized long bytesStillToRecover() { - if (fileDetails.isComplete() == false) { - return -1L; - } - long total = 0L; - for (FileDetail file : fileDetails.values()) { - if (file.reused() == false) { - total += file.length() - file.recovered(); - } - } - return total; - } - - /** - * percent of bytes recovered out of total files bytes *to be* recovered - */ - public synchronized float recoveredBytesPercent() { - long total = 0; - long recovered = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused() == false) { - total += file.length(); - recovered += file.recovered(); - } - } - if (total == 0 && fileDetails.size() == 0) { - // indicates we are still in init phase - return 0.0f; - } - if (total == recovered) { - return 100.0f; - } else { - return 100.0f * recovered / total; - } - } - - public synchronized int reusedFileCount() { - int reused = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused()) { - reused++; - } - } - return reused; - } - - public synchronized long reusedBytes() { - long reused = 0; - for (FileDetail file : fileDetails.values()) { - if (file.reused()) { - reused += file.length(); - } - } - return reused; - } - - @Override - public synchronized XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - // stream size first, as it matters more and the files section can be long - builder.startObject(Fields.SIZE); - builder.humanReadableField(Fields.TOTAL_IN_BYTES, Fields.TOTAL, new ByteSizeValue(totalBytes())); - builder.humanReadableField(Fields.REUSED_IN_BYTES, Fields.REUSED, new ByteSizeValue(reusedBytes())); - builder.humanReadableField(Fields.RECOVERED_IN_BYTES, Fields.RECOVERED, new ByteSizeValue(recoveredBytes())); - builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredBytesPercent())); - builder.endObject(); - - builder.startObject(Fields.FILES); - builder.field(Fields.TOTAL, totalFileCount()); - builder.field(Fields.REUSED, reusedFileCount()); - builder.field(Fields.RECOVERED, recoveredFileCount()); - builder.field(Fields.PERCENT, String.format(Locale.ROOT, "%1.1f%%", recoveredFilesPercent())); - fileDetails.toXContent(builder, params); - builder.endObject(); - builder.humanReadableField(Fields.TOTAL_TIME_IN_MILLIS, Fields.TOTAL_TIME, new TimeValue(time())); - builder.humanReadableField(Fields.SOURCE_THROTTLE_TIME_IN_MILLIS, Fields.SOURCE_THROTTLE_TIME, sourceThrottling()); - builder.humanReadableField(Fields.TARGET_THROTTLE_TIME_IN_MILLIS, Fields.TARGET_THROTTLE_TIME, targetThrottling()); - return builder; - } - - @Override - public synchronized String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return Strings.toString(builder); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } - } - - public synchronized FileDetail getFileDetails(String dest) { - return fileDetails.get(dest); - } - } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java index 79a9f5da8a69f..ebdead1fcf250 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java @@ -312,7 +312,7 @@ public void receiveFileInfo( ActionListener.completeWith(listener, () -> { indexShard.resetRecoveryStage(); indexShard.prepareForIndexRecovery(); - final RecoveryState.Index index = state().getIndex(); + final RecoveryIndex index = state().getIndex(); for (int i = 0; i < phase1ExistingFileNames.size(); i++) { index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true); } diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index 9a4b26300d16c..151b4a4011f5e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -19,8 +19,8 @@ import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.recovery.MultiFileWriter; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryRequestTracker; -import org.opensearch.indices.recovery.RecoveryState; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -41,7 +41,7 @@ public abstract class ReplicationTarget extends AbstractRefCounted { protected final ReplicationListener listener; protected final MultiFileWriter multiFileWriter; protected final Logger logger; - protected final RecoveryState.Index recoveryStateIndex; + protected final RecoveryIndex recoveryStateIndex; protected abstract String getPrefix(); @@ -53,7 +53,7 @@ public abstract class ReplicationTarget extends AbstractRefCounted { public abstract RState state(); - public ReplicationTarget(String name, IndexShard indexShard, RecoveryState.Index recoveryStateIndex, ReplicationListener listener) { + public ReplicationTarget(String name, IndexShard indexShard, RecoveryIndex recoveryStateIndex, ReplicationListener listener) { super(name); this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.listener = listener; diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java index 0856e5bad5dc6..0721756b7ea8f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java @@ -8,6 +8,7 @@ package org.opensearch.indices.replication.copy; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.Timer; import org.opensearch.indices.replication.common.RState; @@ -15,10 +16,10 @@ public class ReplicationState implements RState { private Timer timer; - private RecoveryState.Index index; + private RecoveryIndex index; private Stage stage; - public ReplicationState(RecoveryState.Index index) { + public ReplicationState(RecoveryIndex index) { this.index = index; this.timer = new Timer(); stage = Stage.INACTIVE; @@ -33,41 +34,29 @@ public Timer getTimer() { return timer; } - public RecoveryState.Index getIndex() { + public RecoveryIndex getIndex() { return index; } + /** + * THis class duplicates the purpose/functionality of {@link RecoveryState.Stage} + * so this temporary implementation simply aliases the enums from the other class. + */ public enum Stage { // TODO: Add more steps here. - INACTIVE((byte) 0), + INACTIVE(RecoveryState.Stage.INIT), - ACTIVE((byte) 1); - - private static final ReplicationState.Stage[] STAGES = new ReplicationState.Stage[ReplicationState.Stage.values().length]; - - static { - for (ReplicationState.Stage stage : ReplicationState.Stage.values()) { - assert stage.id() < STAGES.length && stage.id() >= 0; - STAGES[stage.id] = stage; - } - } + ACTIVE(RecoveryState.Stage.INDEX); private final byte id; - Stage(byte id) { - this.id = id; + Stage(RecoveryState.Stage recoveryStage) { + this.id = recoveryStage.id(); } public byte id() { return id; } - - public static ReplicationState.Stage fromId(byte id) { - if (id < 0 || id >= STAGES.length) { - throw new IllegalArgumentException("No mapping for id [" + id + "]"); - } - return STAGES[id]; - } } public synchronized Stage getStage() { diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java index 613125b3aee1a..e4729166654e8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java @@ -44,6 +44,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.SegmentReplicationReplicaService; import org.opensearch.indices.replication.checkpoint.TransportCheckpointInfoResponse; @@ -79,7 +80,7 @@ public SegmentReplicationTarget( PrimaryShardReplicationSource source, SegmentReplicationReplicaService.SegmentReplicationListener listener ) { - super("replication_status", indexShard, new RecoveryState.Index(), listener); + super("replication_status", indexShard, new RecoveryIndex(), listener); this.checkpoint = checkpoint; this.source = source; state = new ReplicationState(recoveryStateIndex); diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index d73f3f81c8138..ef172f325bc68 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -134,6 +134,7 @@ import org.opensearch.indices.IndicesQueryCache; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.repositories.IndexId; @@ -3150,7 +3151,7 @@ public void testRecoverFromLocalShard() throws IOException { RecoveryState recoveryState = targetShard.recoveryState(); assertEquals(RecoveryState.Stage.DONE, recoveryState.getStage()); assertTrue(recoveryState.getIndex().fileDetails().size() > 0); - for (RecoveryState.FileDetail file : recoveryState.getIndex().fileDetails()) { + for (RecoveryIndex.FileDetail file : recoveryState.getIndex().fileDetails()) { if (file.reused()) { assertEquals(file.recovered(), 0); } else { diff --git a/server/src/test/java/org/opensearch/index/shard/StoreRecoveryTests.java b/server/src/test/java/org/opensearch/index/shard/StoreRecoveryTests.java index c325ac6bc754e..06fecdd0d22ce 100644 --- a/server/src/test/java/org/opensearch/index/shard/StoreRecoveryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/StoreRecoveryTests.java @@ -61,7 +61,7 @@ import org.opensearch.index.mapper.IdFieldMapper; import org.opensearch.index.mapper.Uid; import org.opensearch.index.seqno.SequenceNumbers; -import org.opensearch.indices.recovery.RecoveryState; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; @@ -108,7 +108,7 @@ public void testAddIndices() throws IOException { writer.close(); } StoreRecovery storeRecovery = new StoreRecovery(new ShardId("foo", "bar", 1), logger); - RecoveryState.Index indexStats = new RecoveryState.Index(); + RecoveryIndex indexStats = new RecoveryIndex(); Directory target = newFSDirectory(createTempDir()); final long maxSeqNo = randomNonNegativeLong(); final long maxUnsafeAutoIdTimestamp = randomNonNegativeLong(); @@ -174,7 +174,7 @@ public void testSplitShard() throws IOException { writer.commit(); writer.close(); StoreRecovery storeRecovery = new StoreRecovery(new ShardId("foo", "bar", 1), logger); - RecoveryState.Index indexStats = new RecoveryState.Index(); + RecoveryIndex indexStats = new RecoveryIndex(); Directory target = newFSDirectory(createTempDir()); final long maxSeqNo = randomNonNegativeLong(); final long maxUnsafeAutoIdTimestamp = randomNonNegativeLong(); @@ -250,7 +250,7 @@ public void testSplitShard() throws IOException { public void testStatsDirWrapper() throws IOException { Directory dir = newDirectory(); Directory target = newDirectory(); - RecoveryState.Index indexStats = new RecoveryState.Index(); + RecoveryIndex indexStats = new RecoveryIndex(); StoreRecovery.StatsDirectoryWrapper wrapper = new StoreRecovery.StatsDirectoryWrapper(target, indexStats); try (IndexOutput output = dir.createOutput("foo.bar", IOContext.DEFAULT)) { CodecUtil.writeHeader(output, "foo", 0); diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java index b195984de64b5..b005df315aad6 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java @@ -198,7 +198,7 @@ public void testSendFiles() throws Throwable { metas.add(md); } Store targetStore = newStore(createTempDir()); - MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryState.Index.class), "", logger, () -> {}); + MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryIndex.class), "", logger, () -> {}); RecoveryTargetHandler target = new TestRecoveryTargetHandler() { @Override public void writeFileChunk( @@ -528,7 +528,7 @@ public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { ) ); Store targetStore = newStore(createTempDir(), false); - MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryState.Index.class), "", logger, () -> {}); + MultiFileWriter multiFileWriter = new MultiFileWriter(targetStore, mock(RecoveryIndex.class), "", logger, () -> {}); RecoveryTargetHandler target = new TestRecoveryTargetHandler() { @Override public void writeFileChunk( diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTargetTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTargetTests.java index 57bea083cfd2b..f65bf98c5e916 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTargetTests.java @@ -41,8 +41,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.index.shard.ShardId; -import org.opensearch.indices.recovery.RecoveryState.FileDetail; -import org.opensearch.indices.recovery.RecoveryState.Index; +import org.opensearch.indices.recovery.RecoveryIndex.FileDetail; import org.opensearch.indices.recovery.RecoveryState.Stage; import org.opensearch.indices.recovery.RecoveryState.Translog; import org.opensearch.indices.recovery.RecoveryState.VerifyIndex; @@ -136,11 +135,11 @@ Timer createObj(StreamInput in) throws IOException { } }; } else if (randomBoolean()) { - timer = new Index(); + timer = new RecoveryIndex(); streamer = new Streamer(stop, timer) { @Override Timer createObj(StreamInput in) throws IOException { - return new Index(in); + return new RecoveryIndex(in); } }; } else if (randomBoolean()) { @@ -211,7 +210,7 @@ public void testIndex() throws Throwable { } Collections.shuffle(Arrays.asList(files), random()); - final RecoveryState.Index index = new RecoveryState.Index(); + final RecoveryIndex index = new RecoveryIndex(); assertThat(index.bytesStillToRecover(), equalTo(-1L)); if (randomBoolean()) { @@ -238,8 +237,8 @@ public void testIndex() throws Throwable { // before we start we must report 0 assertThat(index.recoveredFilesPercent(), equalTo((float) 0.0)); assertThat(index.recoveredBytesPercent(), equalTo((float) 0.0)); - assertThat(index.sourceThrottling().nanos(), equalTo(Index.UNKNOWN)); - assertThat(index.targetThrottling().nanos(), equalTo(Index.UNKNOWN)); + assertThat(index.sourceThrottling().nanos(), equalTo(RecoveryIndex.UNKNOWN)); + assertThat(index.targetThrottling().nanos(), equalTo(RecoveryIndex.UNKNOWN)); index.start(); for (FileDetail file : files) { @@ -270,24 +269,24 @@ public void testIndex() throws Throwable { } AtomicBoolean streamShouldStop = new AtomicBoolean(); - Streamer backgroundReader = new Streamer(streamShouldStop, index) { + Streamer backgroundReader = new Streamer(streamShouldStop, index) { @Override - Index createObj(StreamInput in) throws IOException { - return new Index(in); + RecoveryIndex createObj(StreamInput in) throws IOException { + return new RecoveryIndex(in); } }; backgroundReader.start(); long recoveredBytes = 0; - long sourceThrottling = Index.UNKNOWN; - long targetThrottling = Index.UNKNOWN; + long sourceThrottling = RecoveryIndex.UNKNOWN; + long targetThrottling = RecoveryIndex.UNKNOWN; while (bytesToRecover > 0) { FileDetail file = randomFrom(filesToRecover); final long toRecover = Math.min(bytesToRecover, randomIntBetween(1, (int) (file.length() - file.recovered()))); final long throttledOnSource = rarely() ? randomIntBetween(10, 200) : 0; index.addSourceThrottling(throttledOnSource); - if (sourceThrottling == Index.UNKNOWN) { + if (sourceThrottling == RecoveryIndex.UNKNOWN) { sourceThrottling = throttledOnSource; } else { sourceThrottling += throttledOnSource; @@ -295,7 +294,7 @@ Index createObj(StreamInput in) throws IOException { index.addRecoveredBytesToFile(file.name(), toRecover); file.addRecoveredBytes(toRecover); final long throttledOnTarget = rarely() ? randomIntBetween(10, 200) : 0; - if (targetThrottling == Index.UNKNOWN) { + if (targetThrottling == RecoveryIndex.UNKNOWN) { targetThrottling = throttledOnTarget; } else { targetThrottling += throttledOnTarget; @@ -317,7 +316,7 @@ Index createObj(StreamInput in) throws IOException { logger.info("testing serialized information"); streamShouldStop.set(true); backgroundReader.join(); - final Index lastRead = backgroundReader.lastRead(); + final RecoveryIndex lastRead = backgroundReader.lastRead(); assertThat(lastRead.fileDetails().toArray(), arrayContainingInAnyOrder(index.fileDetails().toArray())); assertThat(lastRead.startTime(), equalTo(index.startTime())); if (completeRecovery) { @@ -535,12 +534,12 @@ VerifyIndex createObj(StreamInput in) throws IOException { } public void testConcurrentModificationIndexFileDetailsMap() throws InterruptedException { - final Index index = new Index(); + final RecoveryIndex index = new RecoveryIndex(); final AtomicBoolean stop = new AtomicBoolean(false); - Streamer readWriteIndex = new Streamer(stop, index) { + Streamer readWriteIndex = new Streamer(stop, index) { @Override - Index createObj(StreamInput in) throws IOException { - return new Index(in); + RecoveryIndex createObj(StreamInput in) throws IOException { + return new RecoveryIndex(in); } }; Thread modifyThread = new Thread() { diff --git a/server/src/test/java/org/opensearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/opensearch/repositories/fs/FsRepositoryTests.java index f2c6a13b92597..3420963bdd3e6 100644 --- a/server/src/test/java/org/opensearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/fs/FsRepositoryTests.java @@ -66,6 +66,7 @@ import org.opensearch.index.shard.ShardId; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.index.store.Store; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.repositories.IndexId; @@ -203,12 +204,12 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { futureC.actionGet(); assertEquals(secondState.getIndex().reusedFileCount(), commitFileNames.size() - 2); assertEquals(secondState.getIndex().recoveredFileCount(), 2); - List recoveredFiles = secondState.getIndex() + List recoveredFiles = secondState.getIndex() .fileDetails() .stream() .filter(f -> f.reused() == false) .collect(Collectors.toList()); - Collections.sort(recoveredFiles, Comparator.comparing(RecoveryState.FileDetail::name)); + Collections.sort(recoveredFiles, Comparator.comparing(RecoveryIndex.FileDetail::name)); assertTrue(recoveredFiles.get(0).name(), recoveredFiles.get(0).name().endsWith(".liv")); assertTrue(recoveredFiles.get(1).name(), recoveredFiles.get(1).name().endsWith("segments_" + incIndexCommit.getGeneration())); } finally { diff --git a/server/src/test/java/org/opensearch/rest/action/cat/RestRecoveryActionTests.java b/server/src/test/java/org/opensearch/rest/action/cat/RestRecoveryActionTests.java index 4407c3483a1b3..26fae393365b3 100644 --- a/server/src/test/java/org/opensearch/rest/action/cat/RestRecoveryActionTests.java +++ b/server/src/test/java/org/opensearch/rest/action/cat/RestRecoveryActionTests.java @@ -44,6 +44,7 @@ import org.opensearch.common.xcontent.XContentOpenSearchExtension; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; +import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.Timer; import org.opensearch.test.OpenSearchTestCase; @@ -91,7 +92,7 @@ public void testRestRecoveryAction() { when(targetNode.getHostName()).thenReturn(randomAlphaOfLength(8)); when(state.getTargetNode()).thenReturn(targetNode); - RecoveryState.Index index = mock(RecoveryState.Index.class); + RecoveryIndex index = mock(RecoveryIndex.class); final int totalRecoveredFiles = randomIntBetween(1, 64); when(index.totalRecoverFiles()).thenReturn(totalRecoveredFiles); From 065798f9815549adf777cb28db081af58181c40f Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Mon, 14 Mar 2022 13:54:35 -0700 Subject: [PATCH 7/8] Renaming RState class to ReplicationState A small amount of common members and logic from its subclasses has been moved into the parent class. The original ReplicationState child class has been renamed SegmentReplicationState. Signed-off-by: Kartik Ganesh --- .../opensearch/index/shard/IndexShard.java | 32 ++++++++--------- .../cluster/IndicesClusterStateService.java | 6 ++-- .../recovery/PeerRecoveryTargetService.java | 6 ++-- .../indices/recovery/RecoveryState.java | 12 ++----- .../SegmentReplicationReplicaService.java | 16 ++++----- .../indices/replication/common/RState.java | 14 -------- .../common/ReplicationListener.java | 4 +-- .../replication/common/ReplicationState.java | 36 +++++++++++++++++++ .../replication/common/ReplicationTarget.java | 2 +- ...tate.java => SegmentReplicationState.java} | 34 ++++++------------ .../copy/SegmentReplicationTarget.java | 6 ++-- 11 files changed, 86 insertions(+), 82 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/indices/replication/common/RState.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/common/ReplicationState.java rename server/src/main/java/org/opensearch/indices/replication/copy/{ReplicationState.java => SegmentReplicationState.java} (71%) 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 ed2558dee2fef..0da976927a16c 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -167,7 +167,7 @@ import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.indices.replication.copy.ReplicationCheckpoint; import org.opensearch.indices.replication.copy.ReplicationFailedException; -import org.opensearch.indices.replication.copy.ReplicationState; +import org.opensearch.indices.replication.copy.SegmentReplicationState; import org.opensearch.indices.replication.copy.TrackShardResponse; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -262,7 +262,7 @@ Runnable getGlobalCheckpointSyncer() { @Nullable private volatile RecoveryState recoveryState; - private volatile ReplicationState replicationState; + private volatile SegmentReplicationState segRepState; private final RecoveryStats recoveryStats = new RecoveryStats(); private final MeanMetric refreshMetric = new MeanMetric(); @@ -413,7 +413,7 @@ public boolean shouldCache(Query query) { this.useRetentionLeasesInPeerRecovery = replicationTracker.hasAllPeerRecoveryRetentionLeases(); this.refreshPendingLocationListener = new RefreshPendingLocationListener(); this.checkpointRefreshListener = new CheckpointRefreshListener(this, checkpointPublisher); - this.replicationState = new ReplicationState(); + this.segRepState = new SegmentReplicationState(); } public ThreadPool getThreadPool() { @@ -3082,7 +3082,7 @@ public void startRecovery( new ActionListener() { @Override public void onResponse(TrackShardResponse unused) { - segRepListener.onReplicationDone(replicationState); + segRepListener.onReplicationDone(segRepState); recoveryState.getIndex().setFileDetailsComplete(); finalizeRecovery(); postRecovery("Shard setup complete."); @@ -3090,11 +3090,7 @@ public void onResponse(TrackShardResponse unused) { @Override public void onFailure(Exception e) { - segRepListener.onReplicationFailure( - replicationState, - new ReplicationFailedException(indexShard, e), - true - ); + segRepListener.onReplicationFailure(segRepState, new ReplicationFailedException(indexShard, e), true); } } ); @@ -3750,13 +3746,17 @@ public synchronized void onNewCheckpoint( source, new SegmentReplicationReplicaService.SegmentReplicationListener() { @Override - public void onReplicationDone(ReplicationState state) { + public void onReplicationDone(SegmentReplicationState state) { markReplicationComplete(); logger.debug("Replication complete to {}", getLatestReplicationCheckpoint()); } @Override - public void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + public void onReplicationFailure( + SegmentReplicationState state, + ReplicationFailedException e, + boolean sendShardFailure + ) { markReplicationComplete(); logger.error("Failure", e); } @@ -3767,20 +3767,20 @@ public void onReplicationFailure(ReplicationState state, ReplicationFailedExcept } } - public ReplicationState getReplicationState() { - return this.replicationState; + public SegmentReplicationState getReplicationState() { + return this.segRepState; } public void markAsReplicating() { - this.replicationState.setStage(ReplicationState.Stage.ACTIVE); + this.segRepState.setStage(SegmentReplicationState.Stage.ACTIVE); } public void markReplicationComplete() { - this.replicationState.setStage(ReplicationState.Stage.INACTIVE); + this.segRepState.setStage(SegmentReplicationState.Stage.INACTIVE); } private boolean isReplicating() { - return this.replicationState.getStage() == ReplicationState.Stage.ACTIVE; + return this.segRepState.getStage() == SegmentReplicationState.Stage.ACTIVE; } /** diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index 88c60010ddfdb..e09add708b14c 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -83,7 +83,7 @@ import org.opensearch.indices.replication.SegmentReplicationReplicaService; import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.indices.replication.copy.ReplicationFailedException; -import org.opensearch.indices.replication.copy.ReplicationState; +import org.opensearch.indices.replication.copy.SegmentReplicationState; import org.opensearch.repositories.RepositoriesService; import org.opensearch.search.SearchService; import org.opensearch.snapshots.SnapshotShardsService; @@ -766,13 +766,13 @@ private ShardRoutingReplicationListener(final ShardRouting shardRouting, final l } @Override - public void onReplicationDone(final ReplicationState state) { + public void onReplicationDone(final SegmentReplicationState state) { logger.info("Shard setup complete, ready for segment copy."); shardStateAction.shardStarted(shardRouting, primaryTerm, "after replication", SHARD_STATE_ACTION_LISTENER); } @Override - public void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { handleRecoveryFailure(shardRouting, sendShardFailure, e); logger.error("Shard setup failed", e); } diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 03ba6cf2aeb47..bf73aa2692e0c 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -71,7 +71,7 @@ import org.opensearch.index.translog.TranslogCorruptedException; import org.opensearch.indices.recovery.RecoveriesCollection.RecoveryRef; import org.opensearch.indices.replication.common.ReplicationListener; -import org.opensearch.indices.replication.common.RState; +import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.ConnectTransportException; @@ -354,12 +354,12 @@ public static StartRecoveryRequest getStartRecoveryRequest( public interface RecoveryListener extends ReplicationListener { @Override - default void onDone(RState state) { + default void onDone(ReplicationState state) { onRecoveryDone((RecoveryState) state); } @Override - default void onFailure(RState state, OpenSearchException e, boolean sendShardFailure) { + default void onFailure(ReplicationState state, OpenSearchException e, boolean sendShardFailure) { onRecoveryFailure((RecoveryState) state, (RecoveryFailedException) e, sendShardFailure); } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java index 19d3bb24be96b..e1288d6d31e35 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java @@ -45,7 +45,7 @@ import org.opensearch.common.xcontent.XContentBuilder; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; -import org.opensearch.indices.replication.common.RState; +import org.opensearch.indices.replication.common.ReplicationState; import java.io.IOException; import java.util.Locale; @@ -53,7 +53,7 @@ /** * Keeps track of state related to shard recovery. */ -public class RecoveryState implements ToXContentFragment, Writeable, RState { +public class RecoveryState extends ReplicationState implements ToXContentFragment, Writeable { public enum Stage { INIT((byte) 0), @@ -108,12 +108,8 @@ public static Stage fromId(byte id) { } private Stage stage; - - private final RecoveryIndex index; private final Translog translog; private final VerifyIndex verifyIndex; - private final Timer timer; - private RecoverySource recoverySource; private ShardId shardId; @Nullable @@ -126,6 +122,7 @@ public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nulla } public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode, RecoveryIndex index) { + super(index); assert shardRouting.initializing() : "only allow initializing shard routing to be recovered: " + shardRouting; RecoverySource recoverySource = shardRouting.recoverySource(); assert (recoverySource.getType() == RecoverySource.Type.PEER) == (sourceNode != null) @@ -136,11 +133,8 @@ public RecoveryState(ShardRouting shardRouting, DiscoveryNode targetNode, @Nulla this.sourceNode = sourceNode; this.targetNode = targetNode; stage = Stage.INIT; - this.index = index; translog = new Translog(); verifyIndex = new VerifyIndex(); - timer = new Timer(); - timer.start(); } public RecoveryState(StreamInput in) throws IOException { diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index aa8df620f1412..1d3604edd6d15 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -37,12 +37,12 @@ import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.Timer; import org.opensearch.indices.replication.common.ReplicationListener; -import org.opensearch.indices.replication.common.RState; +import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.indices.replication.copy.PrimaryShardReplicationSource; import org.opensearch.indices.replication.copy.ReplicationCheckpoint; import org.opensearch.indices.replication.copy.ReplicationCollection; import org.opensearch.indices.replication.copy.ReplicationFailedException; -import org.opensearch.indices.replication.copy.ReplicationState; +import org.opensearch.indices.replication.copy.SegmentReplicationState; import org.opensearch.indices.replication.copy.SegmentReplicationTarget; import org.opensearch.indices.replication.copy.SegmentReplicationPrimaryService; import org.opensearch.indices.replication.copy.TrackShardRequest; @@ -291,17 +291,17 @@ public void onFailure(Exception e) { public interface SegmentReplicationListener extends ReplicationListener { @Override - default void onDone(RState state) { - onReplicationDone((ReplicationState) state); + default void onDone(ReplicationState state) { + onReplicationDone((SegmentReplicationState) state); } @Override - default void onFailure(RState state, OpenSearchException e, boolean sendShardFailure) { - onReplicationFailure((ReplicationState) state, (ReplicationFailedException) e, sendShardFailure); + default void onFailure(ReplicationState state, OpenSearchException e, boolean sendShardFailure) { + onReplicationFailure((SegmentReplicationState) state, (ReplicationFailedException) e, sendShardFailure); } - void onReplicationDone(ReplicationState state); + void onReplicationDone(SegmentReplicationState state); - void onReplicationFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure); + void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/common/RState.java b/server/src/main/java/org/opensearch/indices/replication/common/RState.java deleted file mode 100644 index bfc57d9b6686f..0000000000000 --- a/server/src/main/java/org/opensearch/indices/replication/common/RState.java +++ /dev/null @@ -1,14 +0,0 @@ -/* - * SPDX-License-Identifier: Apache-2.0 - * - * The OpenSearch Contributors require contributions made to - * this file be licensed under the Apache-2.0 license or a - * compatible open source license. - */ - -package org.opensearch.indices.replication.common; - -public interface RState { - - // TODO Add APIs here -} diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java index 4c5d20580d113..f0972176169bd 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationListener.java @@ -12,7 +12,7 @@ public interface ReplicationListener { - void onDone(RState state); + void onDone(ReplicationState state); - void onFailure(RState state, OpenSearchException e, boolean sendShardFailure); + void onFailure(ReplicationState state, OpenSearchException e, boolean sendShardFailure); } diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationState.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationState.java new file mode 100644 index 0000000000000..8ea9162067513 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationState.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.indices.replication.common; + +import org.opensearch.indices.recovery.RecoveryIndex; +import org.opensearch.indices.recovery.Timer; + +public class ReplicationState { + + protected Timer timer; + protected RecoveryIndex index; + + protected ReplicationState() { + // Empty default constructor for subclasses + } + + protected ReplicationState(RecoveryIndex index) { + this.index = index; + timer = new Timer(); + timer.start(); + } + + public Timer getTimer() { + return timer; + } + + public RecoveryIndex getIndex() { + return index; + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index 151b4a4011f5e..18601e97ad7d8 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -51,7 +51,7 @@ public abstract class ReplicationTarget extends AbstractRefCounted { protected abstract void onFail(OpenSearchException e, boolean sendShardFailure); - public abstract RState state(); + public abstract ReplicationState state(); public ReplicationTarget(String name, IndexShard indexShard, RecoveryIndex recoveryStateIndex, ReplicationListener listener) { super(name); diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationState.java similarity index 71% rename from server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java rename to server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationState.java index 0721756b7ea8f..3ceda8775826b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationState.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationState.java @@ -10,37 +10,34 @@ import org.opensearch.indices.recovery.RecoveryIndex; import org.opensearch.indices.recovery.RecoveryState; -import org.opensearch.indices.recovery.Timer; -import org.opensearch.indices.replication.common.RState; +import org.opensearch.indices.replication.common.ReplicationState; -public class ReplicationState implements RState { +public class SegmentReplicationState extends ReplicationState { - private Timer timer; - private RecoveryIndex index; private Stage stage; - public ReplicationState(RecoveryIndex index) { - this.index = index; - this.timer = new Timer(); + public SegmentReplicationState(RecoveryIndex index) { + super(index); stage = Stage.INACTIVE; - timer.start(); } - public ReplicationState() { + public SegmentReplicationState() { stage = Stage.INACTIVE; } - public Timer getTimer() { - return timer; + public synchronized Stage getStage() { + return this.stage; } - public RecoveryIndex getIndex() { - return index; + // synchronized is strictly speaking not needed (this is called by a single thread), but just to be safe + public synchronized void setStage(Stage stage) { + this.stage = stage; } /** * THis class duplicates the purpose/functionality of {@link RecoveryState.Stage} * so this temporary implementation simply aliases the enums from the other class. + * TODO Merge this class with the above Stage class once segrep lifecycle is finalized */ public enum Stage { // TODO: Add more steps here. @@ -58,13 +55,4 @@ public byte id() { return id; } } - - public synchronized Stage getStage() { - return this.stage; - } - - // synchronized is strictly speaking not needed (this is called by a single thread), but just to be safe - public synchronized void setStage(Stage stage) { - this.stage = stage; - } } diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java index e4729166654e8..d3179692a2bb4 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/SegmentReplicationTarget.java @@ -65,7 +65,7 @@ public class SegmentReplicationTarget extends ReplicationTarget { private final ReplicationCheckpoint checkpoint; private final PrimaryShardReplicationSource source; - private final ReplicationState state; + private final SegmentReplicationState state; /** * Creates a new replication target object that represents a replication to the provided source. @@ -83,7 +83,7 @@ public SegmentReplicationTarget( super("replication_status", indexShard, new RecoveryIndex(), listener); this.checkpoint = checkpoint; this.source = source; - state = new ReplicationState(recoveryStateIndex); + state = new SegmentReplicationState(recoveryStateIndex); } @Override @@ -115,7 +115,7 @@ public void fail(ReplicationFailedException e, boolean sendShardFailure) { } @Override - public ReplicationState state() { + public SegmentReplicationState state() { return state; } From 998808900cf09ae1539cca2cce12a8c89d52f9d9 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Tue, 15 Mar 2022 16:25:56 -0700 Subject: [PATCH 8/8] Changing AutoCloseableRefCounted to a generic class This is in response to PR feedback, and helps avoid a verbose get() method by introducing the generic at the class level. Signed-off-by: Kartik Ganesh --- .../concurrent/AutoCloseableRefCounted.java | 10 ++++----- .../recovery/PeerRecoveryTargetService.java | 22 +++++++++---------- .../recovery/RecoveriesCollection.java | 4 ++-- .../SegmentReplicationReplicaService.java | 4 ++-- .../copy/PrimaryShardReplicationSource.java | 4 ++-- .../copy/ReplicationCollection.java | 4 ++-- .../AutoCloseableRefCountedTests.java | 6 ++--- .../recovery/RecoveriesCollectionTests.java | 12 ++++------ 8 files changed, 31 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java b/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java index 86bcdd3838e23..795d352542881 100644 --- a/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java +++ b/server/src/main/java/org/opensearch/common/concurrent/AutoCloseableRefCounted.java @@ -19,18 +19,18 @@ * Adapter class that enables a {@link RefCounted} implementation to function like an {@link AutoCloseable}. * The {@link #close()} API invokes {@link RefCounted#decRef()} and ensures idempotency using a {@link OneWayGate}. */ -public class AutoCloseableRefCounted implements AutoCloseable { +public class AutoCloseableRefCounted implements AutoCloseable { - private final RefCounted ref; + private final T ref; private final OneWayGate gate; - public AutoCloseableRefCounted(RefCounted ref) { + public AutoCloseableRefCounted(T ref) { this.ref = ref; gate = new OneWayGate(); } - public T get(Class returnType) { - return (T) ref; + public T get() { + return ref; } @Override diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index bf73aa2692e0c..96a9599cbd822 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -225,7 +225,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi logger.trace("not running recovery with id [{}] - can not find it (probably finished)", recoveryId); return; } - final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + final RecoveryTarget recoveryTarget = recoveryRef.get(); timer = recoveryTarget.state().getTimer(); cancellableThreads = recoveryTarget.cancellableThreads(); if (preExistingRequest == null) { @@ -377,7 +377,7 @@ public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, if (listener == null) { return; } - recoveryRef.get(RecoveryTarget.class).prepareForTranslogOperations(request.totalTranslogOps(), listener); + recoveryRef.get().prepareForTranslogOperations(request.totalTranslogOps(), listener); } } } @@ -391,7 +391,7 @@ public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportCh if (listener == null) { return; } - recoveryRef.get(RecoveryTarget.class).finalizeRecovery(request.globalCheckpoint(), request.trimAboveSeqNo(), listener); + recoveryRef.get().finalizeRecovery(request.globalCheckpoint(), request.trimAboveSeqNo(), listener); } } } @@ -402,7 +402,7 @@ class HandoffPrimaryContextRequestHandler implements TransportRequestHandler listener = createOrFinishListener( recoveryRef, channel, @@ -436,7 +436,7 @@ private void performTranslogOps( final ActionListener listener, final RecoveryRef recoveryRef ) { - final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + final RecoveryTarget recoveryTarget = recoveryRef.get(); final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()); final Consumer retryOnMappingException = exception -> { @@ -501,7 +501,7 @@ public void messageReceived(RecoveryFilesInfoRequest request, TransportChannel c return; } - recoveryRef.get(RecoveryTarget.class) + recoveryRef.get() .receiveFileInfo( request.phase1FileNames, request.phase1FileSizes, @@ -524,7 +524,7 @@ public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel return; } - recoveryRef.get(RecoveryTarget.class) + recoveryRef.get() .cleanFiles(request.totalTranslogOps(), request.getGlobalCheckpoint(), request.sourceMetaSnapshot(), listener); } } @@ -538,7 +538,7 @@ class FileChunkTransportRequestHandler implements TransportRequestHandler listener = createOrFinishListener(recoveryRef, channel, Actions.FILE_CHUNK, request); if (listener == null) { return; @@ -588,7 +588,7 @@ private ActionListener createOrFinishListener( final RecoveryTransportRequest request, final CheckedFunction responseFn ) { - final RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + final RecoveryTarget recoveryTarget = recoveryRef.get(); final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); final ActionListener voidListener = ActionListener.map(channelListener, responseFn); @@ -622,7 +622,7 @@ public void onFailure(Exception e) { try (RecoveryRef recoveryRef = onGoingRecoveries.getRecovery(recoveryId)) { if (recoveryRef != null) { logger.error(() -> new ParameterizedMessage("unexpected error during recovery [{}], failing shard", recoveryId), e); - RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + RecoveryTarget recoveryTarget = recoveryRef.get(); onGoingRecoveries.failRecovery( recoveryId, new RecoveryFailedException(recoveryTarget.state(), "unexpected error", e), diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java index 569970471611d..531782962d98b 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoveriesCollection.java @@ -183,7 +183,7 @@ public RecoveryRef getRecoverySafe(long id, ShardId shardId) { if (recoveryRef == null) { throw new IndexShardClosedException(shardId); } - RecoveryTarget recoveryTarget = recoveryRef.get(RecoveryTarget.class); + RecoveryTarget recoveryTarget = recoveryRef.get(); assert recoveryTarget.indexShard().shardId().equals(shardId); return recoveryRef; } @@ -284,7 +284,7 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { * causes {@link RecoveryTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link RecoveryRef#close()} is called. */ - public static class RecoveryRef extends AutoCloseableRefCounted { + public static class RecoveryRef extends AutoCloseableRefCounted { /** * Important: {@link RecoveryTarget#tryIncRef()} should diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java index 1d3604edd6d15..b1334818e3b23 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationReplicaService.java @@ -181,7 +181,7 @@ private void doReplication(final long replicationId) { logger.trace("not running replication with id [{}] - can not find it (probably finished)", replicationId); return; } - final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(); timer = replicationTarget.state().getTimer(); final IndexShard indexShard = replicationTarget.indexShard(); @@ -218,7 +218,7 @@ public void onFailure(Exception e) { () -> new ParameterizedMessage("unexpected error during replication [{}], failing shard", replicationId), e ); - SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); + SegmentReplicationTarget replicationTarget = replicationRef.get(); onGoingReplications.failReplication( replicationId, new ReplicationFailedException(replicationTarget.indexShard(), "unexpected error", e), diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java index d51cfad35145b..8083b76f910f0 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/PrimaryShardReplicationSource.java @@ -231,7 +231,7 @@ public void messageReceived(final ReplicationFileChunkRequest request, Transport ReplicationCollection.ReplicationRef replicationRef = segmentReplicationReplicaService.getOnGoingReplications() .getReplicationSafe(request.getReplicationId(), request.shardId()) ) { - final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(); final ActionListener listener = createOrFinishListener(replicationRef, channel, Actions.FILE_CHUNK, request); if (listener == null) { return; @@ -274,7 +274,7 @@ private ActionListener createOrFinishListener( final ReplicationFileChunkRequest request, final CheckedFunction responseFn ) { - final SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); + final SegmentReplicationTarget replicationTarget = replicationRef.get(); final ActionListener channelListener = new ChannelActionListener<>(channel, action, request); final ActionListener voidListener = ActionListener.map(channelListener, responseFn); diff --git a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java index 76094cd1672af..02b1f6605beac 100644 --- a/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java +++ b/server/src/main/java/org/opensearch/indices/replication/copy/ReplicationCollection.java @@ -121,7 +121,7 @@ public ReplicationRef getReplicationSafe(long id, ShardId shardId) { if (replicationRef == null) { throw new IndexShardClosedException(shardId); } - SegmentReplicationTarget replicationTarget = replicationRef.get(SegmentReplicationTarget.class); + SegmentReplicationTarget replicationTarget = replicationRef.get(); assert replicationTarget.indexShard().shardId().equals(shardId); return replicationRef; } @@ -208,7 +208,7 @@ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { * causes {@link SegmentReplicationTarget#decRef()} to be called. This makes sure that the underlying resources * will not be freed until {@link ReplicationRef#close()} is called. */ - public static class ReplicationRef extends AutoCloseableRefCounted { + public static class ReplicationRef extends AutoCloseableRefCounted { /** * Important: {@link SegmentReplicationTarget#tryIncRef()} should diff --git a/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java b/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java index d0604680e5443..344368988f5ff 100644 --- a/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java +++ b/server/src/test/java/org/opensearch/common/concurrent/AutoCloseableRefCountedTests.java @@ -24,16 +24,16 @@ public class AutoCloseableRefCountedTests extends OpenSearchTestCase { private RefCounted mockRefCounted; - private AutoCloseableRefCounted testObject; + private AutoCloseableRefCounted testObject; @Before public void setup() { mockRefCounted = mock(RefCounted.class); - testObject = new AutoCloseableRefCounted(mockRefCounted); + testObject = new AutoCloseableRefCounted<>(mockRefCounted); } public void testGet() { - assertEquals(mockRefCounted, testObject.get(RefCounted.class)); + assertEquals(mockRefCounted, testObject.get()); } public void testClose() { diff --git a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java index e6be95a1eba58..86edf15d42e68 100644 --- a/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/RecoveriesCollectionTests.java @@ -69,14 +69,10 @@ public void testLastAccessTimeUpdate() throws Exception { final RecoveriesCollection collection = new RecoveriesCollection(logger, threadPool); final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); try (RecoveriesCollection.RecoveryRef status = collection.getRecovery(recoveryId)) { - final long lastSeenTime = status.get(RecoveryTarget.class).lastAccessTime(); + final long lastSeenTime = status.get().lastAccessTime(); assertBusy(() -> { try (RecoveriesCollection.RecoveryRef currentStatus = collection.getRecovery(recoveryId)) { - assertThat( - "access time failed to update", - lastSeenTime, - lessThan(currentStatus.get(RecoveryTarget.class).lastAccessTime()) - ); + assertThat("access time failed to update", lastSeenTime, lessThan(currentStatus.get().lastAccessTime())); } }); } finally { @@ -124,7 +120,7 @@ public void testRecoveryCancellation() throws Exception { final long recoveryId = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); final long recoveryId2 = startRecovery(collection, shards.getPrimaryNode(), shards.addReplica()); try (RecoveriesCollection.RecoveryRef recoveryRef = collection.getRecovery(recoveryId)) { - ShardId shardId = recoveryRef.get(RecoveryTarget.class).indexShard().shardId(); + ShardId shardId = recoveryRef.get().indexShard().shardId(); assertTrue("failed to cancel recoveries", collection.cancelRecoveriesForShard(shardId, "test")); assertThat("all recoveries should be cancelled", collection.size(), equalTo(0)); } finally { @@ -164,7 +160,7 @@ public void testResetRecovery() throws Exception { assertEquals(currentAsTarget, shard.recoveryStats().currentAsTarget()); try (RecoveriesCollection.RecoveryRef newRecoveryRef = collection.getRecovery(resetRecoveryId)) { shards.recoverReplica(shard, (s, n) -> { - RecoveryTarget newRecoveryTarget = newRecoveryRef.get(RecoveryTarget.class); + RecoveryTarget newRecoveryTarget = newRecoveryRef.get(); assertSame(s, newRecoveryTarget.indexShard()); return newRecoveryTarget; }, false);