Skip to content

Commit 233acbc

Browse files
committed
rename to replication
Signed-off-by: Poojita Raj <[email protected]>
1 parent 47379df commit 233acbc

File tree

14 files changed

+97
-97
lines changed

14 files changed

+97
-97
lines changed

server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@
8080
import org.opensearch.indices.recovery.PeerRecoveryTargetService;
8181
import org.opensearch.indices.recovery.RecoveryListener;
8282
import org.opensearch.indices.recovery.RecoveryState;
83-
import org.opensearch.indices.replication.common.ShardTargetState;
83+
import org.opensearch.indices.replication.common.ReplicationState;
8484
import org.opensearch.repositories.RepositoriesService;
8585
import org.opensearch.search.SearchService;
8686
import org.opensearch.snapshots.SnapshotShardsService;
@@ -745,7 +745,7 @@ public synchronized void handleRecoveryFailure(ShardRouting shardRouting, boolea
745745
failAndRemoveShard(shardRouting, sendShardFailure, "failed recovery", failure, clusterService.state());
746746
}
747747

748-
public void handleRecoveryDone(ShardTargetState state, ShardRouting shardRouting, long primaryTerm) {
748+
public void handleRecoveryDone(ReplicationState state, ShardRouting shardRouting, long primaryTerm) {
749749
RecoveryState RecState = (RecoveryState) state;
750750
shardStateAction.shardStarted(shardRouting, primaryTerm, "after " + RecState.getRecoverySource(), SHARD_STATE_ACTION_LISTENER);
751751
}

server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java

Lines changed: 17 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -69,8 +69,8 @@
6969
import org.opensearch.index.store.Store;
7070
import org.opensearch.index.translog.Translog;
7171
import org.opensearch.index.translog.TranslogCorruptedException;
72-
import org.opensearch.indices.replication.common.ShardTargetCollection;
73-
import org.opensearch.indices.replication.common.ShardTargetCollection.ShardTargetRef;
72+
import org.opensearch.indices.replication.common.ReplicationCollection;
73+
import org.opensearch.indices.replication.common.ReplicationCollection.ReplicationRef;
7474
import org.opensearch.indices.replication.common.ReplicationLuceneIndex;
7575
import org.opensearch.indices.replication.common.ReplicationTimer;
7676
import org.opensearch.tasks.Task;
@@ -125,7 +125,7 @@ public static class Actions {
125125
private final RecoverySettings recoverySettings;
126126
private final ClusterService clusterService;
127127

128-
private final ShardTargetCollection<RecoveryTarget> onGoingRecoveries;
128+
private final ReplicationCollection<RecoveryTarget> onGoingRecoveries;
129129

130130
public PeerRecoveryTargetService(
131131
ThreadPool threadPool,
@@ -137,7 +137,7 @@ public PeerRecoveryTargetService(
137137
this.transportService = transportService;
138138
this.recoverySettings = recoverySettings;
139139
this.clusterService = clusterService;
140-
this.onGoingRecoveries = new ShardTargetCollection<>(logger, threadPool);
140+
this.onGoingRecoveries = new ReplicationCollection<>(logger, threadPool);
141141

142142
transportService.registerRequestHandler(
143143
Actions.FILES_INFO,
@@ -229,7 +229,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi
229229
final TransportRequest requestToSend;
230230
final StartRecoveryRequest startRequest;
231231
final ReplicationTimer timer;
232-
try (ShardTargetCollection.ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.get(recoveryId)) {
232+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.get(recoveryId)) {
233233
if (recoveryRef == null) {
234234
logger.trace("not running recovery with id [{}] - can not find it (probably finished)", recoveryId);
235235
return;
@@ -353,7 +353,7 @@ class PrepareForTranslogOperationsRequestHandler implements TransportRequestHand
353353

354354
@Override
355355
public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, TransportChannel channel, Task task) {
356-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
356+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
357357
final ActionListener<Void> listener = createOrFinishListener(recoveryRef, channel, Actions.PREPARE_TRANSLOG, request);
358358
if (listener == null) {
359359
return;
@@ -368,7 +368,7 @@ class FinalizeRecoveryRequestHandler implements TransportRequestHandler<Recovery
368368

369369
@Override
370370
public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportChannel channel, Task task) throws Exception {
371-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
371+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
372372
final ActionListener<Void> listener = createOrFinishListener(recoveryRef, channel, Actions.FINALIZE, request);
373373
if (listener == null) {
374374
return;
@@ -384,7 +384,7 @@ class HandoffPrimaryContextRequestHandler implements TransportRequestHandler<Rec
384384
@Override
385385
public void messageReceived(final RecoveryHandoffPrimaryContextRequest request, final TransportChannel channel, Task task)
386386
throws Exception {
387-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
387+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
388388
recoveryRef.get().handoffPrimaryContext(request.primaryContext());
389389
}
390390
channel.sendResponse(TransportResponse.Empty.INSTANCE);
@@ -397,7 +397,7 @@ class TranslogOperationsRequestHandler implements TransportRequestHandler<Recove
397397
@Override
398398
public void messageReceived(final RecoveryTranslogOperationsRequest request, final TransportChannel channel, Task task)
399399
throws IOException {
400-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
400+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
401401
final RecoveryTarget recoveryTarget = recoveryRef.get();
402402
final ActionListener<Void> listener = createOrFinishListener(
403403
recoveryRef,
@@ -417,7 +417,7 @@ public void messageReceived(final RecoveryTranslogOperationsRequest request, fin
417417
private void performTranslogOps(
418418
final RecoveryTranslogOperationsRequest request,
419419
final ActionListener<Void> listener,
420-
final ShardTargetRef<RecoveryTarget> recoveryRef
420+
final ReplicationRef<RecoveryTarget> recoveryRef
421421
) {
422422
final RecoveryTarget recoveryTarget = recoveryRef.get();
423423

@@ -433,7 +433,7 @@ private void performTranslogOps(
433433
public void onNewClusterState(ClusterState state) {
434434
threadPool.generic().execute(ActionRunnable.wrap(listener, l -> {
435435
try (
436-
ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(
436+
ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(
437437
request.recoveryId(),
438438
request.shardId()
439439
)
@@ -483,7 +483,7 @@ class FilesInfoRequestHandler implements TransportRequestHandler<RecoveryFilesIn
483483

484484
@Override
485485
public void messageReceived(RecoveryFilesInfoRequest request, TransportChannel channel, Task task) throws Exception {
486-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
486+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
487487
final ActionListener<Void> listener = createOrFinishListener(recoveryRef, channel, Actions.FILES_INFO, request);
488488
if (listener == null) {
489489
return;
@@ -506,7 +506,7 @@ class CleanFilesRequestHandler implements TransportRequestHandler<RecoveryCleanF
506506

507507
@Override
508508
public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel channel, Task task) throws Exception {
509-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
509+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
510510
final ActionListener<Void> listener = createOrFinishListener(recoveryRef, channel, Actions.CLEAN_FILES, request);
511511
if (listener == null) {
512512
return;
@@ -525,7 +525,7 @@ class FileChunkTransportRequestHandler implements TransportRequestHandler<Recove
525525

526526
@Override
527527
public void messageReceived(final RecoveryFileChunkRequest request, TransportChannel channel, Task task) throws Exception {
528-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
528+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.getSafe(request.recoveryId(), request.shardId())) {
529529
final RecoveryTarget recoveryTarget = recoveryRef.get();
530530
final ActionListener<Void> listener = createOrFinishListener(recoveryRef, channel, Actions.FILE_CHUNK, request);
531531
if (listener == null) {
@@ -561,7 +561,7 @@ public void messageReceived(final RecoveryFileChunkRequest request, TransportCha
561561
}
562562

563563
private ActionListener<Void> createOrFinishListener(
564-
final ShardTargetRef<RecoveryTarget> recoveryRef,
564+
final ReplicationRef<RecoveryTarget> recoveryRef,
565565
final TransportChannel channel,
566566
final String action,
567567
final RecoveryTransportRequest request
@@ -570,7 +570,7 @@ private ActionListener<Void> createOrFinishListener(
570570
}
571571

572572
private ActionListener<Void> createOrFinishListener(
573-
final ShardTargetRef<RecoveryTarget> recoveryRef,
573+
final ReplicationRef<RecoveryTarget> recoveryRef,
574574
final TransportChannel channel,
575575
final String action,
576576
final RecoveryTransportRequest request,
@@ -607,7 +607,7 @@ class RecoveryRunner extends AbstractRunnable {
607607

608608
@Override
609609
public void onFailure(Exception e) {
610-
try (ShardTargetRef<RecoveryTarget> recoveryRef = onGoingRecoveries.get(recoveryId)) {
610+
try (ReplicationRef<RecoveryTarget> recoveryRef = onGoingRecoveries.get(recoveryId)) {
611611
if (recoveryRef != null) {
612612
logger.error(() -> new ParameterizedMessage("unexpected error during recovery [{}], failing shard", recoveryId), e);
613613
onGoingRecoveries.fail(

server/src/main/java/org/opensearch/indices/recovery/RecoveryListener.java

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -11,15 +11,15 @@
1111
import org.opensearch.OpenSearchException;
1212
import org.opensearch.cluster.routing.ShardRouting;
1313
import org.opensearch.indices.cluster.IndicesClusterStateService;
14-
import org.opensearch.indices.replication.common.ShardTargetListener;
15-
import org.opensearch.indices.replication.common.ShardTargetState;
14+
import org.opensearch.indices.replication.common.ReplicationListener;
15+
import org.opensearch.indices.replication.common.ReplicationState;
1616

1717
/**
1818
* Listener that runs on changes in Recovery state
1919
*
2020
* @opensearch.internal
2121
*/
22-
public class RecoveryListener implements ShardTargetListener {
22+
public class RecoveryListener implements ReplicationListener {
2323

2424
/**
2525
* ShardRouting with which the shard was created
@@ -44,12 +44,12 @@ public RecoveryListener(
4444
}
4545

4646
@Override
47-
public void onDone(ShardTargetState state) {
47+
public void onDone(ReplicationState state) {
4848
indicesClusterStateService.handleRecoveryDone(state, shardRouting, primaryTerm);
4949
}
5050

5151
@Override
52-
public void onFailure(ShardTargetState state, OpenSearchException e, boolean sendShardFailure) {
52+
public void onFailure(ReplicationState state, OpenSearchException e, boolean sendShardFailure) {
5353
indicesClusterStateService.handleRecoveryFailure(shardRouting, sendShardFailure, e);
5454
}
5555
}

server/src/main/java/org/opensearch/indices/recovery/RecoveryState.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,7 +45,7 @@
4545
import org.opensearch.common.xcontent.XContentBuilder;
4646
import org.opensearch.index.shard.IndexShard;
4747
import org.opensearch.index.shard.ShardId;
48-
import org.opensearch.indices.replication.common.ShardTargetState;
48+
import org.opensearch.indices.replication.common.ReplicationState;
4949
import org.opensearch.indices.replication.common.ReplicationLuceneIndex;
5050
import org.opensearch.indices.replication.common.ReplicationTimer;
5151

@@ -57,7 +57,7 @@
5757
*
5858
* @opensearch.internal
5959
*/
60-
public class RecoveryState implements ShardTargetState, ToXContentFragment, Writeable {
60+
public class RecoveryState implements ReplicationState, ToXContentFragment, Writeable {
6161

6262
/**
6363
* The stage of the recovery state

server/src/main/java/org/opensearch/indices/recovery/RecoveryTarget.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -56,9 +56,9 @@
5656
import org.opensearch.index.store.StoreFileMetadata;
5757
import org.opensearch.index.translog.Translog;
5858
import org.opensearch.indices.replication.common.ReplicationLuceneIndex;
59-
import org.opensearch.indices.replication.common.ShardTarget;
60-
import org.opensearch.indices.replication.common.ShardTargetListener;
61-
import org.opensearch.indices.replication.common.ShardTargetCollection;
59+
import org.opensearch.indices.replication.common.ReplicationTarget;
60+
import org.opensearch.indices.replication.common.ReplicationListener;
61+
import org.opensearch.indices.replication.common.ReplicationCollection;
6262

6363
import java.io.IOException;
6464
import java.nio.file.Path;
@@ -67,11 +67,11 @@
6767

6868
/**
6969
* Represents a recovery where the current node is the target node of the recovery. To track recoveries in a central place, instances of
70-
* this class are created through {@link ShardTargetCollection}.
70+
* this class are created through {@link ReplicationCollection}.
7171
*
7272
* @opensearch.internal
7373
*/
74-
public class RecoveryTarget extends ShardTarget implements RecoveryTargetHandler {
74+
public class RecoveryTarget extends ReplicationTarget implements RecoveryTargetHandler {
7575

7676
private static final String RECOVERY_PREFIX = "recovery.";
7777

@@ -90,7 +90,7 @@ public class RecoveryTarget extends ShardTarget implements RecoveryTargetHandler
9090
* @param sourceNode source node of the recovery where we recover from
9191
* @param listener called when recovery is completed/failed
9292
*/
93-
public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, ShardTargetListener listener) {
93+
public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, ReplicationListener listener) {
9494
super("recovery_status", indexShard, indexShard.recoveryState().getIndex(), listener);
9595
this.cancellableThreads = new CancellableThreads();
9696
this.sourceNode = sourceNode;
Lines changed: 20 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -51,22 +51,22 @@
5151
import java.util.concurrent.ConcurrentMap;
5252

5353
/**
54-
* This class holds a collection of all on going events on the current node (i.e., the node is the target node
55-
* of those events). The class is used to guarantee concurrent semantics such that once a recoveries was done/cancelled/failed
56-
* no other thread will be able to find it. Last, the {@link ShardTargetRef} inner class verifies that recovery temporary files
54+
* This class holds a collection of all on going replication events on the current node (i.e., the node is the target node
55+
* of those events). The class is used to guarantee concurrent semantics such that once an event was done/cancelled/failed
56+
* no other thread will be able to find it. Last, the {@link ReplicationRef} inner class verifies that temporary files
5757
* and store will only be cleared once on going usage is finished.
5858
*
5959
* @opensearch.internal
6060
*/
61-
public class ShardTargetCollection<T extends ShardTarget> {
61+
public class ReplicationCollection<T extends ReplicationTarget> {
6262

6363
/** This is the single source of truth for ongoing target events. If it's not here, it was canceled or done */
6464
private final ConcurrentMap<Long, T> onGoingTargetEvents = ConcurrentCollections.newConcurrentMap();
6565

6666
private final Logger logger;
6767
private final ThreadPool threadPool;
6868

69-
public ShardTargetCollection(Logger logger, ThreadPool threadPool) {
69+
public ReplicationCollection(Logger logger, ThreadPool threadPool) {
7070
this.logger = logger;
7171
this.threadPool = threadPool;
7272
}
@@ -86,7 +86,7 @@ private void startInternal(T target, TimeValue activityTimeout) {
8686
assert existingTarget == null : "found two Target instances with the same id";
8787
logger.trace("started {}", target.description());
8888
threadPool.schedule(
89-
new ShardTargetMonitor(target.getId(), target.lastAccessTime(), activityTimeout),
89+
new ReplicationMonitor(target.getId(), target.lastAccessTime(), activityTimeout),
9090
activityTimeout,
9191
ThreadPool.Names.GENERIC
9292
);
@@ -143,23 +143,23 @@ public T getTarget(long id) {
143143
}
144144

145145
/**
146-
* gets the {@link ShardTarget } for a given id. The ShardTarget returned has it's ref count already incremented
147-
* to make sure it's safe to use. However, you must call {@link ShardTarget#decRef()} when you are done with it, typically
146+
* gets the {@link ReplicationTarget } for a given id. The ShardTarget returned has it's ref count already incremented
147+
* to make sure it's safe to use. However, you must call {@link ReplicationTarget#decRef()} when you are done with it, typically
148148
* by using this method in a try-with-resources clause.
149149
* <p>
150150
* Returns null if recovery is not found
151151
*/
152-
public ShardTargetRef<T> get(long id) {
152+
public ReplicationRef<T> get(long id) {
153153
T status = onGoingTargetEvents.get(id);
154154
if (status != null && status.tryIncRef()) {
155-
return new ShardTargetRef<T>(status);
155+
return new ReplicationRef<T>(status);
156156
}
157157
return null;
158158
}
159159

160160
/** Similar to {@link #get(long)} but throws an exception if no target is found */
161-
public ShardTargetRef<T> getSafe(long id, ShardId shardId) {
162-
ShardTargetRef<T> ref = get(id);
161+
public ReplicationRef<T> getSafe(long id, ShardId shardId) {
162+
ReplicationRef<T> ref = get(id);
163163
if (ref == null) {
164164
throw new IndexShardClosedException(shardId);
165165
}
@@ -236,31 +236,31 @@ public boolean cancelForShard(ShardId shardId, String reason) {
236236
}
237237

238238
/**
239-
* a reference to {@link ShardTarget}, which implements {@link AutoCloseable}. closing the reference
240-
* causes {@link ShardTarget#decRef()} to be called. This makes sure that the underlying resources
241-
* will not be freed until {@link ShardTargetRef#close()} is called.
239+
* a reference to {@link ReplicationTarget}, which implements {@link AutoCloseable}. closing the reference
240+
* causes {@link ReplicationTarget#decRef()} to be called. This makes sure that the underlying resources
241+
* will not be freed until {@link ReplicationRef#close()} is called.
242242
*
243243
* @opensearch.internal
244244
*/
245-
public static class ShardTargetRef<T extends ShardTarget> extends AutoCloseableRefCounted<T> {
245+
public static class ReplicationRef<T extends ReplicationTarget> extends AutoCloseableRefCounted<T> {
246246

247247
/**
248-
* Important: {@link ShardTarget#tryIncRef()} should
248+
* Important: {@link ReplicationTarget#tryIncRef()} should
249249
* be *successfully* called on status before
250250
*/
251-
public ShardTargetRef(T status) {
251+
public ReplicationRef(T status) {
252252
super(status);
253253
status.setLastAccessTime();
254254
}
255255
}
256256

257-
private class ShardTargetMonitor extends AbstractRunnable {
257+
private class ReplicationMonitor extends AbstractRunnable {
258258
private final long id;
259259
private final TimeValue checkInterval;
260260

261261
private volatile long lastSeenAccessTime;
262262

263-
private ShardTargetMonitor(long id, long lastSeenAccessTime, TimeValue checkInterval) {
263+
private ReplicationMonitor(long id, long lastSeenAccessTime, TimeValue checkInterval) {
264264
this.id = id;
265265
this.checkInterval = checkInterval;
266266
this.lastSeenAccessTime = lastSeenAccessTime;

0 commit comments

Comments
 (0)