diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java index 56fb688290002..0657fab55b220 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java @@ -38,6 +38,7 @@ import org.apache.logging.log4j.util.MessageSupplier; import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; +import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.ActionRunnable; import org.opensearch.action.DocWriteRequest; import org.opensearch.action.DocWriteResponse; @@ -46,25 +47,36 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.ChannelActionListener; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.action.support.replication.ReplicationOperation; +import org.opensearch.action.support.replication.ReplicationTask; import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.action.support.replication.TransportWriteAction; import org.opensearch.action.update.UpdateHelper; import org.opensearch.action.update.UpdateRequest; import org.opensearch.action.update.UpdateResponse; +import org.opensearch.client.transport.NoNodeAvailableException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateObserver; import org.opensearch.cluster.action.index.MappingUpdatedAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.AllocationId; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.xcontent.ToXContent; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; @@ -78,17 +90,24 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; +import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndicesService; import org.opensearch.indices.SystemIndices; import org.opensearch.node.NodeClosedException; +import org.opensearch.tasks.Task; +import org.opensearch.tasks.TaskId; import org.opensearch.threadpool.ThreadPool; import org.opensearch.threadpool.ThreadPool.Names; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportRequest; import org.opensearch.transport.TransportRequestOptions; import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.concurrent.Executor; import java.util.function.Consumer; import java.util.function.Function; @@ -115,6 +134,15 @@ public class TransportShardBulkAction extends TransportWriteAction listener = new ChannelActionListener<>(channel, transportPrimaryTermValidationAction, request); + final ShardId shardId = request.getShardId(); + assert shardId != null : "request shardId must be set"; + IndexShard replica = getIndexShard(shardId); + try { + new PrimaryTermValidationReplicaAction(listener, replica, (ReplicationTask) task, request).run(); + } catch (RuntimeException e) { + listener.onFailure(e); + } + } + + /** + * This action is the primary term validation action which is used for doing primary term validation with replicas. + * This is only applicable for TransportShardBulkAction because all writes (delete/update/single write/bulk) + * ultimately boils down to TransportShardBulkAction and isolated primary could continue to acknowledge if it is not + * aware that the primary has changed. This helps achieve the same. More details in java doc of + * {@link TransportShardBulkAction#transportPrimaryTermValidationAction}. + * + * @opensearch.internal + */ + private static final class PrimaryTermValidationReplicaAction extends AbstractRunnable implements ActionListener { + + private final ActionListener onCompletionListener; + private final IndexShard replica; + private final ReplicationTask task; + private final PrimaryTermValidationRequest request; + + public PrimaryTermValidationReplicaAction( + ActionListener onCompletionListener, + IndexShard replica, + ReplicationTask task, + PrimaryTermValidationRequest request + ) { + this.onCompletionListener = onCompletionListener; + this.replica = replica; + this.task = task; + this.request = request; + } + + @Override + public void onResponse(Releasable releasable) { + setPhase(task, "finished"); + onCompletionListener.onResponse(new ReplicaResponse(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED)); + } + + @Override + public void onFailure(Exception e) { + setPhase(task, "failed"); + onCompletionListener.onFailure(e); + } + + @Override + protected void doRun() throws Exception { + setPhase(task, "primary-term-validation"); + final String actualAllocationId = this.replica.routingEntry().allocationId().getId(); + if (actualAllocationId.equals(request.getTargetAllocationID()) == false) { + throw new ShardNotFoundException( + this.replica.shardId(), + "expected allocation id [{}] but found [{}]", + request.getTargetAllocationID(), + actualAllocationId + ); + } + // Check operation primary term against the incoming primary term + // If the request primary term is low, then trigger lister failure + if (request.getPrimaryTerm() < replica.getOperationPrimaryTerm()) { + final String message = String.format( + Locale.ROOT, + "%s operation primary term [%d] is too old (current [%d])", + request.getShardId(), + request.getPrimaryTerm(), + replica.getOperationPrimaryTerm() + ); + onFailure(new IllegalStateException(message)); + } else { + onResponse(null); + } + } + } + + /** + * Primary term validation request sent to a specific allocation id + * + * @opensearch.internal + */ + protected static final class PrimaryTermValidationRequest extends TransportRequest { + + /** + * {@link AllocationId#getId()} of the shard this request is sent to + **/ + private final String targetAllocationID; + private final long primaryTerm; + private final ShardId shardId; + + public PrimaryTermValidationRequest(String targetAllocationID, long primaryTerm, ShardId shardId) { + this.targetAllocationID = Objects.requireNonNull(targetAllocationID); + this.primaryTerm = primaryTerm; + this.shardId = Objects.requireNonNull(shardId); + } + + public PrimaryTermValidationRequest(StreamInput in) throws IOException { + super(in); + targetAllocationID = in.readString(); + primaryTerm = in.readVLong(); + shardId = new ShardId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(targetAllocationID); + out.writeVLong(primaryTerm); + shardId.writeTo(out); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new ReplicationTask(id, type, action, getDescription(), parentTaskId, headers); + } + + public String getTargetAllocationID() { + return targetAllocationID; + } + + public long getPrimaryTerm() { + return primaryTerm; + } + + public ShardId getShardId() { + return shardId; + } + + @Override + public String getDescription() { + return toString(); + } + + @Override + public String toString() { + return "PrimaryTermValidationRequest [" + + shardId + + "] for targetAllocationID [" + + targetAllocationID + + "] with primaryTerm [" + + primaryTerm + + "]"; + } + } + + @Override + protected ReplicationOperation.Replicas primaryTermValidationReplicasProxy() { + return new PrimaryTermValidationProxy(); + } + + /** + * This {@link org.opensearch.action.support.replication.TransportReplicationAction.ReplicasProxy} implementation is + * used for primary term validation and is only relevant for TransportShardBulkAction replication action. + * + * @opensearch.internal + */ + private final class PrimaryTermValidationProxy extends WriteActionReplicasProxy { + + @Override + public void performOn( + ShardRouting replica, + BulkShardRequest request, + long primaryTerm, + long globalCheckpoint, + long maxSeqNoOfUpdatesOrDeletes, + ActionListener listener + ) { + String nodeId = replica.currentNodeId(); + final DiscoveryNode node = clusterService.state().nodes().get(nodeId); + if (node == null) { + listener.onFailure(new NoNodeAvailableException("unknown node [" + nodeId + "]")); + return; + } + final PrimaryTermValidationRequest validationRequest = new PrimaryTermValidationRequest( + replica.allocationId().getId(), + primaryTerm, + replica.shardId() + ); + final ActionListenerResponseHandler handler = new ActionListenerResponseHandler<>( + listener, + ReplicaResponse::new + ); + transportService.sendRequest(node, transportPrimaryTermValidationAction, validationRequest, transportOptions, handler); + } } @Override @@ -193,6 +427,14 @@ protected long primaryOperationSize(BulkShardRequest request) { return request.ramBytesUsed(); } + @Override + public ReplicationMode getReplicationMode(IndexShard indexShard) { + if (indexShard.isRemoteTranslogEnabled()) { + return ReplicationMode.PRIMARY_TERM_VALIDATION; + } + return super.getReplicationMode(indexShard); + } + public static void performOnPrimary( BulkShardRequest request, IndexShard primary, diff --git a/server/src/main/java/org/opensearch/action/support/replication/FanoutReplicationProxy.java b/server/src/main/java/org/opensearch/action/support/replication/FanoutReplicationProxy.java new file mode 100644 index 0000000000000..51b95468d6b25 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/support/replication/FanoutReplicationProxy.java @@ -0,0 +1,45 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.support.replication; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.replication.ReplicationOperation.ReplicaResponse; +import org.opensearch.action.support.replication.ReplicationOperation.Replicas; +import org.opensearch.cluster.routing.ShardRouting; + +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +/** + * This implementation of {@link ReplicationProxy} fans out the replication request to current shard routing if + * it is not the primary and has replication mode as {@link ReplicationMode#FULL_REPLICATION}. + * + * @opensearch.internal + */ +public class FanoutReplicationProxy> extends ReplicationProxy { + + public FanoutReplicationProxy(Replicas replicasProxy) { + super(replicasProxy); + } + + @Override + protected void performOnReplicaProxy( + ReplicationProxyRequest proxyRequest, + ReplicationMode replicationMode, + BiConsumer>, ReplicationProxyRequest> performOnReplicaConsumer + ) { + assert replicationMode == ReplicationMode.FULL_REPLICATION : "FanoutReplicationProxy allows only full replication mode"; + performOnReplicaConsumer.accept(getReplicasProxyConsumer(fullReplicationProxy, proxyRequest), proxyRequest); + } + + @Override + ReplicationMode determineReplicationMode(ShardRouting shardRouting, ShardRouting primaryRouting) { + return shardRouting.isSameAllocation(primaryRouting) == false ? ReplicationMode.FULL_REPLICATION : ReplicationMode.NO_REPLICATION; + } +} diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java new file mode 100644 index 0000000000000..f9b85cc4bd7aa --- /dev/null +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java @@ -0,0 +1,32 @@ +/* + * 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.action.support.replication; + +/** + * The type of replication used for inter-node replication. + * + * @opensearch.internal + */ +public enum ReplicationMode { + /** + * In this mode, a {@code TransportReplicationAction} is fanned out to underlying concerned shard and is replicated logically. + * In short, this mode would replicate the {@link ReplicationRequest} to + * the replica shard along with primary term validation. + */ + FULL_REPLICATION, + /** + * In this mode, a {@code TransportReplicationAction} is fanned out to underlying concerned shard and used for + * primary term validation only. The request is not replicated logically. + */ + PRIMARY_TERM_VALIDATION, + /** + * In this mode, a {@code TransportReplicationAction} does not fan out to the underlying concerned shard. + */ + NO_REPLICATION; +} diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationModeAwareProxy.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationModeAwareProxy.java new file mode 100644 index 0000000000000..26d3b3c2f64ef --- /dev/null +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationModeAwareProxy.java @@ -0,0 +1,75 @@ +/* + * 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.action.support.replication; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.replication.ReplicationOperation.ReplicaResponse; +import org.opensearch.cluster.routing.ShardRouting; + +import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +/** + * This implementation of {@link ReplicationProxy} fans out the replication request to current shard routing basis + * the shard routing's replication mode and replication override policy. + * + * @opensearch.internal + */ +public class ReplicationModeAwareProxy> extends ReplicationProxy { + + private final ReplicationMode replicationModeOverride; + + /** + * This ReplicasProxy is used for performing primary term validation. + */ + private final ReplicationOperation.Replicas primaryTermValidationProxy; + + public ReplicationModeAwareProxy( + ReplicationMode replicationModeOverride, + ReplicationOperation.Replicas replicasProxy, + ReplicationOperation.Replicas primaryTermValidationProxy + ) { + super(replicasProxy); + this.replicationModeOverride = Objects.requireNonNull(replicationModeOverride); + this.primaryTermValidationProxy = Objects.requireNonNull(primaryTermValidationProxy); + } + + @Override + protected void performOnReplicaProxy( + ReplicationProxyRequest proxyRequest, + ReplicationMode replicationMode, + BiConsumer>, ReplicationProxyRequest> performOnReplicaConsumer + ) { + assert replicationMode == ReplicationMode.FULL_REPLICATION || replicationMode == ReplicationMode.PRIMARY_TERM_VALIDATION; + + Consumer> replicasProxyConsumer; + if (replicationMode == ReplicationMode.FULL_REPLICATION) { + replicasProxyConsumer = getReplicasProxyConsumer(fullReplicationProxy, proxyRequest); + } else { + replicasProxyConsumer = getReplicasProxyConsumer(primaryTermValidationProxy, proxyRequest); + } + performOnReplicaConsumer.accept(replicasProxyConsumer, proxyRequest); + } + + @Override + ReplicationMode determineReplicationMode(ShardRouting shardRouting, ShardRouting primaryRouting) { + + // If the current routing is the primary, then it does not need to be replicated + if (shardRouting.isSameAllocation(primaryRouting)) { + return ReplicationMode.NO_REPLICATION; + } + + if (primaryRouting.relocating() && shardRouting.isSameAllocation(primaryRouting.getTargetRelocatingShard())) { + return ReplicationMode.FULL_REPLICATION; + } + + return replicationModeOverride; + } +} diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java index da37eee88a4e0..944729df2ab1e 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java @@ -35,13 +35,14 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.AlreadyClosedException; 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.UnavailableShardsException; import org.opensearch.action.support.ActiveShardCount; import org.opensearch.action.support.RetryableAction; import org.opensearch.action.support.TransportActions; +import org.opensearch.action.support.replication.ReplicationProxyRequest.Builder; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.ShardRouting; @@ -65,6 +66,7 @@ import java.util.Locale; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.LongSupplier; /** @@ -99,6 +101,7 @@ public class ReplicationOperation< private final TimeValue initialRetryBackoffBound; private final TimeValue retryTimeout; private final long primaryTerm; + private final ReplicationProxy replicationProxy; // exposed for tests private final ActionListener resultListener; @@ -117,7 +120,8 @@ public ReplicationOperation( String opType, long primaryTerm, TimeValue initialRetryBackoffBound, - TimeValue retryTimeout + TimeValue retryTimeout, + ReplicationProxy replicationProxy ) { this.replicasProxy = replicas; this.primary = primary; @@ -129,6 +133,7 @@ public ReplicationOperation( this.primaryTerm = primaryTerm; this.initialRetryBackoffBound = initialRetryBackoffBound; this.retryTimeout = retryTimeout; + this.replicationProxy = replicationProxy; } public void execute() throws Exception { @@ -226,20 +231,28 @@ private void performOnReplicas( final ShardRouting primaryRouting = primary.routingEntry(); - for (final ShardRouting shard : replicationGroup.getReplicationTargets()) { - if (shard.isSameAllocation(primaryRouting) == false) { - performOnReplica(shard, replicaRequest, globalCheckpoint, maxSeqNoOfUpdatesOrDeletes, pendingReplicationActions); - } + for (final ShardRouting shardRouting : replicationGroup.getReplicationTargets()) { + ReplicationProxyRequest proxyRequest = new Builder( + shardRouting, + primaryRouting, + globalCheckpoint, + maxSeqNoOfUpdatesOrDeletes, + pendingReplicationActions, + replicaRequest, + primaryTerm + ).build(); + replicationProxy.performOnReplicaProxy(proxyRequest, this::performOnReplica); } } private void performOnReplica( - final ShardRouting shard, - final ReplicaRequest replicaRequest, - final long globalCheckpoint, - final long maxSeqNoOfUpdatesOrDeletes, - final PendingReplicationActions pendingReplicationActions + final Consumer> replicasProxyConsumer, + final ReplicationProxyRequest replicationProxyRequest ) { + final ShardRouting shard = replicationProxyRequest.getShardRouting(); + final ReplicaRequest replicaRequest = replicationProxyRequest.getReplicaRequest(); + final PendingReplicationActions pendingReplicationActions = replicationProxyRequest.getPendingReplicationActions(); + if (logger.isTraceEnabled()) { logger.trace("[{}] sending op [{}] to replica {} for request [{}]", shard.shardId(), opType, shard, replicaRequest); } @@ -309,7 +322,7 @@ public String toString() { @Override public void tryAction(ActionListener listener) { - replicasProxy.performOn(shard, replicaRequest, primaryTerm, globalCheckpoint, maxSeqNoOfUpdatesOrDeletes, listener); + replicasProxyConsumer.accept(listener); } @Override diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxy.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxy.java new file mode 100644 index 0000000000000..20f7b5fc6a586 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxy.java @@ -0,0 +1,93 @@ +/* + * 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.action.support.replication; + +import org.opensearch.action.ActionListener; +import org.opensearch.action.support.replication.ReplicationOperation.ReplicaResponse; +import org.opensearch.action.support.replication.ReplicationOperation.Replicas; +import org.opensearch.cluster.routing.ShardRouting; + +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +/** + * Used for performing any replication operation on replicas. Depending on the implementation, the replication call + * can fanout or stops here. + * + * @opensearch.internal + */ +public abstract class ReplicationProxy> { + + /** + * This is the replicas proxy which is used for full replication. + */ + protected final Replicas fullReplicationProxy; + + public ReplicationProxy(Replicas fullReplicationProxy) { + this.fullReplicationProxy = fullReplicationProxy; + } + + /** + * Depending on the actual implementation and the passed {@link ReplicationMode}, the replication + * mode is determined using which the replication request is performed on the replica or not. + * + * @param proxyRequest replication proxy request + * @param performOnReplicaConsumer performOnReplicasProxy + */ + final void performOnReplicaProxy( + ReplicationProxyRequest proxyRequest, + BiConsumer>, ReplicationProxyRequest> performOnReplicaConsumer + ) { + ReplicationMode replicationMode = determineReplicationMode(proxyRequest.getShardRouting(), proxyRequest.getPrimaryRouting()); + // If the replication modes are 1. Logical replication or 2. Primary term validation, we let the call get performed on the + // replica shard. + if (replicationMode == ReplicationMode.NO_REPLICATION) { + return; + } + performOnReplicaProxy(proxyRequest, replicationMode, performOnReplicaConsumer); + } + + /** + * The implementor can decide the {@code Consumer>} basis the + * proxyRequest and replicationMode. This will ultimately make the calls to replica. + * + * @param proxyRequest replication proxy request + * @param replicationMode replication mode + * @param performOnReplicaConsumer performOnReplicasProxy + */ + protected abstract void performOnReplicaProxy( + ReplicationProxyRequest proxyRequest, + ReplicationMode replicationMode, + BiConsumer>, ReplicationProxyRequest> performOnReplicaConsumer + ); + + /** + * Determines what is the replication mode basis the constructor arguments of the implementation and the current + * replication mode aware shard routing. + * + * @param shardRouting replication mode aware ShardRouting + * @param primaryRouting primary ShardRouting + * @return the determined replication mode. + */ + abstract ReplicationMode determineReplicationMode(final ShardRouting shardRouting, final ShardRouting primaryRouting); + + protected Consumer> getReplicasProxyConsumer( + Replicas proxy, + ReplicationProxyRequest proxyRequest + ) { + return (listener) -> proxy.performOn( + proxyRequest.getShardRouting(), + proxyRequest.getReplicaRequest(), + proxyRequest.getPrimaryTerm(), + proxyRequest.getGlobalCheckpoint(), + proxyRequest.getMaxSeqNoOfUpdatesOrDeletes(), + listener + ); + } +} diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxyRequest.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxyRequest.java new file mode 100644 index 0000000000000..c65e55867f706 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationProxyRequest.java @@ -0,0 +1,128 @@ +/* + * 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.action.support.replication; + +import org.opensearch.cluster.routing.ShardRouting; + +import java.util.Objects; + +/** + * This is proxy wrapper over the replication request whose object can be created using the Builder present inside. + * + * @opensearch.internal + */ +public class ReplicationProxyRequest { + + private final ShardRouting shardRouting; + + private final ShardRouting primaryRouting; + + private final long globalCheckpoint; + + private final long maxSeqNoOfUpdatesOrDeletes; + + private final PendingReplicationActions pendingReplicationActions; + + private final ReplicaRequest replicaRequest; + + private final long primaryTerm; + + private ReplicationProxyRequest( + ShardRouting shardRouting, + ShardRouting primaryRouting, + long globalCheckpoint, + long maxSeqNoOfUpdatesOrDeletes, + PendingReplicationActions pendingReplicationActions, + ReplicaRequest replicaRequest, + long primaryTerm + ) { + this.shardRouting = Objects.requireNonNull(shardRouting); + this.primaryRouting = Objects.requireNonNull(primaryRouting); + this.globalCheckpoint = globalCheckpoint; + this.maxSeqNoOfUpdatesOrDeletes = maxSeqNoOfUpdatesOrDeletes; + this.pendingReplicationActions = Objects.requireNonNull(pendingReplicationActions); + this.replicaRequest = Objects.requireNonNull(replicaRequest); + this.primaryTerm = primaryTerm; + } + + public ShardRouting getShardRouting() { + return shardRouting; + } + + public ShardRouting getPrimaryRouting() { + return primaryRouting; + } + + public long getGlobalCheckpoint() { + return globalCheckpoint; + } + + public long getMaxSeqNoOfUpdatesOrDeletes() { + return maxSeqNoOfUpdatesOrDeletes; + } + + public PendingReplicationActions getPendingReplicationActions() { + return pendingReplicationActions; + } + + public ReplicaRequest getReplicaRequest() { + return replicaRequest; + } + + public long getPrimaryTerm() { + return primaryTerm; + } + + /** + * Builder of ReplicationProxyRequest. + * + * @opensearch.internal + */ + public static class Builder { + + private final ShardRouting shardRouting; + private final ShardRouting primaryRouting; + private final long globalCheckpoint; + private final long maxSeqNoOfUpdatesOrDeletes; + private final PendingReplicationActions pendingReplicationActions; + private final ReplicaRequest replicaRequest; + private final long primaryTerm; + + public Builder( + ShardRouting shardRouting, + ShardRouting primaryRouting, + long globalCheckpoint, + long maxSeqNoOfUpdatesOrDeletes, + PendingReplicationActions pendingReplicationActions, + ReplicaRequest replicaRequest, + long primaryTerm + ) { + this.shardRouting = shardRouting; + this.primaryRouting = primaryRouting; + this.globalCheckpoint = globalCheckpoint; + this.maxSeqNoOfUpdatesOrDeletes = maxSeqNoOfUpdatesOrDeletes; + this.pendingReplicationActions = pendingReplicationActions; + this.replicaRequest = replicaRequest; + this.primaryTerm = primaryTerm; + } + + public ReplicationProxyRequest build() { + return new ReplicationProxyRequest<>( + shardRouting, + primaryRouting, + globalCheckpoint, + maxSeqNoOfUpdatesOrDeletes, + pendingReplicationActions, + replicaRequest, + primaryTerm + ); + } + + } +} diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java index 9d3ee8e49e8c2..e804aa31adb4e 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java @@ -46,6 +46,7 @@ import org.opensearch.action.support.ChannelActionListener; import org.opensearch.action.support.TransportAction; import org.opensearch.action.support.TransportActions; +import org.opensearch.action.support.replication.ReplicationOperation.Replicas; import org.opensearch.client.transport.NoNodeAvailableException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.ClusterStateObserver; @@ -254,10 +255,46 @@ private void runReroutePhase(Task task, Request request, ActionListener newReplicasProxy() { + protected Replicas newReplicasProxy() { return new ReplicasProxy(); } + /** + * This returns a ReplicaProxy that is used for primary term validation. The default behavior is that the control + * must not reach inside the performOn method for ReplicationActions. However, the implementations of the underlying + * class can provide primary term validation proxy that can allow performOn method to make calls to replica. + * + * @return Primary term validation replicas proxy. + */ + protected Replicas primaryTermValidationReplicasProxy() { + return new ReplicasProxy() { + @Override + public void performOn( + ShardRouting replica, + ReplicaRequest request, + long primaryTerm, + long globalCheckpoint, + long maxSeqNoOfUpdatesOrDeletes, + ActionListener listener + ) { + throw new UnsupportedOperationException("Primary term validation is not available for " + actionName); + } + }; + } + + /** + * This method is used for defining the {@link ReplicationMode} override per {@link TransportReplicationAction}. + * + * @param indexShard index shard used to determining the policy. + * @return the overridden replication mode. + */ + public ReplicationMode getReplicationMode(IndexShard indexShard) { + if (indexShard.isRemoteTranslogEnabled()) { + return ReplicationMode.NO_REPLICATION; + } + return ReplicationMode.FULL_REPLICATION; + } + protected abstract Response newResponseInstance(StreamInput in) throws IOException; /** @@ -523,17 +560,24 @@ public void handleException(TransportException exp) { onCompletionListener.onResponse(response); }, e -> handleException(primaryShardReference, e)); + final Replicas replicasProxy = newReplicasProxy(); + final IndexShard indexShard = primaryShardReference.indexShard; + final Replicas termValidationProxy = primaryTermValidationReplicasProxy(); + new ReplicationOperation<>( primaryRequest.getRequest(), primaryShardReference, ActionListener.map(responseListener, result -> result.finalResponseIfSuccessful), - newReplicasProxy(), + replicasProxy, logger, threadPool, actionName, primaryRequest.getPrimaryTerm(), initialRetryBackoffBound, - retryTimeout + retryTimeout, + indexShard.isRemoteTranslogEnabled() + ? new ReplicationModeAwareProxy<>(getReplicationMode(indexShard), replicasProxy, termValidationProxy) + : new FanoutReplicationProxy<>(replicasProxy) ).execute(); } } catch (Exception e) { @@ -813,7 +857,7 @@ protected void doRun() throws Exception { } } - private IndexShard getIndexShard(final ShardId shardId) { + protected IndexShard getIndexShard(final ShardId shardId) { IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex()); return indexService.getShard(shardId.id()); } @@ -1266,7 +1310,7 @@ public static class ReplicaResponse extends ActionResponse implements Replicatio private long localCheckpoint; private long globalCheckpoint; - ReplicaResponse(StreamInput in) throws IOException { + public ReplicaResponse(StreamInput in) throws IOException { super(in); localCheckpoint = in.readZLong(); globalCheckpoint = in.readZLong(); @@ -1321,7 +1365,7 @@ public int hashCode() { * * @opensearch.internal */ - protected class ReplicasProxy implements ReplicationOperation.Replicas { + protected class ReplicasProxy implements Replicas { @Override public void performOn( @@ -1384,7 +1428,9 @@ public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, l */ public static class ConcreteShardRequest extends TransportRequest { - /** {@link AllocationId#getId()} of the shard this request is sent to **/ + /** + * {@link AllocationId#getId()} of the shard this request is sent to + **/ private final String targetAllocationID; private final long primaryTerm; private final R request; @@ -1551,7 +1597,7 @@ public String toString() { * Sets the current phase on the task if it isn't null. Pulled into its own * method because its more convenient that way. */ - static void setPhase(ReplicationTask task, String phase) { + protected static void setPhase(ReplicationTask task, String phase) { if (task != null) { task.setPhase(phase); } diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java index 7fc810808f560..26b15195cd8fc 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java @@ -502,7 +502,7 @@ void run() { * * @opensearch.internal */ - class WriteActionReplicasProxy extends ReplicasProxy { + protected class WriteActionReplicasProxy extends ReplicasProxy { @Override public void failShardIfNeeded( diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index 55d95381923b3..e00fafaa10f13 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -622,7 +622,9 @@ public synchronized void renewPeerRecoveryRetentionLeases() { * If this shard copy is tracked then we got here here via a rolling upgrade from an older version that doesn't * create peer recovery retention leases for every shard copy. */ - assert checkpoints.get(shardRouting.allocationId().getId()).tracked == false + assert (checkpoints.get(shardRouting.allocationId().getId()).tracked + && checkpoints.get(shardRouting.allocationId().getId()).replicated == false) + || checkpoints.get(shardRouting.allocationId().getId()).tracked == false || hasAllPeerRecoveryRetentionLeases == false; return false; } @@ -680,20 +682,29 @@ public static class CheckpointState implements Writeable { */ long globalCheckpoint; /** - * whether this shard is treated as in-sync and thus contributes to the global checkpoint calculation + * When a shard is in-sync, it is capable of being promoted as the primary during a failover. An in-sync shard + * contributes to global checkpoint calculation on the primary iff {@link CheckpointState#replicated} is true. */ boolean inSync; /** - * whether this shard is tracked in the replication group, i.e., should receive document updates from the primary. + * whether this shard is tracked in the replication group and has localTranslog, i.e., should receive document updates + * from the primary. Tracked shards with localTranslog would have corresponding retention leases on the primary shard's + * {@link ReplicationTracker}. */ boolean tracked; - public CheckpointState(long localCheckpoint, long globalCheckpoint, boolean inSync, boolean tracked) { + /** + * Whether the replication requests to the primary are replicated to the concerned shard or not. + */ + boolean replicated; + + public CheckpointState(long localCheckpoint, long globalCheckpoint, boolean inSync, boolean tracked, boolean replicated) { this.localCheckpoint = localCheckpoint; this.globalCheckpoint = globalCheckpoint; this.inSync = inSync; this.tracked = tracked; + this.replicated = replicated; } public CheckpointState(StreamInput in) throws IOException { @@ -701,6 +712,11 @@ public CheckpointState(StreamInput in) throws IOException { this.globalCheckpoint = in.readZLong(); this.inSync = in.readBoolean(); this.tracked = in.readBoolean(); + if (in.getVersion().onOrAfter(Version.CURRENT)) { + this.replicated = in.readBoolean(); + } else { + this.replicated = true; + } } @Override @@ -709,13 +725,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeZLong(globalCheckpoint); out.writeBoolean(inSync); out.writeBoolean(tracked); + out.writeBoolean(replicated); } /** * Returns a full copy of this object */ public CheckpointState copy() { - return new CheckpointState(localCheckpoint, globalCheckpoint, inSync, tracked); + return new CheckpointState(localCheckpoint, globalCheckpoint, inSync, tracked, replicated); } public long getLocalCheckpoint() { @@ -737,6 +754,8 @@ public String toString() { + inSync + ", tracked=" + tracked + + ", replicated=" + + replicated + '}'; } @@ -750,7 +769,8 @@ public boolean equals(Object o) { if (localCheckpoint != that.localCheckpoint) return false; if (globalCheckpoint != that.globalCheckpoint) return false; if (inSync != that.inSync) return false; - return tracked == that.tracked; + if (tracked != that.tracked) return false; + return replicated == that.replicated; } @Override @@ -759,6 +779,7 @@ public int hashCode() { result = 31 * result + Long.hashCode(globalCheckpoint); result = 31 * result + Boolean.hashCode(inSync); result = 31 * result + Boolean.hashCode(tracked); + result = 31 * result + Boolean.hashCode(replicated); return result; } } @@ -774,7 +795,7 @@ public synchronized ObjectLongMap getInSyncGlobalCheckpoints() { final ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(checkpoints.size()); // upper bound on the size checkpoints.entrySet() .stream() - .filter(e -> e.getValue().inSync) + .filter(e -> e.getValue().inSync && e.getValue().replicated) .forEach(e -> globalCheckpoints.put(e.getKey(), e.getValue().globalCheckpoint)); return globalCheckpoints; } @@ -833,6 +854,9 @@ private boolean invariant() { // the current shard is marked as in-sync when the global checkpoint tracker operates in primary mode assert !primaryMode || checkpoints.get(shardAllocationId).inSync; + // the current shard is marked as tracked when the global checkpoint tracker operates in primary mode + assert !primaryMode || checkpoints.get(shardAllocationId).tracked; + // the routing table and replication group is set when the global checkpoint tracker operates in primary mode assert !primaryMode || (routingTable != null && replicationGroup != null) : "primary mode but routing table is " + routingTable @@ -902,7 +926,8 @@ private boolean invariant() { if (primaryMode && indexSettings.isSoftDeleteEnabled() && hasAllPeerRecoveryRetentionLeases) { // all tracked shard copies have a corresponding peer-recovery retention lease for (final ShardRouting shardRouting : routingTable.assignedShards()) { - if (checkpoints.get(shardRouting.allocationId().getId()).tracked) { + final CheckpointState cps = checkpoints.get(shardRouting.allocationId().getId()); + if (cps.tracked && cps.replicated) { assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : "no retention lease for tracked shard [" + shardRouting + "] in " + retentionLeases; assert PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals( @@ -926,7 +951,11 @@ private static long inSyncCheckpointStates( Function reducer ) { final OptionalLong value = reducer.apply( - checkpoints.values().stream().filter(cps -> cps.inSync).mapToLong(function).filter(v -> v != SequenceNumbers.UNASSIGNED_SEQ_NO) + checkpoints.values() + .stream() + .filter(cps -> cps.inSync && cps.replicated) + .mapToLong(function) + .filter(v -> v != SequenceNumbers.UNASSIGNED_SEQ_NO) ); return value.isPresent() ? value.getAsLong() : SequenceNumbers.UNASSIGNED_SEQ_NO; } @@ -1030,6 +1059,11 @@ private ReplicationGroup calculateReplicationGroup() { } else { newVersion = replicationGroup.getVersion() + 1; } + + assert indexSettings().isRemoteTranslogStoreEnabled() + || checkpoints.entrySet().stream().filter(e -> e.getValue().tracked).allMatch(e -> e.getValue().replicated) + : "In absence of remote translog store, all tracked shards must have replication mode as LOGICAL_REPLICATION"; + return new ReplicationGroup( routingTable, checkpoints.entrySet().stream().filter(e -> e.getValue().inSync).map(Map.Entry::getKey).collect(Collectors.toSet()), @@ -1124,10 +1158,11 @@ public synchronized void activatePrimaryMode(final long localCheckpoint) { } /** - * Creates a peer recovery retention lease for this shard, if one does not already exist and this shard is the sole shard copy in the - * replication group. If one does not already exist and yet there are other shard copies in this group then we must have just done - * a rolling upgrade from a version before {@link LegacyESVersion#V_7_4_0}, in which case the missing leases should be created - * asynchronously by the caller using {@link ReplicationTracker#createMissingPeerRecoveryRetentionLeases(ActionListener)}. + * Creates a peer recovery retention lease for this shard, if one does not already exist and this shard is the sole + * shard copy with local translog in the replication group. If one does not already exist and yet there are other + * shard copies in this group then we must have just done a rolling upgrade from a version before {@code LegacyESVersion#V_7_4_0}, + * in which case the missing leases should be created asynchronously by the caller using + * {@link ReplicationTracker#createMissingPeerRecoveryRetentionLeases(ActionListener)}. */ private void addPeerRecoveryRetentionLeaseForSolePrimary() { assert primaryMode; @@ -1136,7 +1171,8 @@ private void addPeerRecoveryRetentionLeaseForSolePrimary() { final ShardRouting primaryShard = routingTable.primaryShard(); final String leaseId = getPeerRecoveryRetentionLeaseId(primaryShard); if (retentionLeases.get(leaseId) == null) { - if (replicationGroup.getReplicationTargets().equals(Collections.singletonList(primaryShard))) { + if (replicationGroup.getReplicationTargets().equals(Collections.singletonList(primaryShard)) + || indexSettings().isRemoteTranslogStoreEnabled()) { assert primaryShard.allocationId().getId().equals(shardAllocationId) : routingTable.assignedShards() + " vs " + shardAllocationId; @@ -1199,6 +1235,12 @@ public synchronized void updateFromClusterManager( boolean removedEntries = checkpoints.keySet() .removeIf(aid -> !inSyncAllocationIds.contains(aid) && !initializingAllocationIds.contains(aid)); + final ShardRouting primary = routingTable.primaryShard(); + final String primaryAllocationId = primary.allocationId().getId(); + final String primaryTargetAllocationId = primary.relocating() + ? primary.getTargetRelocatingShard().allocationId().getId() + : null; + if (primaryMode) { // add new initializingIds that are missing locally. These are fresh shard copies - and not in-sync for (String initializingId : initializingAllocationIds) { @@ -1209,7 +1251,16 @@ public synchronized void updateFromClusterManager( + " as in-sync but it does not exist locally"; final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, inSync, inSync)); + checkpoints.put( + initializingId, + new CheckpointState( + localCheckpoint, + globalCheckpoint, + inSync, + inSync, + isReplicated(initializingId, primaryAllocationId, primaryTargetAllocationId) + ) + ); } } if (removedEntries) { @@ -1219,12 +1270,30 @@ public synchronized void updateFromClusterManager( for (String initializingId : initializingAllocationIds) { final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false)); + checkpoints.put( + initializingId, + new CheckpointState( + localCheckpoint, + globalCheckpoint, + false, + false, + isReplicated(initializingId, primaryAllocationId, primaryTargetAllocationId) + ) + ); } for (String inSyncId : inSyncAllocationIds) { final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true)); + checkpoints.put( + inSyncId, + new CheckpointState( + localCheckpoint, + globalCheckpoint, + true, + true, + isReplicated(inSyncId, primaryAllocationId, primaryTargetAllocationId) + ) + ); } } appliedClusterStateVersion = applyingClusterStateVersion; @@ -1239,6 +1308,26 @@ public synchronized void updateFromClusterManager( assert invariant(); } + /** + * Returns whether the requests are replicated considering the remote translog existence, current/primary/primary target allocation ids. + * + * @param allocationId given allocation id + * @param primaryAllocationId primary allocation id + * @param primaryTargetAllocationId primary target allocation id + * @return the replication mode. + */ + private boolean isReplicated(String allocationId, String primaryAllocationId, String primaryTargetAllocationId) { + // If remote translog is enabled, then returns replication mode checking current allocation id against the + // primary and primary target allocation id. + // If remote translog is enabled, then returns true if given allocation id matches the primary or it's relocation target allocation + // id. + if (indexSettings().isRemoteTranslogStoreEnabled()) { + return (allocationId.equals(primaryAllocationId) || allocationId.equals(primaryTargetAllocationId)); + } + // For other case which is local translog, return true as the requests are replicated to all shards in the replication group. + return true; + } + /** * Notifies the tracker of the current allocation IDs in the cluster state. * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the cluster-manager @@ -1300,13 +1389,14 @@ public synchronized void markAllocationIdAsInSync(final String allocationId, fin updateLocalCheckpoint(allocationId, cps, localCheckpoint); // if it was already in-sync (because of a previously failed recovery attempt), global checkpoint must have been // stuck from advancing - assert !cps.inSync || (cps.localCheckpoint >= getGlobalCheckpoint()) : "shard copy " + assert !cps.inSync || cps.localCheckpoint >= getGlobalCheckpoint() || cps.replicated == false : "shard copy " + allocationId + " that's already in-sync should have a local checkpoint " + cps.localCheckpoint + " that's above the global checkpoint " - + getGlobalCheckpoint(); - if (cps.localCheckpoint < getGlobalCheckpoint()) { + + getGlobalCheckpoint() + + " or it's not replicated"; + if (cps.replicated && cps.localCheckpoint < getGlobalCheckpoint()) { pendingInSync.add(allocationId); try { while (true) { @@ -1377,7 +1467,7 @@ public synchronized void updateLocalCheckpoint(final String allocationId, final logger.trace("marked [{}] as in-sync", allocationId); notifyAllWaiters(); } - if (increasedLocalCheckpoint && pending == false) { + if (cps.replicated && increasedLocalCheckpoint && pending == false) { updateGlobalCheckpointOnPrimary(); } assert invariant(); @@ -1397,7 +1487,7 @@ private static long computeGlobalCheckpoint( return fallback; } for (final CheckpointState cps : localCheckpoints) { - if (cps.inSync) { + if (cps.inSync && cps.replicated) { if (cps.localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO) { // unassigned in-sync replica return fallback; diff --git a/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java new file mode 100644 index 0000000000000..9ffe61208b78c --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java @@ -0,0 +1,231 @@ +/* + * 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.apache.lucene.index.IndexCommit; +import org.apache.lucene.util.SetOnce; +import org.opensearch.action.ActionListener; +import org.opensearch.action.StepListener; +import org.opensearch.action.support.ThreadedActionListener; +import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.concurrent.GatedCloseable; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.engine.RecoveryEngineException; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.seqno.RetentionLease; +import org.opensearch.index.seqno.RetentionLeaseNotFoundException; +import org.opensearch.index.seqno.RetentionLeases; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.translog.Translog; +import org.opensearch.indices.RunUnderPrimaryPermit; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.Transports; + +import java.io.Closeable; +import java.io.IOException; +import java.util.function.Consumer; + +/** + * This handler is used for node-to-node peer recovery when the recovery target is a replica/ or a relocating primary + * shard with translog backed by local store. + * + * @opensearch.internal + */ +public class LocalStorePeerRecoverySourceHandler extends RecoverySourceHandler { + + public LocalStorePeerRecoverySourceHandler( + IndexShard shard, + RecoveryTargetHandler recoveryTarget, + ThreadPool threadPool, + StartRecoveryRequest request, + int fileChunkSizeInBytes, + int maxConcurrentFileChunks, + int maxConcurrentOperations + ) { + super(shard, recoveryTarget, threadPool, request, fileChunkSizeInBytes, maxConcurrentFileChunks, maxConcurrentOperations); + } + + @Override + protected void innerRecoveryToTarget(ActionListener listener, Consumer onFailure) throws IOException { + final SetOnce retentionLeaseRef = new SetOnce<>(); + + RunUnderPrimaryPermit.run(() -> { + final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); + ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); + if (targetShardRouting == null) { + logger.debug( + "delaying recovery of {} as it is not listed as assigned to target node {}", + request.shardId(), + request.targetNode() + ); + throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); + } + assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; + retentionLeaseRef.set(shard.getRetentionLeases().get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting))); + }, shardId + " validating recovery target [" + request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); + final Closeable retentionLock = shard.acquireHistoryRetentionLock(); + resources.add(retentionLock); + final long startingSeqNo; + final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO + && isTargetSameHistory() + && shard.hasCompleteHistoryOperations(PEER_RECOVERY_NAME, request.startingSeqNo()) + && ((retentionLeaseRef.get() == null && shard.useRetentionLeasesInPeerRecovery() == false) + || (retentionLeaseRef.get() != null && retentionLeaseRef.get().retainingSequenceNumber() <= request.startingSeqNo())); + // NB check hasCompleteHistoryOperations when computing isSequenceNumberBasedRecovery, even if there is a retention lease, + // because when doing a rolling upgrade from earlier than 7.4 we may create some leases that are initially unsatisfied. It's + // possible there are other cases where we cannot satisfy all leases, because that's not a property we currently expect to hold. + // Also it's pretty cheap when soft deletes are enabled, and it'd be a disaster if we tried a sequence-number-based recovery + // without having a complete history. + + if (isSequenceNumberBasedRecovery && retentionLeaseRef.get() != null) { + // all the history we need is retained by an existing retention lease, so we do not need a separate retention lock + retentionLock.close(); + logger.trace("history is retained by {}", retentionLeaseRef.get()); + } else { + // all the history we need is retained by the retention lock, obtained before calling shard.hasCompleteHistoryOperations() + // and before acquiring the safe commit we'll be using, so we can be certain that all operations after the safe commit's + // local checkpoint will be retained for the duration of this recovery. + logger.trace("history is retained by retention lock"); + } + + final StepListener sendFileStep = new StepListener<>(); + final StepListener prepareEngineStep = new StepListener<>(); + final StepListener sendSnapshotStep = new StepListener<>(); + final StepListener finalizeStep = new StepListener<>(); + + if (isSequenceNumberBasedRecovery) { + logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); + startingSeqNo = request.startingSeqNo(); + if (retentionLeaseRef.get() == null) { + createRetentionLease(startingSeqNo, ActionListener.map(sendFileStep, ignored -> SendFileResult.EMPTY)); + } else { + sendFileStep.onResponse(SendFileResult.EMPTY); + } + } else { + final GatedCloseable wrappedSafeCommit; + try { + wrappedSafeCommit = acquireSafeCommit(shard); + resources.add(wrappedSafeCommit); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); + } + + // Try and copy enough operations to the recovering peer so that if it is promoted to primary then it has a chance of being + // able to recover other replicas using operations-based recoveries. If we are not using retention leases then we + // conservatively copy all available operations. If we are using retention leases then "enough operations" is just the + // operations from the local checkpoint of the safe commit onwards, because when using soft deletes the safe commit retains + // at least as much history as anything else. The safe commit will often contain all the history retained by the current set + // of retention leases, but this is not guaranteed: an earlier peer recovery from a different primary might have created a + // retention lease for some history that this primary already discarded, since we discard history when the global checkpoint + // advances and not when creating a new safe commit. In any case this is a best-effort thing since future recoveries can + // always fall back to file-based ones, and only really presents a problem if this primary fails before things have settled + // down. + startingSeqNo = Long.parseLong(wrappedSafeCommit.get().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1L; + logger.trace("performing file-based recovery followed by history replay starting at [{}]", startingSeqNo); + + try { + final int estimateNumOps = countNumberOfHistoryOperations(startingSeqNo); + final Releasable releaseStore = acquireStore(shard.store()); + resources.add(releaseStore); + onSendFileStepComplete(sendFileStep, wrappedSafeCommit, releaseStore); + + final StepListener deleteRetentionLeaseStep = new StepListener<>(); + RunUnderPrimaryPermit.run(() -> { + try { + // If the target previously had a copy of this shard then a file-based recovery might move its global + // checkpoint backwards. We must therefore remove any existing retention lease so that we can create a + // new one later on in the recovery. + shard.removePeerRecoveryRetentionLease( + request.targetNode().getId(), + new ThreadedActionListener<>( + logger, + shard.getThreadPool(), + ThreadPool.Names.GENERIC, + deleteRetentionLeaseStep, + false + ) + ); + } catch (RetentionLeaseNotFoundException e) { + logger.debug("no peer-recovery retention lease for " + request.targetAllocationId()); + deleteRetentionLeaseStep.onResponse(null); + } + }, shardId + " removing retention lease for [" + request.targetAllocationId() + "]", shard, cancellableThreads, logger); + + deleteRetentionLeaseStep.whenComplete(ignored -> { + assert Transports.assertNotTransportThread(this + "[phase1]"); + phase1(wrappedSafeCommit.get(), startingSeqNo, () -> estimateNumOps, sendFileStep, false); + }, onFailure); + + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); + } + } + assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo; + + sendFileStep.whenComplete(r -> { + assert Transports.assertNotTransportThread(this + "[prepareTargetForTranslog]"); + // For a sequence based recovery, the target can keep its local translog + prepareTargetForTranslog(countNumberOfHistoryOperations(startingSeqNo), prepareEngineStep); + }, onFailure); + + prepareEngineStep.whenComplete(prepareEngineTime -> { + assert Transports.assertNotTransportThread(this + "[phase2]"); + /* + * add shard to replication group (shard will receive replication requests from this point on) now that engine is open. + * This means that any document indexed into the primary after this will be replicated to this replica as well + * make sure to do this before sampling the max sequence number in the next step, to ensure that we send + * all documents up to maxSeqNo in phase2. + */ + RunUnderPrimaryPermit.run( + () -> shard.initiateTracking(request.targetAllocationId()), + shardId + " initiating tracking of " + request.targetAllocationId(), + shard, + cancellableThreads, + logger + ); + + final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); + if (logger.isTraceEnabled()) { + logger.trace("snapshot translog for recovery; current size is [{}]", countNumberOfHistoryOperations(startingSeqNo)); + } + final Translog.Snapshot phase2Snapshot = shard.newChangesSnapshot( + PEER_RECOVERY_NAME, + startingSeqNo, + Long.MAX_VALUE, + false, + true + ); + resources.add(phase2Snapshot); + retentionLock.close(); + + // we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values + // are at least as high as the corresponding values on the primary when any of these operations were executed on it. + final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); + final long maxSeqNoOfUpdatesOrDeletes = shard.getMaxSeqNoOfUpdatesOrDeletes(); + final RetentionLeases retentionLeases = shard.getRetentionLeases(); + final long mappingVersionOnPrimary = shard.indexSettings().getIndexMetadata().getMappingVersion(); + phase2( + startingSeqNo, + endingSeqNo, + phase2Snapshot, + maxSeenAutoIdTimestamp, + maxSeqNoOfUpdatesOrDeletes, + retentionLeases, + mappingVersionOnPrimary, + sendSnapshotStep + ); + + }, onFailure); + finalizeStepAndCompleteFuture(startingSeqNo, sendSnapshotStep, sendFileStep, prepareEngineStep, onFailure); + } +} diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java index a1cf78920cf7e..8bea14a1a1c86 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoverySourceService.java @@ -378,15 +378,7 @@ private Tuple createRecovery recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime) ); - handler = new RecoverySourceHandler( - shard, - recoveryTarget, - shard.getThreadPool(), - request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), - recoverySettings.getMaxConcurrentFileChunks(), - recoverySettings.getMaxConcurrentOperations() - ); + handler = RecoverySourceHandlerFactory.create(shard, recoveryTarget, request, recoverySettings); return Tuple.tuple(handler, recoveryTarget); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java index 665e79722770e..6259842c282bf 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java @@ -39,7 +39,6 @@ import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.SetOnce; import org.opensearch.LegacyESVersion; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionRunnable; @@ -47,8 +46,6 @@ import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.ThreadedActionListener; import org.opensearch.action.support.replication.ReplicationResponse; -import org.opensearch.cluster.routing.IndexShardRoutingTable; -import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.StopWatch; import org.opensearch.common.concurrent.GatedCloseable; @@ -63,7 +60,6 @@ import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.core.internal.io.IOUtils; import org.opensearch.index.engine.RecoveryEngineException; -import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeaseNotFoundException; import org.opensearch.index.seqno.RetentionLeases; @@ -98,7 +94,7 @@ * RecoverySourceHandler handles the three phases of shard recovery, which is * everything relating to copying the segment files as well as sending translog * operations across the wire once the segments have been copied. - * + *

* Note: There is always one source handler per recovery that handles all the * file and translog transfer. This handler is completely isolated from other recoveries * while the {@link RateLimiter} passed via {@link RecoverySettings} is shared across recoveries @@ -107,25 +103,25 @@ * * @opensearch.internal */ -public class RecoverySourceHandler { +public abstract class RecoverySourceHandler { protected final Logger logger; // Shard that is going to be recovered (the "source") - private final IndexShard shard; - private final int shardId; + protected final IndexShard shard; + protected final int shardId; // Request containing source and target node information - private final StartRecoveryRequest request; + protected final StartRecoveryRequest request; private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; private final int maxConcurrentOperations; private final ThreadPool threadPool; - private final CancellableThreads cancellableThreads = new CancellableThreads(); - private final List resources = new CopyOnWriteArrayList<>(); - private final ListenableFuture future = new ListenableFuture<>(); + protected final CancellableThreads cancellableThreads = new CancellableThreads(); + protected final List resources = new CopyOnWriteArrayList<>(); + protected final ListenableFuture future = new ListenableFuture<>(); public static final String PEER_RECOVERY_NAME = "peer-recovery"; private final SegmentFileTransferHandler transferHandler; - public RecoverySourceHandler( + RecoverySourceHandler( IndexShard shard, RecoveryTargetHandler recoveryTarget, ThreadPool threadPool, @@ -184,251 +180,70 @@ public void recoverToTarget(ActionListener listener) { throw e; }); final Consumer onFailure = e -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[onFailure]"); + assert Transports.assertNotTransportThread(this + "[onFailure]"); IOUtils.closeWhileHandlingException(releaseResources, () -> future.onFailure(e)); }; + innerRecoveryToTarget(listener, onFailure); + } catch (Exception e) { + IOUtils.closeWhileHandlingException(releaseResources, () -> future.onFailure(e)); + } + } - final SetOnce retentionLeaseRef = new SetOnce<>(); + protected abstract void innerRecoveryToTarget(ActionListener listener, Consumer onFailure) + throws IOException; - RunUnderPrimaryPermit.run(() -> { - final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); - ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); - if (targetShardRouting == null) { - logger.debug( - "delaying recovery of {} as it is not listed as assigned to target node {}", - request.shardId(), - request.targetNode() - ); - throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); - } - assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; - retentionLeaseRef.set( - shard.getRetentionLeases().get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting)) - ); - }, - shardId + " validating recovery target [" + request.targetAllocationId() + "] registered ", - shard, - cancellableThreads, - logger + protected void finalizeStepAndCompleteFuture( + long startingSeqNo, + StepListener sendSnapshotStep, + StepListener sendFileStep, + StepListener prepareEngineStep, + Consumer onFailure + ) { + final StepListener finalizeStep = new StepListener<>(); + // Recovery target can trim all operations >= startingSeqNo as we have sent all these operations in the phase 2 + final long trimAboveSeqNo = startingSeqNo - 1; + sendSnapshotStep.whenComplete(r -> finalizeRecovery(r.targetLocalCheckpoint, trimAboveSeqNo, finalizeStep), onFailure); + + finalizeStep.whenComplete(r -> { + final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time + final SendSnapshotResult sendSnapshotResult = sendSnapshotStep.result(); + final SendFileResult sendFileResult = sendFileStep.result(); + final RecoveryResponse response = new RecoveryResponse( + sendFileResult.phase1FileNames, + sendFileResult.phase1FileSizes, + sendFileResult.phase1ExistingFileNames, + sendFileResult.phase1ExistingFileSizes, + sendFileResult.totalSize, + sendFileResult.existingTotalSize, + sendFileResult.took.millis(), + phase1ThrottlingWaitTime, + prepareEngineStep.result().millis(), + sendSnapshotResult.sentOperations, + sendSnapshotResult.tookTime.millis() ); - final Closeable retentionLock = shard.acquireHistoryRetentionLock(); - resources.add(retentionLock); - final long startingSeqNo; - final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO - && isTargetSameHistory() - && shard.hasCompleteHistoryOperations(PEER_RECOVERY_NAME, request.startingSeqNo()) - && ((retentionLeaseRef.get() == null && shard.useRetentionLeasesInPeerRecovery() == false) - || (retentionLeaseRef.get() != null && retentionLeaseRef.get().retainingSequenceNumber() <= request.startingSeqNo())); - // NB check hasCompleteHistoryOperations when computing isSequenceNumberBasedRecovery, even if there is a retention lease, - // because when doing a rolling upgrade from earlier than 7.4 we may create some leases that are initially unsatisfied. It's - // possible there are other cases where we cannot satisfy all leases, because that's not a property we currently expect to hold. - // Also it's pretty cheap when soft deletes are enabled, and it'd be a disaster if we tried a sequence-number-based recovery - // without having a complete history. - - if (isSequenceNumberBasedRecovery && retentionLeaseRef.get() != null) { - // all the history we need is retained by an existing retention lease, so we do not need a separate retention lock - retentionLock.close(); - logger.trace("history is retained by {}", retentionLeaseRef.get()); - } else { - // all the history we need is retained by the retention lock, obtained before calling shard.hasCompleteHistoryOperations() - // and before acquiring the safe commit we'll be using, so we can be certain that all operations after the safe commit's - // local checkpoint will be retained for the duration of this recovery. - logger.trace("history is retained by retention lock"); - } - - final StepListener sendFileStep = new StepListener<>(); - final StepListener prepareEngineStep = new StepListener<>(); - final StepListener sendSnapshotStep = new StepListener<>(); - final StepListener finalizeStep = new StepListener<>(); - - if (isSequenceNumberBasedRecovery) { - logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); - startingSeqNo = request.startingSeqNo(); - if (retentionLeaseRef.get() == null) { - createRetentionLease(startingSeqNo, ActionListener.map(sendFileStep, ignored -> SendFileResult.EMPTY)); - } else { - sendFileStep.onResponse(SendFileResult.EMPTY); - } - } else { - final GatedCloseable wrappedSafeCommit; - try { - wrappedSafeCommit = acquireSafeCommit(shard); - resources.add(wrappedSafeCommit); - } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); - } - - // Try and copy enough operations to the recovering peer so that if it is promoted to primary then it has a chance of being - // able to recover other replicas using operations-based recoveries. If we are not using retention leases then we - // conservatively copy all available operations. If we are using retention leases then "enough operations" is just the - // operations from the local checkpoint of the safe commit onwards, because when using soft deletes the safe commit retains - // at least as much history as anything else. The safe commit will often contain all the history retained by the current set - // of retention leases, but this is not guaranteed: an earlier peer recovery from a different primary might have created a - // retention lease for some history that this primary already discarded, since we discard history when the global checkpoint - // advances and not when creating a new safe commit. In any case this is a best-effort thing since future recoveries can - // always fall back to file-based ones, and only really presents a problem if this primary fails before things have settled - // down. - startingSeqNo = Long.parseLong(wrappedSafeCommit.get().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1L; - logger.trace("performing file-based recovery followed by history replay starting at [{}]", startingSeqNo); - - try { - final int estimateNumOps = countNumberOfHistoryOperations(startingSeqNo); - final Releasable releaseStore = acquireStore(shard.store()); - resources.add(releaseStore); - sendFileStep.whenComplete(r -> IOUtils.close(wrappedSafeCommit, releaseStore), e -> { - try { - IOUtils.close(wrappedSafeCommit, releaseStore); - } catch (final IOException ex) { - logger.warn("releasing snapshot caused exception", ex); - } - }); - - final StepListener deleteRetentionLeaseStep = new StepListener<>(); - RunUnderPrimaryPermit.run(() -> { - try { - // If the target previously had a copy of this shard then a file-based recovery might move its global - // checkpoint backwards. We must therefore remove any existing retention lease so that we can create a - // new one later on in the recovery. - shard.removePeerRecoveryRetentionLease( - request.targetNode().getId(), - new ThreadedActionListener<>( - logger, - shard.getThreadPool(), - ThreadPool.Names.GENERIC, - deleteRetentionLeaseStep, - false - ) - ); - } catch (RetentionLeaseNotFoundException e) { - logger.debug("no peer-recovery retention lease for " + request.targetAllocationId()); - deleteRetentionLeaseStep.onResponse(null); - } - }, shardId + " removing retention lease for [" + request.targetAllocationId() + "]", shard, cancellableThreads, logger); - - deleteRetentionLeaseStep.whenComplete(ignored -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[phase1]"); - phase1(wrappedSafeCommit.get(), startingSeqNo, () -> estimateNumOps, sendFileStep); - }, onFailure); - - } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); - } + try { + future.onResponse(response); + } finally { + IOUtils.close(resources); } - assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo; - - boolean isRecoveringReplicaWithRemoteTxLogEnabledIndex = request.isPrimaryRelocation() == false - && shard.isRemoteTranslogEnabled(); - - if (isRecoveringReplicaWithRemoteTxLogEnabledIndex) { - sendFileStep.whenComplete(r -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[prepareTargetForTranslog]"); - // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(0, prepareEngineStep); - }, onFailure); - - prepareEngineStep.whenComplete(prepareEngineTime -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[phase2]"); - RunUnderPrimaryPermit.run( - () -> shard.initiateTracking(request.targetAllocationId()), - shardId + " initiating tracking of " + request.targetAllocationId(), - shard, - cancellableThreads, - logger - ); - final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - retentionLock.close(); - sendSnapshotStep.onResponse(new SendSnapshotResult(endingSeqNo, 0, TimeValue.ZERO)); - }, onFailure); - } else { - sendFileStep.whenComplete(r -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[prepareTargetForTranslog]"); - // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(countNumberOfHistoryOperations(startingSeqNo), prepareEngineStep); - }, onFailure); - - prepareEngineStep.whenComplete(prepareEngineTime -> { - assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[phase2]"); - /* - * add shard to replication group (shard will receive replication requests from this point on) now that engine is open. - * This means that any document indexed into the primary after this will be replicated to this replica as well - * make sure to do this before sampling the max sequence number in the next step, to ensure that we send - * all documents up to maxSeqNo in phase2. - */ - RunUnderPrimaryPermit.run( - () -> shard.initiateTracking(request.targetAllocationId()), - shardId + " initiating tracking of " + request.targetAllocationId(), - shard, - cancellableThreads, - logger - ); - - final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - if (logger.isTraceEnabled()) { - logger.trace("snapshot translog for recovery; current size is [{}]", countNumberOfHistoryOperations(startingSeqNo)); - } - final Translog.Snapshot phase2Snapshot = shard.newChangesSnapshot( - PEER_RECOVERY_NAME, - startingSeqNo, - Long.MAX_VALUE, - false, - true - ); - resources.add(phase2Snapshot); - retentionLock.close(); - - // we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values - // are at least as high as the corresponding values on the primary when any of these operations were executed on it. - final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); - final long maxSeqNoOfUpdatesOrDeletes = shard.getMaxSeqNoOfUpdatesOrDeletes(); - final RetentionLeases retentionLeases = shard.getRetentionLeases(); - final long mappingVersionOnPrimary = shard.indexSettings().getIndexMetadata().getMappingVersion(); - phase2( - startingSeqNo, - endingSeqNo, - phase2Snapshot, - maxSeenAutoIdTimestamp, - maxSeqNoOfUpdatesOrDeletes, - retentionLeases, - mappingVersionOnPrimary, - sendSnapshotStep - ); + }, onFailure); + } - }, onFailure); + protected void onSendFileStepComplete( + StepListener sendFileStep, + GatedCloseable wrappedSafeCommit, + Releasable releaseStore + ) { + sendFileStep.whenComplete(r -> IOUtils.close(wrappedSafeCommit, releaseStore), e -> { + try { + IOUtils.close(wrappedSafeCommit, releaseStore); + } catch (final IOException ex) { + logger.warn("releasing snapshot caused exception", ex); } - - // Recovery target can trim all operations >= startingSeqNo as we have sent all these operations in the phase 2 - final long trimAboveSeqNo = startingSeqNo - 1; - sendSnapshotStep.whenComplete(r -> finalizeRecovery(r.targetLocalCheckpoint, trimAboveSeqNo, finalizeStep), onFailure); - - finalizeStep.whenComplete(r -> { - final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time - final SendSnapshotResult sendSnapshotResult = sendSnapshotStep.result(); - final SendFileResult sendFileResult = sendFileStep.result(); - final RecoveryResponse response = new RecoveryResponse( - sendFileResult.phase1FileNames, - sendFileResult.phase1FileSizes, - sendFileResult.phase1ExistingFileNames, - sendFileResult.phase1ExistingFileSizes, - sendFileResult.totalSize, - sendFileResult.existingTotalSize, - sendFileResult.took.millis(), - phase1ThrottlingWaitTime, - prepareEngineStep.result().millis(), - sendSnapshotResult.sentOperations, - sendSnapshotResult.tookTime.millis() - ); - try { - future.onResponse(response); - } finally { - IOUtils.close(resources); - } - }, onFailure); - } catch (Exception e) { - IOUtils.closeWhileHandlingException(releaseResources, () -> future.onFailure(e)); - } + }); } - private boolean isTargetSameHistory() { + protected boolean isTargetSameHistory() { final String targetHistoryUUID = request.metadataSnapshot().getHistoryUUID(); assert targetHistoryUUID != null : "incoming target history missing"; return targetHistoryUUID.equals(shard.getHistoryUUID()); @@ -436,10 +251,11 @@ private boolean isTargetSameHistory() { /** * Counts the number of history operations from the starting sequence number - * @param startingSeqNo the starting sequence number to count; included - * @return number of history operations + * + * @param startingSeqNo the starting sequence number to count; included + * @return number of history operations */ - private int countNumberOfHistoryOperations(long startingSeqNo) throws IOException { + protected int countNumberOfHistoryOperations(long startingSeqNo) throws IOException { return shard.countNumberOfHistoryOperations(PEER_RECOVERY_NAME, startingSeqNo, Long.MAX_VALUE); } @@ -447,7 +263,7 @@ private int countNumberOfHistoryOperations(long startingSeqNo) throws IOExceptio * Increases the store reference and returns a {@link Releasable} that will decrease the store reference using the generic thread pool. * We must never release the store using an interruptible thread as we can risk invalidating the node lock. */ - private Releasable acquireStore(Store store) { + protected Releasable acquireStore(Store store) { store.incRef(); return Releasables.releaseOnce(() -> runWithGenericThreadPool(store::decRef)); } @@ -457,7 +273,7 @@ private Releasable acquireStore(Store store) { * with the file systems due to interrupt (see {@link org.apache.lucene.store.NIOFSDirectory} javadocs for more detail). * This method acquires a safe commit and wraps it to make sure that it will be released using the generic thread pool. */ - private GatedCloseable acquireSafeCommit(IndexShard shard) { + protected GatedCloseable acquireSafeCommit(IndexShard shard) { final GatedCloseable wrappedSafeCommit = shard.acquireSafeIndexCommit(); final AtomicBoolean closed = new AtomicBoolean(false); return new GatedCloseable<>(wrappedSafeCommit.get(), () -> { @@ -531,7 +347,13 @@ static final class SendFileResult { * segments that are missing. Only segments that have the same size and * checksum can be reused */ - void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, ActionListener listener) { + void phase1( + IndexCommit snapshot, + long startingSeqNo, + IntSupplier translogOps, + ActionListener listener, + boolean skipCreateRetentionLeaseStep + ) { cancellableThreads.checkForCancel(); final Store store = shard.store(); try { @@ -629,7 +451,12 @@ void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, A listener::onFailure ); - sendFilesStep.whenComplete(r -> createRetentionLease(startingSeqNo, createRetentionLeaseStep), listener::onFailure); + // When doing peer recovery of remote store enabled replica, retention leases are not required. + if (skipCreateRetentionLeaseStep) { + sendFilesStep.whenComplete(r -> createRetentionLeaseStep.onResponse(null), listener::onFailure); + } else { + sendFilesStep.whenComplete(r -> createRetentionLease(startingSeqNo, createRetentionLeaseStep), listener::onFailure); + } createRetentionLeaseStep.whenComplete(retentionLease -> { final long lastKnownGlobalCheckpoint = shard.getLastKnownGlobalCheckpoint(); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java new file mode 100644 index 0000000000000..ea13ca18bbfca --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandlerFactory.java @@ -0,0 +1,49 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.recovery; + +import org.opensearch.index.shard.IndexShard; + +/** + * Factory that supplies {@link RecoverySourceHandler}. + * + * @opensearch.internal + */ +public class RecoverySourceHandlerFactory { + + public static RecoverySourceHandler create( + IndexShard shard, + RecoveryTargetHandler recoveryTarget, + StartRecoveryRequest request, + RecoverySettings recoverySettings + ) { + boolean isReplicaRecoveryWithRemoteTranslog = request.isPrimaryRelocation() == false && shard.isRemoteTranslogEnabled(); + if (isReplicaRecoveryWithRemoteTranslog) { + return new RemoteStorePeerRecoverySourceHandler( + shard, + recoveryTarget, + shard.getThreadPool(), + request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + recoverySettings.getMaxConcurrentFileChunks(), + recoverySettings.getMaxConcurrentOperations() + ); + } else { + return new LocalStorePeerRecoverySourceHandler( + shard, + recoveryTarget, + shard.getThreadPool(), + request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + recoverySettings.getMaxConcurrentFileChunks(), + recoverySettings.getMaxConcurrentOperations() + ); + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java new file mode 100644 index 0000000000000..ff218ef71e397 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java @@ -0,0 +1,103 @@ +/* + * 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.apache.lucene.index.IndexCommit; +import org.opensearch.action.ActionListener; +import org.opensearch.action.StepListener; +import org.opensearch.common.concurrent.GatedCloseable; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.engine.RecoveryEngineException; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.RunUnderPrimaryPermit; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.Transports; + +import java.io.IOException; +import java.util.function.Consumer; + +/** + * This handler is used when peer recovery target is a remote store enabled replica. + * + * @opensearch.internal + */ +public class RemoteStorePeerRecoverySourceHandler extends RecoverySourceHandler { + + public RemoteStorePeerRecoverySourceHandler( + IndexShard shard, + RecoveryTargetHandler recoveryTarget, + ThreadPool threadPool, + StartRecoveryRequest request, + int fileChunkSizeInBytes, + int maxConcurrentFileChunks, + int maxConcurrentOperations + ) { + super(shard, recoveryTarget, threadPool, request, fileChunkSizeInBytes, maxConcurrentFileChunks, maxConcurrentOperations); + } + + @Override + protected void innerRecoveryToTarget(ActionListener listener, Consumer onFailure) throws IOException { + // A replica of an index with remote translog does not require the translogs locally and keeps receiving the + // updated segments file on refresh, flushes, and merges. In recovery, here, only file-based recovery is performed + // and there is no translog replay done. + + final StepListener sendFileStep = new StepListener<>(); + final StepListener prepareEngineStep = new StepListener<>(); + final StepListener sendSnapshotStep = new StepListener<>(); + + // It is always file based recovery while recovering replicas which are not relocating primary where the + // underlying indices are backed by remote store for storing segments and translog + + final GatedCloseable wrappedSafeCommit; + try { + wrappedSafeCommit = acquireSafeCommit(shard); + resources.add(wrappedSafeCommit); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); + } + + final long startingSeqNo = Long.parseLong(wrappedSafeCommit.get().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1L; + logger.trace("performing file-based recovery followed by history replay starting at [{}]", startingSeqNo); + + try { + final Releasable releaseStore = acquireStore(shard.store()); + resources.add(releaseStore); + onSendFileStepComplete(sendFileStep, wrappedSafeCommit, releaseStore); + + assert Transports.assertNotTransportThread(this + "[phase1]"); + phase1(wrappedSafeCommit.get(), startingSeqNo, () -> 0, sendFileStep, true); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); + } + assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo; + + sendFileStep.whenComplete(r -> { + assert Transports.assertNotTransportThread(this + "[prepareTargetForTranslog]"); + // For a sequence based recovery, the target can keep its local translog + prepareTargetForTranslog(0, prepareEngineStep); + }, onFailure); + + prepareEngineStep.whenComplete(prepareEngineTime -> { + assert Transports.assertNotTransportThread(this + "[phase2]"); + RunUnderPrimaryPermit.run( + () -> shard.initiateTracking(request.targetAllocationId()), + shardId + " initiating tracking of " + request.targetAllocationId(), + shard, + cancellableThreads, + logger + ); + final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); + sendSnapshotStep.onResponse(new SendSnapshotResult(endingSeqNo, 0, TimeValue.ZERO)); + }, onFailure); + + finalizeStepAndCompleteFuture(startingSeqNo, sendSnapshotStep, sendFileStep, prepareEngineStep, onFailure); + } +} diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java index cc51082639cdb..e7b53874c9d1b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointAction.java @@ -40,6 +40,8 @@ import java.io.IOException; import java.util.Objects; +import org.opensearch.action.support.replication.ReplicationMode; + /** * Replication action responsible for publishing checkpoint to a replica shard. * @@ -93,6 +95,14 @@ protected void doExecute(Task task, PublishCheckpointRequest request, ActionList assert false : "use PublishCheckpointAction#publish"; } + @Override + public ReplicationMode getReplicationMode(IndexShard indexShard) { + if (indexShard.isRemoteTranslogEnabled()) { + return ReplicationMode.FULL_REPLICATION; + } + return super.getReplicationMode(indexShard); + } + /** * Publish checkpoint request to shard */ diff --git a/server/src/test/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 72c7b5168fe15..000dac92506f6 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -32,11 +32,18 @@ package org.opensearch.action.admin.indices.close; import org.apache.lucene.util.SetOnce; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.mockito.ArgumentCaptor; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.FanoutReplicationProxy; import org.opensearch.action.support.replication.PendingReplicationActions; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.ReplicationOperation; import org.opensearch.action.support.replication.ReplicationResponse; import org.opensearch.action.support.replication.TransportReplicationAction; @@ -65,11 +72,6 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.mockito.ArgumentCaptor; import java.util.Collections; import java.util.List; @@ -77,21 +79,21 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import static org.mockito.Mockito.doNothing; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; -import static org.opensearch.test.ClusterServiceUtils.setState; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; public class TransportVerifyShardBeforeCloseActionTests extends OpenSearchTestCase { @@ -290,7 +292,8 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { "test", primaryTerm, TimeValue.timeValueMillis(20), - TimeValue.timeValueSeconds(60) + TimeValue.timeValueSeconds(60), + new FanoutReplicationProxy<>(proxy) ); operation.execute(); @@ -323,6 +326,32 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { assertThat(shardInfo.getSuccessful(), equalTo(1 + nbReplicas - unavailableShards.size())); } + public void testGetReplicationModeWithRemoteTranslog() { + TransportVerifyShardBeforeCloseAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + TransportVerifyShardBeforeCloseAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportVerifyShardBeforeCloseAction createAction() { + return new TransportVerifyShardBeforeCloseAction( + Settings.EMPTY, + mock(TransportService.class), + clusterService, + mock(IndicesService.class), + mock(ThreadPool.class), + mock(ShardStateAction.class), + mock(ActionFilters.class) + ); + } + private static ReplicationOperation.Primary< TransportVerifyShardBeforeCloseAction.ShardRequest, diff --git a/server/src/test/java/org/opensearch/action/admin/indices/flush/TransportShardFlushActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/flush/TransportShardFlushActionTests.java new file mode 100644 index 0000000000000..09215088bd04b --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/flush/TransportShardFlushActionTests.java @@ -0,0 +1,56 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.flush; + +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportShardFlushActionTests extends OpenSearchTestCase { + + public void testGetReplicationModeWithRemoteTranslog() { + TransportShardFlushAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + TransportShardFlushAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportShardFlushAction createAction() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return new TransportShardFlushAction( + Settings.EMPTY, + mock(TransportService.class), + clusterService, + mock(IndicesService.class), + mock(ThreadPool.class), + mock(ShardStateAction.class), + mock(ActionFilters.class) + ); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockActionTests.java new file mode 100644 index 0000000000000..8c4a6c023f9a5 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockActionTests.java @@ -0,0 +1,56 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.readonly; + +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportVerifyShardIndexBlockActionTests extends OpenSearchTestCase { + + public void testGetReplicationModeWithRemoteTranslog() { + TransportVerifyShardIndexBlockAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + TransportVerifyShardIndexBlockAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportVerifyShardIndexBlockAction createAction() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return new TransportVerifyShardIndexBlockAction( + Settings.EMPTY, + mock(TransportService.class), + clusterService, + mock(IndicesService.class), + mock(ThreadPool.class), + mock(ShardStateAction.class), + mock(ActionFilters.class) + ); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/refresh/TransportShardRefreshActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/refresh/TransportShardRefreshActionTests.java new file mode 100644 index 0000000000000..b2eee904bad38 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/refresh/TransportShardRefreshActionTests.java @@ -0,0 +1,56 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.refresh; + +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.cluster.action.shard.ShardStateAction; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TransportShardRefreshActionTests extends OpenSearchTestCase { + + public void testGetReplicationModeWithRemoteTranslog() { + TransportShardRefreshAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + TransportShardRefreshAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportShardRefreshAction createAction() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return new TransportShardRefreshAction( + Settings.EMPTY, + mock(TransportService.class), + clusterService, + mock(IndicesService.class), + mock(ThreadPool.class), + mock(ShardStateAction.class), + mock(ActionFilters.class) + ); + } +} diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java index b1fa20307a12b..2aff8f6bfc6ab 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportShardBulkActionTests.java @@ -43,18 +43,32 @@ import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.WriteRequest.RefreshPolicy; +import org.opensearch.action.support.replication.ReplicationMode; +import org.opensearch.action.support.replication.ReplicationTask; +import org.opensearch.action.support.replication.TransportReplicationAction.ReplicaResponse; import org.opensearch.action.support.replication.TransportWriteAction.WritePrimaryResult; import org.opensearch.action.update.UpdateHelper; import org.opensearch.action.update.UpdateRequest; import org.opensearch.action.update.UpdateResponse; import org.opensearch.client.Requests; +import org.opensearch.cluster.action.index.MappingUpdatedAction; +import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.AllocationId; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.lucene.uid.Versions; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.OpenSearchRejectedExecutionException; +import org.opensearch.index.Index; +import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.IndexingPressureService; import org.opensearch.index.VersionType; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.VersionConflictEngineException; @@ -62,14 +76,22 @@ import org.opensearch.index.mapper.Mapping; import org.opensearch.index.mapper.MetadataFieldMapper; import org.opensearch.index.mapper.RootObjectMapper; +import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.shard.ShardId; +import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.index.translog.Translog; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.SystemIndices; import org.opensearch.rest.RestStatus; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.threadpool.ThreadPool.Names; +import org.opensearch.transport.TestTransportChannel; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportResponse; +import org.opensearch.transport.TransportService; import java.io.IOException; import java.util.Collections; @@ -85,6 +107,7 @@ import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyBoolean; import static org.mockito.Mockito.anyLong; @@ -1030,6 +1053,161 @@ public void testForceExecutionOnRejectionAfterMappingUpdate() throws Exception { } } + public void testHandlePrimaryTermValidationRequestWithDifferentAllocationId() { + + final String aId = "test-allocation-id"; + final ShardId shardId = new ShardId("test", "_na_", 0); + final ReplicationTask task = createReplicationTask(); + PlainActionFuture listener = new PlainActionFuture<>(); + TransportShardBulkAction action = new TransportShardBulkAction( + Settings.EMPTY, + mock(TransportService.class), + mockClusterService(), + mockIndicesService(aId, 1L), + threadPool, + mock(ShardStateAction.class), + mock(MappingUpdatedAction.class), + mock(UpdateHelper.class), + mock(ActionFilters.class), + mock(IndexingPressureService.class), + mock(SystemIndices.class) + ); + action.handlePrimaryTermValidationRequest( + new TransportShardBulkAction.PrimaryTermValidationRequest(aId + "-1", 1, shardId), + createTransportChannel(listener), + task + ); + assertThrows(ShardNotFoundException.class, listener::actionGet); + assertNotNull(task.getPhase()); + assertEquals("failed", task.getPhase()); + } + + public void testHandlePrimaryTermValidationRequestWithOlderPrimaryTerm() { + + final String aId = "test-allocation-id"; + final ShardId shardId = new ShardId("test", "_na_", 0); + final ReplicationTask task = createReplicationTask(); + PlainActionFuture listener = new PlainActionFuture<>(); + TransportShardBulkAction action = new TransportShardBulkAction( + Settings.EMPTY, + mock(TransportService.class), + mockClusterService(), + mockIndicesService(aId, 2L), + threadPool, + mock(ShardStateAction.class), + mock(MappingUpdatedAction.class), + mock(UpdateHelper.class), + mock(ActionFilters.class), + mock(IndexingPressureService.class), + mock(SystemIndices.class) + ); + action.handlePrimaryTermValidationRequest( + new TransportShardBulkAction.PrimaryTermValidationRequest(aId, 1, shardId), + createTransportChannel(listener), + task + ); + assertThrows(IllegalStateException.class, listener::actionGet); + assertNotNull(task.getPhase()); + assertEquals("failed", task.getPhase()); + } + + public void testHandlePrimaryTermValidationRequestSuccess() { + + final String aId = "test-allocation-id"; + final ShardId shardId = new ShardId("test", "_na_", 0); + final ReplicationTask task = createReplicationTask(); + PlainActionFuture listener = new PlainActionFuture<>(); + TransportShardBulkAction action = new TransportShardBulkAction( + Settings.EMPTY, + mock(TransportService.class), + mockClusterService(), + mockIndicesService(aId, 1L), + threadPool, + mock(ShardStateAction.class), + mock(MappingUpdatedAction.class), + mock(UpdateHelper.class), + mock(ActionFilters.class), + mock(IndexingPressureService.class), + mock(SystemIndices.class) + ); + action.handlePrimaryTermValidationRequest( + new TransportShardBulkAction.PrimaryTermValidationRequest(aId, 1, shardId), + createTransportChannel(listener), + task + ); + assertTrue(listener.actionGet() instanceof ReplicaResponse); + assertEquals(SequenceNumbers.NO_OPS_PERFORMED, ((ReplicaResponse) listener.actionGet()).localCheckpoint()); + assertEquals(SequenceNumbers.NO_OPS_PERFORMED, ((ReplicaResponse) listener.actionGet()).globalCheckpoint()); + assertNotNull(task.getPhase()); + assertEquals("finished", task.getPhase()); + } + + public void testGetReplicationModeWithRemoteTranslog() { + TransportShardBulkAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.PRIMARY_TERM_VALIDATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + TransportShardBulkAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportShardBulkAction createAction() { + return new TransportShardBulkAction( + Settings.EMPTY, + mock(TransportService.class), + mockClusterService(), + mock(IndicesService.class), + threadPool, + mock(ShardStateAction.class), + mock(MappingUpdatedAction.class), + mock(UpdateHelper.class), + mock(ActionFilters.class), + mock(IndexingPressureService.class), + mock(SystemIndices.class) + ); + } + + private ClusterService mockClusterService() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return clusterService; + } + + private IndicesService mockIndicesService(String aId, long primaryTerm) { + // Mock few of the required classes + IndicesService indicesService = mock(IndicesService.class); + IndexService indexService = mock(IndexService.class); + IndexShard indexShard = mock(IndexShard.class); + when(indicesService.indexServiceSafe(any(Index.class))).thenReturn(indexService); + when(indexService.getShard(anyInt())).thenReturn(indexShard); + when(indexShard.getOperationPrimaryTerm()).thenReturn(primaryTerm); + + // Mock routing entry, allocation id + AllocationId allocationId = mock(AllocationId.class); + ShardRouting shardRouting = mock(ShardRouting.class); + when(indexShard.routingEntry()).thenReturn(shardRouting); + when(shardRouting.allocationId()).thenReturn(allocationId); + when(allocationId.getId()).thenReturn(aId); + return indicesService; + } + + private ReplicationTask createReplicationTask() { + return new ReplicationTask(0, null, null, null, null, null); + } + + /** + * Transport channel that is needed for replica operation testing. + */ + private TransportChannel createTransportChannel(final PlainActionFuture listener) { + return new TestTransportChannel(listener); + } + private void randomlySetIgnoredPrimaryResponse(BulkItemRequest primaryRequest) { if (randomBoolean()) { // add a response to the request and thereby check that it is ignored for the primary. diff --git a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java index 2ebca16519258..d6d944b5b9b45 100644 --- a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java @@ -31,10 +31,14 @@ package org.opensearch.action.resync; +import org.junit.AfterClass; +import org.junit.BeforeClass; import org.opensearch.Version; import org.opensearch.action.ActionListener; import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.PendingReplicationActions; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.block.ClusterBlocks; @@ -47,6 +51,7 @@ import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.lease.Releasable; import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.index.Index; @@ -65,31 +70,31 @@ import org.opensearch.test.transport.MockTransportService; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; import org.opensearch.transport.nio.MockNioTransport; -import org.junit.AfterClass; -import org.junit.BeforeClass; import java.nio.charset.Charset; import java.util.Collections; import java.util.HashSet; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; -import static org.opensearch.test.ClusterServiceUtils.setState; -import static org.opensearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.eq; import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; +import static org.opensearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; public class TransportResyncReplicationActionTests extends OpenSearchTestCase { @@ -156,23 +161,26 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { final AtomicInteger acquiredPermits = new AtomicInteger(); final IndexShard indexShard = mock(IndexShard.class); + final PendingReplicationActions replicationActions = new PendingReplicationActions(shardId, threadPool); when(indexShard.indexSettings()).thenReturn(new IndexSettings(indexMetadata, Settings.EMPTY)); when(indexShard.shardId()).thenReturn(shardId); when(indexShard.routingEntry()).thenReturn(primaryShardRouting); when(indexShard.getPendingPrimaryTerm()).thenReturn(primaryTerm); when(indexShard.getOperationPrimaryTerm()).thenReturn(primaryTerm); when(indexShard.getActiveOperationsCount()).then(i -> acquiredPermits.get()); + when(indexShard.getPendingReplicationActions()).thenReturn(replicationActions); doAnswer(invocation -> { ActionListener callback = (ActionListener) invocation.getArguments()[0]; acquiredPermits.incrementAndGet(); callback.onResponse(acquiredPermits::decrementAndGet); return null; }).when(indexShard).acquirePrimaryOperationPermit(any(ActionListener.class), anyString(), any(), eq(true)); + Set trackedAllocationIds = shardRoutingTable.getAllAllocationIds(); when(indexShard.getReplicationGroup()).thenReturn( new ReplicationGroup( shardRoutingTable, clusterService.state().metadata().index(index).inSyncAllocationIds(shardId.id()), - shardRoutingTable.getAllAllocationIds(), + trackedAllocationIds, 0 ) ); @@ -217,4 +225,35 @@ public void testResyncDoesNotBlockOnPrimaryAction() throws Exception { } } } + + public void testGetReplicationModeWithRemoteTranslog() { + final TransportResyncReplicationAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final TransportResyncReplicationAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private TransportResyncReplicationAction createAction() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return new TransportResyncReplicationAction( + Settings.EMPTY, + mock(TransportService.class), + clusterService, + mock(IndicesService.class), + threadPool, + mock(ShardStateAction.class), + new ActionFilters(new HashSet<>()), + mock(IndexingPressureService.class), + new SystemIndices(emptyMap()) + ); + } } diff --git a/server/src/test/java/org/opensearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/opensearch/action/support/replication/ReplicationOperationTests.java index 8a4cdfc953bf8..c5d4f3326746d 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/ReplicationOperationTests.java @@ -45,6 +45,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.AllocationId; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; @@ -80,14 +81,18 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; +import static org.opensearch.action.support.replication.ReplicationOperation.RetryOnPrimaryException; +import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; public class ReplicationOperationTests extends OpenSearchTestCase { @@ -157,7 +162,14 @@ public void testReplication() throws Exception { final TestReplicaProxy replicasProxy = new TestReplicaProxy(simulatedFailures); final TestPrimary primary = new TestPrimary(primaryShard, () -> replicationGroup, threadPool); - final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy, primaryTerm); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + replicasProxy, + primaryTerm, + new FanoutReplicationProxy<>(replicasProxy) + ); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); assertThat(request.processedOnReplicas, equalTo(expectedReplicas)); @@ -179,6 +191,199 @@ public void testReplication() throws Exception { assertThat(primary.knownGlobalCheckpoints, equalTo(replicasProxy.generatedGlobalCheckpoints)); } + public void testReplicationWithRemoteTranslogEnabled() throws Exception { + Set initializingIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> initializingIds.add(AllocationId.newInitializing())); + Set activeIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> activeIds.add(AllocationId.newInitializing())); + + AllocationId primaryId = activeIds.iterator().next(); + + ShardId shardId = new ShardId("test", "_na_", 0); + IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId); + final ShardRouting primaryShard = newShardRouting( + shardId, + nodeIdFromAllocationId(primaryId), + null, + true, + ShardRoutingState.STARTED, + primaryId + ); + initializingIds.forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.INITIALIZING, aId)) + ); + activeIds.stream() + .filter(aId -> !aId.equals(primaryId)) + .forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.STARTED, aId)) + ); + builder.addShard(primaryShard); + IndexShardRoutingTable routingTable = builder.build(); + + Set inSyncAllocationIds = activeIds.stream().map(AllocationId::getId).collect(Collectors.toSet()); + ReplicationGroup replicationGroup = new ReplicationGroup(routingTable, inSyncAllocationIds, inSyncAllocationIds, 0); + List replicationTargets = replicationGroup.getReplicationTargets(); + assertEquals(inSyncAllocationIds.size(), replicationTargets.size()); + assertTrue( + replicationTargets.stream().map(sh -> sh.allocationId().getId()).collect(Collectors.toSet()).containsAll(inSyncAllocationIds) + ); + + Request request = new Request(shardId); + PlainActionFuture listener = new PlainActionFuture<>(); + Map simulatedFailures = new HashMap<>(); + TestReplicaProxy replicasProxy = new TestReplicaProxy(simulatedFailures); + TestPrimary primary = new TestPrimary(primaryShard, () -> replicationGroup, threadPool); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + replicasProxy, + 0, + new ReplicationModeAwareProxy<>(ReplicationMode.NO_REPLICATION, replicasProxy, replicasProxy) + ); + op.execute(); + assertTrue("request was not processed on primary", request.processedOnPrimary.get()); + assertEquals(0, request.processedOnReplicas.size()); + assertEquals(0, replicasProxy.failedReplicas.size()); + assertEquals(0, replicasProxy.markedAsStaleCopies.size()); + assertTrue("post replication operations not run on primary", request.runPostReplicationActionsOnPrimary.get()); + assertTrue("listener is not marked as done", listener.isDone()); + + ShardInfo shardInfo = listener.actionGet().getShardInfo(); + assertEquals(1 + initializingIds.size(), shardInfo.getTotal()); + } + + public void testPrimaryToPrimaryReplicationWithRemoteTranslogEnabled() throws Exception { + Set initializingIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> initializingIds.add(AllocationId.newInitializing())); + Set activeIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> activeIds.add(AllocationId.newInitializing())); + + AllocationId primaryId = AllocationId.newRelocation(AllocationId.newInitializing()); + AllocationId relocationTargetId = AllocationId.newTargetRelocation(primaryId); + + ShardId shardId = new ShardId("test", "_na_", 0); + IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId); + final ShardRouting primaryShard = newShardRouting( + shardId, + nodeIdFromAllocationId(primaryId), + nodeIdFromAllocationId(relocationTargetId), + true, + ShardRoutingState.RELOCATING, + primaryId + ); + initializingIds.forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.INITIALIZING, aId)) + ); + activeIds.forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.STARTED, aId)) + ); + builder.addShard(primaryShard); + IndexShardRoutingTable routingTable = builder.build(); + + // Add primary and it's relocating target to activeIds + activeIds.add(primaryId); + activeIds.add(relocationTargetId); + + Set inSyncAllocationIds = activeIds.stream().map(AllocationId::getId).collect(Collectors.toSet()); + ReplicationGroup replicationGroup = new ReplicationGroup(routingTable, inSyncAllocationIds, inSyncAllocationIds, 0); + List replicationTargets = replicationGroup.getReplicationTargets(); + assertEquals(inSyncAllocationIds.size(), replicationTargets.size()); + assertTrue( + replicationTargets.stream().map(sh -> sh.allocationId().getId()).collect(Collectors.toSet()).containsAll(inSyncAllocationIds) + ); + + Request request = new Request(shardId); + PlainActionFuture listener = new PlainActionFuture<>(); + Map simulatedFailures = new HashMap<>(); + TestReplicaProxy replicasProxy = new TestReplicaProxy(simulatedFailures); + TestPrimary primary = new TestPrimary(primaryShard, () -> replicationGroup, threadPool); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + replicasProxy, + 0, + new ReplicationModeAwareProxy<>(ReplicationMode.NO_REPLICATION, replicasProxy, replicasProxy) + ); + op.execute(); + assertTrue("request was not processed on primary", request.processedOnPrimary.get()); + assertEquals(1, request.processedOnReplicas.size()); + assertEquals(0, replicasProxy.failedReplicas.size()); + assertEquals(0, replicasProxy.markedAsStaleCopies.size()); + assertTrue("post replication operations not run on primary", request.runPostReplicationActionsOnPrimary.get()); + assertTrue("listener is not marked as done", listener.isDone()); + + ShardInfo shardInfo = listener.actionGet().getShardInfo(); + assertEquals(2 + initializingIds.size(), shardInfo.getTotal()); + } + + public void testForceReplicationWithRemoteTranslogEnabled() throws Exception { + Set initializingIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> initializingIds.add(AllocationId.newInitializing())); + Set activeIds = new HashSet<>(); + IntStream.range(0, randomIntBetween(2, 5)).forEach(x -> activeIds.add(AllocationId.newInitializing())); + + AllocationId primaryId = activeIds.iterator().next(); + + ShardId shardId = new ShardId("test", "_na_", 0); + IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId); + final ShardRouting primaryShard = newShardRouting( + shardId, + nodeIdFromAllocationId(primaryId), + null, + true, + ShardRoutingState.STARTED, + primaryId + ); + initializingIds.forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.INITIALIZING, aId)) + ); + activeIds.stream() + .filter(aId -> !aId.equals(primaryId)) + .forEach( + aId -> builder.addShard(newShardRouting(shardId, nodeIdFromAllocationId(aId), null, false, ShardRoutingState.STARTED, aId)) + ); + builder.addShard(primaryShard); + IndexShardRoutingTable routingTable = builder.build(); + + Set inSyncAllocationIds = activeIds.stream().map(AllocationId::getId).collect(Collectors.toSet()); + ReplicationGroup replicationGroup = new ReplicationGroup(routingTable, inSyncAllocationIds, inSyncAllocationIds, 0); + List replicationTargets = replicationGroup.getReplicationTargets(); + assertEquals(inSyncAllocationIds.size(), replicationTargets.size()); + assertTrue( + replicationTargets.stream().map(sh -> sh.allocationId().getId()).collect(Collectors.toSet()).containsAll(inSyncAllocationIds) + ); + + Request request = new Request(shardId); + PlainActionFuture listener = new PlainActionFuture<>(); + Map simulatedFailures = new HashMap<>(); + TestReplicaProxy replicasProxy = new TestReplicaProxy(simulatedFailures); + TestPrimary primary = new TestPrimary(primaryShard, () -> replicationGroup, threadPool); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + replicasProxy, + 0, + new FanoutReplicationProxy<>(replicasProxy) + ); + op.execute(); + assertTrue("request was not processed on primary", request.processedOnPrimary.get()); + assertEquals(activeIds.size() - 1, request.processedOnReplicas.size()); + assertEquals(0, replicasProxy.failedReplicas.size()); + assertEquals(0, replicasProxy.markedAsStaleCopies.size()); + assertTrue("post replication operations not run on primary", request.runPostReplicationActionsOnPrimary.get()); + assertTrue("listener is not marked as done", listener.isDone()); + + ShardInfo shardInfo = listener.actionGet().getShardInfo(); + assertEquals(activeIds.size() + initializingIds.size(), shardInfo.getTotal()); + } + + static String nodeIdFromAllocationId(final AllocationId allocationId) { + return "n-" + allocationId.getId().substring(0, 8); + } + public void testRetryTransientReplicationFailure() throws Exception { final String index = "test"; final ShardId shardId = new ShardId(index, "_na_", 0); @@ -242,7 +447,8 @@ public void testRetryTransientReplicationFailure() throws Exception { replicasProxy, primaryTerm, TimeValue.timeValueMillis(20), - TimeValue.timeValueSeconds(60) + TimeValue.timeValueSeconds(60), + new FanoutReplicationProxy<>(replicasProxy) ); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); @@ -379,7 +585,14 @@ public void failShard(String message, Exception exception) { assertTrue(primaryFailed.compareAndSet(false, true)); } }; - final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy, primaryTerm); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + replicasProxy, + primaryTerm, + new FanoutReplicationProxy<>(replicasProxy) + ); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); @@ -389,7 +602,7 @@ public void failShard(String message, Exception exception) { } else { assertFalse(primaryFailed.get()); } - assertListenerThrows("should throw exception to trigger retry", listener, ReplicationOperation.RetryOnPrimaryException.class); + assertListenerThrows("should throw exception to trigger retry", listener, RetryOnPrimaryException.class); } public void testAddedReplicaAfterPrimaryOperation() throws Exception { @@ -438,7 +651,14 @@ public void perform(Request request, ActionListener listener) { Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); - final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, new TestReplicaProxy(), primaryTerm); + final TestReplicationOperation op = new TestReplicationOperation( + request, + primary, + listener, + new TestReplicaProxy(), + primaryTerm, + new FanoutReplicationProxy<>(new TestReplicaProxy()) + ); op.execute(); assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); @@ -493,7 +713,8 @@ public void testWaitForActiveShards() throws Exception { logger, threadPool, "test", - primaryTerm + primaryTerm, + new FanoutReplicationProxy<>(new TestReplicaProxy()) ); if (passesActiveShardCheck) { @@ -554,7 +775,14 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) final PlainActionFuture listener = new PlainActionFuture<>(); final ReplicationOperation.Replicas replicas = new TestReplicaProxy(Collections.emptyMap()); - TestReplicationOperation operation = new TestReplicationOperation(request, primary, listener, replicas, primaryTerm); + TestReplicationOperation operation = new TestReplicationOperation( + request, + primary, + listener, + replicas, + primaryTerm, + new FanoutReplicationProxy<>(replicas) + ); operation.execute(); assertThat(primaryFailed.get(), equalTo(fatal)); @@ -841,7 +1069,8 @@ class TestReplicationOperation extends ReplicationOperation replicas, long primaryTerm, TimeValue initialRetryBackoffBound, - TimeValue retryTimeout + TimeValue retryTimeout, + ReplicationProxy replicationProxy ) { this( request, @@ -853,7 +1082,8 @@ class TestReplicationOperation extends ReplicationOperation primary, ActionListener listener, Replicas replicas, - long primaryTerm + long primaryTerm, + ReplicationProxy replicationProxy ) { - this(request, primary, listener, replicas, ReplicationOperationTests.this.logger, threadPool, "test", primaryTerm); + this( + request, + primary, + listener, + replicas, + ReplicationOperationTests.this.logger, + threadPool, + "test", + primaryTerm, + replicationProxy + ); } TestReplicationOperation( @@ -875,7 +1116,8 @@ class TestReplicationOperation extends ReplicationOperation replicationProxy ) { this( request, @@ -887,7 +1129,8 @@ class TestReplicationOperation extends ReplicationOperation replicationProxy ) { - super(request, primary, listener, replicas, logger, threadPool, opType, primaryTerm, initialRetryBackoffBound, retryTimeout); + super( + request, + primary, + listener, + replicas, + logger, + threadPool, + opType, + primaryTerm, + initialRetryBackoffBound, + retryTimeout, + replicationProxy + ); } } diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java index e4ea7aaa9bfc2..166da26a4564e 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java @@ -33,6 +33,12 @@ package org.opensearch.action.support.replication; import org.apache.lucene.store.AlreadyClosedException; +import org.hamcrest.Matcher; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionListener; @@ -99,12 +105,6 @@ import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; import org.opensearch.transport.nio.MockNioTransport; -import org.hamcrest.Matcher; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; import java.io.IOException; import java.util.Collections; @@ -121,11 +121,6 @@ import java.util.stream.Collectors; import static java.util.Collections.singleton; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS; -import static org.opensearch.test.ClusterServiceUtils.createClusterService; -import static org.opensearch.test.ClusterServiceUtils.setState; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; @@ -138,12 +133,17 @@ import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.anyString; -import static org.mockito.Mockito.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.state; +import static org.opensearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; public class TransportReplicationActionTests extends OpenSearchTestCase { @@ -955,7 +955,8 @@ public void testSeqNoIsSetOnPrimary() { Set inSyncIds = randomBoolean() ? singleton(routingEntry.allocationId().getId()) : clusterService.state().metadata().index(index).inSyncAllocationIds(0); - ReplicationGroup replicationGroup = new ReplicationGroup(shardRoutingTable, inSyncIds, shardRoutingTable.getAllAllocationIds(), 0); + Set trackedAllocationIds = shardRoutingTable.getAllAllocationIds(); + ReplicationGroup replicationGroup = new ReplicationGroup(shardRoutingTable, inSyncIds, trackedAllocationIds, 0); when(shard.getReplicationGroup()).thenReturn(replicationGroup); PendingReplicationActions replicationActions = new PendingReplicationActions(shardId, threadPool); replicationActions.accept(replicationGroup); diff --git a/server/src/test/java/org/opensearch/index/seqno/GlobalCheckpointSyncActionTests.java b/server/src/test/java/org/opensearch/index/seqno/GlobalCheckpointSyncActionTests.java index 1e20c2e948f6e..75063d76ff8dc 100644 --- a/server/src/test/java/org/opensearch/index/seqno/GlobalCheckpointSyncActionTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/GlobalCheckpointSyncActionTests.java @@ -34,6 +34,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.ActionTestUtils; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -152,4 +153,31 @@ public void testTranslogSyncAfterGlobalCheckpointSync() throws Exception { } } + public void testGetReplicationModeWithRemoteTranslog() { + final GlobalCheckpointSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final GlobalCheckpointSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private GlobalCheckpointSyncAction createAction() { + final IndicesService indicesService = mock(IndicesService.class); + return new GlobalCheckpointSyncAction( + Settings.EMPTY, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()) + ); + } + } diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTestCase.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTestCase.java index 34a2d1189d234..17a6bfc8fbd82 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTestCase.java @@ -43,6 +43,7 @@ import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.IndexSettingsModule; +import java.util.Collections; import java.util.Set; import java.util.function.LongConsumer; import java.util.function.LongSupplier; @@ -55,12 +56,13 @@ public abstract class ReplicationTrackerTestCase extends OpenSearchTestCase { ReplicationTracker newTracker( final AllocationId allocationId, final LongConsumer updatedGlobalCheckpoint, - final LongSupplier currentTimeMillisSupplier + final LongSupplier currentTimeMillisSupplier, + final Settings settings ) { return new ReplicationTracker( new ShardId("test", "_na_", 0), allocationId.getId(), - IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), + IndexSettingsModule.newIndexSettings("test", settings), randomNonNegativeLong(), UNASSIGNED_SEQ_NO, updatedGlobalCheckpoint, @@ -70,6 +72,14 @@ ReplicationTracker newTracker( ); } + ReplicationTracker newTracker( + final AllocationId allocationId, + final LongConsumer updatedGlobalCheckpoint, + final LongSupplier currentTimeMillisSupplier + ) { + return newTracker(allocationId, updatedGlobalCheckpoint, currentTimeMillisSupplier, Settings.EMPTY); + } + static final Supplier OPS_BASED_RECOVERY_ALWAYS_REASONABLE = () -> SafeCommitInfo.EMPTY; static String nodeIdFromAllocationId(final AllocationId allocationId) { @@ -77,6 +87,14 @@ static String nodeIdFromAllocationId(final AllocationId allocationId) { } static IndexShardRoutingTable routingTable(final Set initializingIds, final AllocationId primaryId) { + return routingTable(initializingIds, Collections.singleton(primaryId), primaryId); + } + + static IndexShardRoutingTable routingTable( + final Set initializingIds, + final Set activeIds, + final AllocationId primaryId + ) { final ShardId shardId = new ShardId("test", "_na_", 0); final ShardRouting primaryShard = TestShardRouting.newShardRouting( shardId, @@ -86,11 +104,17 @@ static IndexShardRoutingTable routingTable(final Set initializingI ShardRoutingState.STARTED, primaryId ); - return routingTable(initializingIds, primaryShard); + return routingTable(initializingIds, activeIds, primaryShard); } - static IndexShardRoutingTable routingTable(final Set initializingIds, final ShardRouting primaryShard) { + static IndexShardRoutingTable routingTable( + final Set initializingIds, + final Set activeIds, + final ShardRouting primaryShard + ) { + assert initializingIds != null && activeIds != null; assert !initializingIds.contains(primaryShard.allocationId()); + assert activeIds.contains(primaryShard.allocationId()); final ShardId shardId = new ShardId("test", "_na_", 0); final IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId); for (final AllocationId initializingId : initializingIds) { @@ -105,6 +129,21 @@ static IndexShardRoutingTable routingTable(final Set initializingI ) ); } + for (final AllocationId activeId : activeIds) { + if (activeId.equals(primaryShard.allocationId())) { + continue; + } + builder.addShard( + TestShardRouting.newShardRouting( + shardId, + nodeIdFromAllocationId(activeId), + null, + false, + ShardRoutingState.STARTED, + activeId + ) + ); + } builder.addShard(primaryShard); diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index 66c484cd40cce..8ea64e71fb9dc 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -437,6 +437,10 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { private AtomicLong updatedGlobalCheckpoint = new AtomicLong(UNASSIGNED_SEQ_NO); + private ReplicationTracker newTracker(final AllocationId allocationId, Settings settings) { + return newTracker(allocationId, updatedGlobalCheckpoint::set, () -> 0L, settings); + } + private ReplicationTracker newTracker(final AllocationId allocationId) { return newTracker(allocationId, updatedGlobalCheckpoint::set, () -> 0L); } @@ -966,7 +970,11 @@ private static FakeClusterState initialState() { relocatingId ); - return new FakeClusterState(initialClusterStateVersion, activeAllocationIds, routingTable(initializingAllocationIds, primaryShard)); + return new FakeClusterState( + initialClusterStateVersion, + activeAllocationIds, + routingTable(initializingAllocationIds, Collections.singleton(primaryShard.allocationId()), primaryShard) + ); } private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) { @@ -1007,6 +1015,7 @@ private static FakeClusterState randomUpdateClusterState(Set allocationI remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds, routingTable( Sets.difference(Sets.union(initializingIdsExceptRelocationTargets, initializingIdsToAdd), initializingIdsToRemove), + Collections.singleton(clusterState.routingTable.primaryShard().allocationId()), clusterState.routingTable.primaryShard() ) ); @@ -1046,9 +1055,20 @@ private static void markAsTrackingAndInSyncQuietly( final ReplicationTracker tracker, final String allocationId, final long localCheckpoint + ) { + markAsTrackingAndInSyncQuietly(tracker, allocationId, localCheckpoint, true); + } + + private static void markAsTrackingAndInSyncQuietly( + final ReplicationTracker tracker, + final String allocationId, + final long localCheckpoint, + final boolean addPRRL ) { try { - addPeerRecoveryRetentionLease(tracker, allocationId); + if (addPRRL) { + addPeerRecoveryRetentionLease(tracker, allocationId); + } tracker.initiateTracking(allocationId); tracker.markAllocationIdAsInSync(allocationId, localCheckpoint); } catch (final InterruptedException e) { @@ -1252,4 +1272,695 @@ public void testPeerRecoveryRetentionLeaseCreationAndRenewal() { ); } + /** + * This test checks that the global checkpoint update mechanism is honored and relies only on the shards that have + * translog stored locally. + */ + public void testGlobalCheckpointUpdateWithRemoteTranslogEnabled() { + final long initialClusterStateVersion = randomNonNegativeLong(); + Map activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 5); + Set active = new HashSet<>(activeWithCheckpoints.keySet()); + Map allocations = new HashMap<>(activeWithCheckpoints); + Map initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5); + Set initializing = new HashSet<>(initializingWithCheckpoints.keySet()); + allocations.putAll(initializingWithCheckpoints); + assertThat(allocations.size(), equalTo(active.size() + initializing.size())); + + final AllocationId primaryId = active.iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + + long primaryLocalCheckpoint = activeWithCheckpoints.get(primaryId); + + logger.info("--> using allocations"); + allocations.keySet().forEach(aId -> { + final String type; + if (active.contains(aId)) { + type = "active"; + } else if (initializing.contains(aId)) { + type = "init"; + } else { + throw new IllegalStateException(aId + " not found in any map"); + } + logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); + }); + + tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1)); + initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED, false)); + assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1 + initializing.size())); + Set replicationTargets = tracker.getReplicationGroup() + .getReplicationTargets() + .stream() + .map(ShardRouting::allocationId) + .collect(Collectors.toSet()); + assertTrue(replicationTargets.containsAll(initializing)); + allocations.keySet().forEach(aId -> updateLocalCheckpoint(tracker, aId.getId(), allocations.get(aId))); + + assertEquals(tracker.getGlobalCheckpoint(), primaryLocalCheckpoint); + + // increment checkpoints + active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); + initializing.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); + allocations.keySet().forEach(aId -> updateLocalCheckpoint(tracker, aId.getId(), allocations.get(aId))); + + final long minLocalCheckpointAfterUpdates = allocations.values().stream().min(Long::compareTo).orElse(UNASSIGNED_SEQ_NO); + + // now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested. + final AllocationId extraId = AllocationId.newInitializing(); + + // first check that adding it without the cluster-manager blessing doesn't change anything. + updateLocalCheckpoint(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4)); + assertNull(tracker.checkpoints.get(extraId.getId())); + expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(extraId.getId())); + + Set newInitializing = new HashSet<>(initializing); + newInitializing.add(extraId); + tracker.updateFromClusterManager(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId)); + + tracker.initiateTracking(extraId.getId()); + + // now notify for the new id + if (randomBoolean()) { + updateLocalCheckpoint(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4)); + markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), randomInt((int) minLocalCheckpointAfterUpdates), false); + } else { + markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4), false); + } + } + + public void testUpdateFromClusterManagerWithRemoteTranslogEnabled() { + final long initialClusterStateVersion = randomNonNegativeLong(); + Map activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(2, 5); + Set active = new HashSet<>(activeWithCheckpoints.keySet()); + Map allocations = new HashMap<>(activeWithCheckpoints); + Map initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5); + Set initializing = new HashSet<>(initializingWithCheckpoints.keySet()); + allocations.putAll(initializingWithCheckpoints); + assertThat(allocations.size(), equalTo(active.size() + initializing.size())); + + final AllocationId primaryId = active.iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + + long primaryLocalCheckpoint = activeWithCheckpoints.get(primaryId); + + logger.info("--> using allocations"); + allocations.keySet().forEach(aId -> { + final String type; + if (active.contains(aId)) { + type = "active"; + } else if (initializing.contains(aId)) { + type = "init"; + } else { + throw new IllegalStateException(aId + " not found in any map"); + } + logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); + }); + + tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, active, primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + assertEquals(tracker.getReplicationGroup().getReplicationTargets().size(), active.size()); + initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED, false)); + assertEquals(tracker.getReplicationGroup().getReplicationTargets().size(), active.size() + initializing.size()); + Set replicationTargets = tracker.getReplicationGroup() + .getReplicationTargets() + .stream() + .map(ShardRouting::allocationId) + .collect(Collectors.toSet()); + assertTrue(replicationTargets.containsAll(initializing)); + assertTrue(replicationTargets.containsAll(active)); + allocations.keySet().forEach(aId -> updateLocalCheckpoint(tracker, aId.getId(), allocations.get(aId))); + + assertEquals(tracker.getGlobalCheckpoint(), primaryLocalCheckpoint); + + // increment checkpoints + active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); + initializing.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); + allocations.keySet().forEach(aId -> updateLocalCheckpoint(tracker, aId.getId(), allocations.get(aId))); + + final long minLocalCheckpointAfterUpdates = allocations.values().stream().min(Long::compareTo).orElse(UNASSIGNED_SEQ_NO); + + // now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested. + final AllocationId extraId = AllocationId.newInitializing(); + + // first check that adding it without the cluster-manager blessing doesn't change anything. + updateLocalCheckpoint(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4)); + assertNull(tracker.checkpoints.get(extraId.getId())); + expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(extraId.getId())); + + Set newInitializing = new HashSet<>(initializing); + newInitializing.add(extraId); + tracker.updateFromClusterManager(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId)); + + tracker.initiateTracking(extraId.getId()); + + // now notify for the new id + if (randomBoolean()) { + updateLocalCheckpoint(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4)); + markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), randomInt((int) minLocalCheckpointAfterUpdates), false); + } else { + markAsTrackingAndInSyncQuietly(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4), false); + } + } + + /** + * This test checks that updateGlobalCheckpointOnReplica with remote translog does not violate any of the invariants + */ + public void testUpdateGlobalCheckpointOnReplicaWithRemoteTranslogEnabled() { + final AllocationId active = AllocationId.newInitializing(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(active, settings); + final long globalCheckpoint = randomLongBetween(NO_OPS_PERFORMED, Long.MAX_VALUE - 1); + tracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "test"); + assertEquals(updatedGlobalCheckpoint.get(), globalCheckpoint); + final long nonUpdate = randomLongBetween(NO_OPS_PERFORMED, globalCheckpoint); + updatedGlobalCheckpoint.set(UNASSIGNED_SEQ_NO); + tracker.updateGlobalCheckpointOnReplica(nonUpdate, "test"); + assertEquals(updatedGlobalCheckpoint.get(), UNASSIGNED_SEQ_NO); + final long update = randomLongBetween(globalCheckpoint, Long.MAX_VALUE); + tracker.updateGlobalCheckpointOnReplica(update, "test"); + assertEquals(updatedGlobalCheckpoint.get(), update); + } + + public void testMarkAllocationIdAsInSyncWithRemoteTranslogEnabled() throws Exception { + final long initialClusterStateVersion = randomNonNegativeLong(); + Map activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 1); + Set active = new HashSet<>(activeWithCheckpoints.keySet()); + Map initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 1); + Set initializing = new HashSet<>(initializingWithCheckpoints.keySet()); + final AllocationId primaryId = active.iterator().next(); + final AllocationId replicaId = initializing.iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); + final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1); + tracker.activatePrimaryMode(localCheckpoint); + tracker.initiateTracking(replicaId.getId()); + tracker.markAllocationIdAsInSync(replicaId.getId(), randomLongBetween(NO_OPS_PERFORMED, localCheckpoint - 1)); + assertFalse(tracker.pendingInSync()); + final long updatedLocalCheckpoint = randomLongBetween(1 + localCheckpoint, Long.MAX_VALUE); + updatedGlobalCheckpoint.set(UNASSIGNED_SEQ_NO); + tracker.updateLocalCheckpoint(primaryId.getId(), updatedLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), updatedLocalCheckpoint); + tracker.updateLocalCheckpoint(replicaId.getId(), localCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), updatedLocalCheckpoint); + tracker.markAllocationIdAsInSync(replicaId.getId(), updatedLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), updatedLocalCheckpoint); + } + + public void testMissingActiveIdsDoesNotPreventAdvanceWithRemoteTranslogEnabled() { + final Map active = randomAllocationsWithLocalCheckpoints(2, 5); + final Map initializing = randomAllocationsWithLocalCheckpoints(0, 5); + final Map assigned = new HashMap<>(); + assigned.putAll(active); + assigned.putAll(initializing); + AllocationId primaryId = active.keySet().iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + List initializingRandomSubset = randomSubsetOf(initializing.keySet()); + initializingRandomSubset.forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); + final AllocationId missingActiveID = randomFrom(active.keySet()); + assigned.entrySet() + .stream() + .filter(e -> !e.getKey().equals(missingActiveID)) + .forEach(e -> updateLocalCheckpoint(tracker, e.getKey().getId(), e.getValue())); + long primaryLocalCheckpoint = active.get(primaryId); + + assertEquals(1 + initializingRandomSubset.size(), tracker.getReplicationGroup().getReplicationTargets().size()); + if (missingActiveID.equals(primaryId) == false) { + assertEquals(tracker.getGlobalCheckpoint(), primaryLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), primaryLocalCheckpoint); + } + // now update all knowledge of all shards + assigned.forEach((aid, localCP) -> updateLocalCheckpoint(tracker, aid.getId(), 10 + localCP)); + assertEquals(tracker.getGlobalCheckpoint(), 10 + primaryLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), 10 + primaryLocalCheckpoint); + } + + public void testMissingInSyncIdsDoesNotPreventAdvanceWithRemoteTranslogEnabled() { + final Map active = randomAllocationsWithLocalCheckpoints(1, 5); + final Map initializing = randomAllocationsWithLocalCheckpoints(2, 5); + logger.info("active: {}, initializing: {}", active, initializing); + + AllocationId primaryId = active.keySet().iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + randomSubsetOf(randomIntBetween(1, initializing.size() - 1), initializing.keySet()).forEach( + aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED) + ); + long primaryLocalCheckpoint = active.get(primaryId); + + active.forEach((aid, localCP) -> updateLocalCheckpoint(tracker, aid.getId(), localCP)); + + assertEquals(tracker.getGlobalCheckpoint(), primaryLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), primaryLocalCheckpoint); + + // update again + initializing.forEach((aid, localCP) -> updateLocalCheckpoint(tracker, aid.getId(), localCP)); + assertEquals(tracker.getGlobalCheckpoint(), primaryLocalCheckpoint); + assertEquals(updatedGlobalCheckpoint.get(), primaryLocalCheckpoint); + } + + public void testInSyncIdsAreIgnoredIfNotValidatedByClusterManagerWithRemoteTranslogEnabled() { + final Map active = randomAllocationsWithLocalCheckpoints(1, 5); + final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); + final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); + final AllocationId primaryId = active.keySet().iterator().next(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + initializing.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); + nonApproved.keySet() + .forEach( + k -> expectThrows(IllegalStateException.class, () -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)) + ); + + List> allocations = Arrays.asList(active, initializing, nonApproved); + Collections.shuffle(allocations, random()); + allocations.forEach(a -> a.forEach((aid, localCP) -> updateLocalCheckpoint(tracker, aid.getId(), localCP))); + + assertNotEquals(UNASSIGNED_SEQ_NO, tracker.getGlobalCheckpoint()); + } + + public void testInSyncIdsAreRemovedIfNotValidatedByClusterManagerWithRemoteTranslogEnabled() { + final long initialClusterStateVersion = randomNonNegativeLong(); + final Map activeToStay = randomAllocationsWithLocalCheckpoints(1, 5); + final Map initializingToStay = randomAllocationsWithLocalCheckpoints(1, 5); + final Map activeToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5); + final Map initializingToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5); + final Set active = Sets.union(activeToStay.keySet(), activeToBeRemoved.keySet()); + final Set initializing = Sets.union(initializingToStay.keySet(), initializingToBeRemoved.keySet()); + final Map allocations = new HashMap<>(); + final AllocationId primaryId = active.iterator().next(); + if (activeToBeRemoved.containsKey(primaryId)) { + activeToStay.put(primaryId, activeToBeRemoved.remove(primaryId)); + } + allocations.putAll(activeToStay); + if (randomBoolean()) { + allocations.putAll(activeToBeRemoved); + } + allocations.putAll(initializingToStay); + if (randomBoolean()) { + allocations.putAll(initializingToBeRemoved); + } + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + if (randomBoolean()) { + initializingToStay.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); + } else { + initializing.forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); + } + if (randomBoolean()) { + allocations.forEach((aid, localCP) -> updateLocalCheckpoint(tracker, aid.getId(), localCP)); + } + + // now remove shards + if (randomBoolean()) { + tracker.updateFromClusterManager( + initialClusterStateVersion + 1, + ids(activeToStay.keySet()), + routingTable(initializingToStay.keySet(), primaryId) + ); + allocations.forEach((aid, ckp) -> updateLocalCheckpoint(tracker, aid.getId(), ckp + 10L)); + } else { + allocations.forEach((aid, ckp) -> updateLocalCheckpoint(tracker, aid.getId(), ckp + 10L)); + tracker.updateFromClusterManager( + initialClusterStateVersion + 2, + ids(activeToStay.keySet()), + routingTable(initializingToStay.keySet(), primaryId) + ); + } + + final long checkpoint = activeToStay.get(primaryId) + 10; + assertEquals(tracker.getGlobalCheckpoint(), checkpoint); + } + + public void testUpdateAllocationIdsFromClusterManagerWithRemoteTranslogEnabled() throws Exception { + final long initialClusterStateVersion = randomNonNegativeLong(); + final int numberOfActiveAllocationsIds = randomIntBetween(2, 16); + final int numberOfInitializingIds = randomIntBetween(2, 16); + final Tuple, Set> activeAndInitializingAllocationIds = randomActiveAndInitializingAllocationIds( + numberOfActiveAllocationsIds, + numberOfInitializingIds + ); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set initializingIds = activeAndInitializingAllocationIds.v2(); + AllocationId primaryId = activeAllocationIds.iterator().next(); + IndexShardRoutingTable routingTable = routingTable(initializingIds, primaryId); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(primaryId, settings); + tracker.updateFromClusterManager(initialClusterStateVersion, ids(activeAllocationIds), routingTable); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + assertThat(tracker.getReplicationGroup().getInSyncAllocationIds(), equalTo(ids(activeAllocationIds))); + assertThat(tracker.getReplicationGroup().getRoutingTable(), equalTo(routingTable)); + + // first we assert that the in-sync and tracking sets are set up correctly + assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue( + activeAllocationIds.stream() + .filter(a -> a.equals(primaryId) == false) + .allMatch( + a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint() == SequenceNumbers.UNASSIGNED_SEQ_NO + ) + ); + assertTrue(initializingIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue( + initializingIds.stream() + .filter(a -> a.equals(primaryId) == false) + .allMatch( + a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint() == SequenceNumbers.UNASSIGNED_SEQ_NO + ) + ); + + // now we will remove some allocation IDs from these and ensure that they propagate through + final Set removingActiveAllocationIds = new HashSet<>(randomSubsetOf(activeAllocationIds)); + removingActiveAllocationIds.remove(primaryId); + final Set newActiveAllocationIds = activeAllocationIds.stream() + .filter(a -> !removingActiveAllocationIds.contains(a)) + .collect(Collectors.toSet()); + final List removingInitializingAllocationIds = randomSubsetOf(initializingIds); + final Set newInitializingAllocationIds = initializingIds.stream() + .filter(a -> !removingInitializingAllocationIds.contains(a)) + .collect(Collectors.toSet()); + routingTable = routingTable(newInitializingAllocationIds, primaryId); + tracker.updateFromClusterManager(initialClusterStateVersion + 1, ids(newActiveAllocationIds), routingTable); + assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue(removingActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()) == null)); + assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue(removingInitializingAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()) == null)); + assertThat( + tracker.getReplicationGroup().getInSyncAllocationIds(), + equalTo(ids(Sets.difference(Sets.union(activeAllocationIds, newActiveAllocationIds), removingActiveAllocationIds))) + ); + assertThat(tracker.getReplicationGroup().getRoutingTable(), equalTo(routingTable)); + + /* + * Now we will add an allocation ID to each of active and initializing and ensure they propagate through. Using different lengths + * than we have been using above ensures that we can not collide with a previous allocation ID + */ + newInitializingAllocationIds.add(AllocationId.newInitializing()); + tracker.updateFromClusterManager( + initialClusterStateVersion + 2, + ids(newActiveAllocationIds), + routingTable(newInitializingAllocationIds, primaryId) + ); + assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue( + newActiveAllocationIds.stream() + .filter(a -> a.equals(primaryId) == false) + .allMatch( + a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint() == SequenceNumbers.UNASSIGNED_SEQ_NO + ) + ); + assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + assertTrue( + newInitializingAllocationIds.stream() + .allMatch( + a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint() == SequenceNumbers.UNASSIGNED_SEQ_NO + ) + ); + + // the tracking allocation IDs should play no role in determining the global checkpoint + final Map activeLocalCheckpoints = newActiveAllocationIds.stream() + .collect(Collectors.toMap(Function.identity(), a -> randomIntBetween(1, 1024))); + activeLocalCheckpoints.forEach((a, l) -> updateLocalCheckpoint(tracker, a.getId(), l)); + final Map initializingLocalCheckpoints = newInitializingAllocationIds.stream() + .collect(Collectors.toMap(Function.identity(), a -> randomIntBetween(1, 1024))); + initializingLocalCheckpoints.forEach((a, l) -> updateLocalCheckpoint(tracker, a.getId(), l)); + assertTrue( + activeLocalCheckpoints.entrySet() + .stream() + .allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey().getId()).getLocalCheckpoint() == e.getValue()) + ); + assertTrue( + initializingLocalCheckpoints.entrySet() + .stream() + .allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey().getId()).getLocalCheckpoint() == e.getValue()) + ); + final long primaryLocalCheckpoint = activeLocalCheckpoints.get(primaryId); + assertThat(tracker.getGlobalCheckpoint(), equalTo(primaryLocalCheckpoint)); + assertThat(updatedGlobalCheckpoint.get(), equalTo(primaryLocalCheckpoint)); + final long minimumInitailizingLocalCheckpoint = (long) initializingLocalCheckpoints.values().stream().min(Integer::compareTo).get(); + + // now we are going to add a new allocation ID and bring it in sync which should move it to the in-sync allocation IDs + final long localCheckpoint = randomIntBetween( + 0, + Math.toIntExact(Math.min(primaryLocalCheckpoint, minimumInitailizingLocalCheckpoint) - 1) + ); + + // using a different length than we have been using above ensures that we can not collide with a previous allocation ID + final AllocationId newSyncingAllocationId = AllocationId.newInitializing(); + newInitializingAllocationIds.add(newSyncingAllocationId); + tracker.updateFromClusterManager( + initialClusterStateVersion + 3, + ids(newActiveAllocationIds), + routingTable(newInitializingAllocationIds, primaryId) + ); + addPeerRecoveryRetentionLease(tracker, newSyncingAllocationId); + final CyclicBarrier barrier = new CyclicBarrier(2); + final Thread thread = new Thread(() -> { + try { + barrier.await(); + tracker.initiateTracking(newSyncingAllocationId.getId()); + tracker.markAllocationIdAsInSync(newSyncingAllocationId.getId(), localCheckpoint); + barrier.await(); + } catch (final BrokenBarrierException | InterruptedException e) { + throw new RuntimeException(e); + } + }); + + thread.start(); + + barrier.await(); + + assertBusy(() -> { + assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); + assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); + }); + + tracker.updateLocalCheckpoint(newSyncingAllocationId.getId(), randomIntBetween(Math.toIntExact(primaryLocalCheckpoint), 1024)); + + barrier.await(); + + assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); + assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); + + /* + * The new in-sync allocation ID is in the in-sync set now yet the cluster-manager does not know this; the allocation ID should still be in + * the in-sync set even if we receive a cluster state update that does not reflect this. + * + */ + tracker.updateFromClusterManager( + initialClusterStateVersion + 4, + ids(newActiveAllocationIds), + routingTable(newInitializingAllocationIds, primaryId) + ); + assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); + assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); + } + + public void testPrimaryContextHandoffWithRemoteTranslogEnabled() throws IOException { + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", settings); + final ShardId shardId = new ShardId("test", "_na_", 0); + + FakeClusterState clusterState = initialState(); + final AllocationId aId = clusterState.routingTable.primaryShard().allocationId(); + final LongConsumer onUpdate = updatedGlobalCheckpoint -> {}; + final long primaryTerm = randomNonNegativeLong(); + final long globalCheckpoint = UNASSIGNED_SEQ_NO; + final BiConsumer> onNewRetentionLease = (leases, listener) -> {}; + ReplicationTracker oldPrimary = new ReplicationTracker( + shardId, + aId.getId(), + indexSettings, + primaryTerm, + globalCheckpoint, + onUpdate, + () -> 0L, + onNewRetentionLease, + OPS_BASED_RECOVERY_ALWAYS_REASONABLE + ); + ReplicationTracker newPrimary = new ReplicationTracker( + shardId, + aId.getRelocationId(), + indexSettings, + primaryTerm, + globalCheckpoint, + onUpdate, + () -> 0L, + onNewRetentionLease, + OPS_BASED_RECOVERY_ALWAYS_REASONABLE + ); + + Set allocationIds = new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId)); + + clusterState.apply(oldPrimary); + clusterState.apply(newPrimary); + + oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10)); + addPeerRecoveryRetentionLease(oldPrimary, newPrimary.shardAllocationId); + newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); + + final int numUpdates = randomInt(10); + for (int i = 0; i < numUpdates; i++) { + if (rarely()) { + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(oldPrimary); + clusterState.apply(newPrimary); + } + if (randomBoolean()) { + randomLocalCheckpointUpdate(oldPrimary); + } + if (randomBoolean()) { + randomMarkInSync(oldPrimary, newPrimary); + } + } + + // simulate transferring the global checkpoint to the new primary after finalizing recovery before the handoff + markAsTrackingAndInSyncQuietly( + oldPrimary, + newPrimary.shardAllocationId, + Math.max(SequenceNumbers.NO_OPS_PERFORMED, oldPrimary.getGlobalCheckpoint() + randomInt(5)) + ); + oldPrimary.updateGlobalCheckpointForShard(newPrimary.shardAllocationId, oldPrimary.getGlobalCheckpoint()); + ReplicationTracker.PrimaryContext primaryContext = oldPrimary.startRelocationHandoff(newPrimary.shardAllocationId); + + if (randomBoolean()) { + // cluster state update after primary context handoff + if (randomBoolean()) { + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(oldPrimary); + clusterState.apply(newPrimary); + } + + // abort handoff, check that we can continue updates and retry handoff + oldPrimary.abortRelocationHandoff(); + + if (rarely()) { + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(oldPrimary); + clusterState.apply(newPrimary); + } + if (randomBoolean()) { + randomLocalCheckpointUpdate(oldPrimary); + } + if (randomBoolean()) { + randomMarkInSync(oldPrimary, newPrimary); + } + + // do another handoff + primaryContext = oldPrimary.startRelocationHandoff(newPrimary.shardAllocationId); + } + + // send primary context through the wire + BytesStreamOutput output = new BytesStreamOutput(); + primaryContext.writeTo(output); + StreamInput streamInput = output.bytes().streamInput(); + primaryContext = new ReplicationTracker.PrimaryContext(streamInput); + switch (randomInt(3)) { + case 0: { + // apply cluster state update on old primary while primary context is being transferred + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(oldPrimary); + // activate new primary + newPrimary.activateWithPrimaryContext(primaryContext); + // apply cluster state update on new primary so that the states on old and new primary are comparable + clusterState.apply(newPrimary); + break; + } + case 1: { + // apply cluster state update on new primary while primary context is being transferred + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(newPrimary); + // activate new primary + newPrimary.activateWithPrimaryContext(primaryContext); + // apply cluster state update on old primary so that the states on old and new primary are comparable + clusterState.apply(oldPrimary); + break; + } + case 2: { + // apply cluster state update on both copies while primary context is being transferred + clusterState = randomUpdateClusterState(allocationIds, clusterState); + clusterState.apply(oldPrimary); + clusterState.apply(newPrimary); + newPrimary.activateWithPrimaryContext(primaryContext); + break; + } + case 3: { + // no cluster state update + newPrimary.activateWithPrimaryContext(primaryContext); + break; + } + } + + assertTrue(oldPrimary.primaryMode); + assertTrue(newPrimary.primaryMode); + assertThat(newPrimary.appliedClusterStateVersion, equalTo(oldPrimary.appliedClusterStateVersion)); + /* + * We can not assert on shared knowledge of the global checkpoint between the old primary and the new primary as the new primary + * will update its global checkpoint state without the old primary learning of it, and the old primary could have updated its + * global checkpoint state after the primary context was transferred. + */ + Map oldPrimaryCheckpointsCopy = new HashMap<>(oldPrimary.checkpoints); + oldPrimaryCheckpointsCopy.remove(oldPrimary.shardAllocationId); + oldPrimaryCheckpointsCopy.remove(newPrimary.shardAllocationId); + Map newPrimaryCheckpointsCopy = new HashMap<>(newPrimary.checkpoints); + newPrimaryCheckpointsCopy.remove(oldPrimary.shardAllocationId); + newPrimaryCheckpointsCopy.remove(newPrimary.shardAllocationId); + assertThat(newPrimaryCheckpointsCopy, equalTo(oldPrimaryCheckpointsCopy)); + // we can however assert that shared knowledge of the local checkpoint and in-sync status is equal + assertThat( + oldPrimary.checkpoints.get(oldPrimary.shardAllocationId).localCheckpoint, + equalTo(newPrimary.checkpoints.get(oldPrimary.shardAllocationId).localCheckpoint) + ); + assertThat( + oldPrimary.checkpoints.get(newPrimary.shardAllocationId).localCheckpoint, + equalTo(newPrimary.checkpoints.get(newPrimary.shardAllocationId).localCheckpoint) + ); + assertThat( + oldPrimary.checkpoints.get(oldPrimary.shardAllocationId).inSync, + equalTo(newPrimary.checkpoints.get(oldPrimary.shardAllocationId).inSync) + ); + assertThat( + oldPrimary.checkpoints.get(newPrimary.shardAllocationId).inSync, + equalTo(newPrimary.checkpoints.get(newPrimary.shardAllocationId).inSync) + ); + assertThat(newPrimary.getGlobalCheckpoint(), equalTo(oldPrimary.getGlobalCheckpoint())); + assertThat(newPrimary.routingTable, equalTo(oldPrimary.routingTable)); + assertThat(newPrimary.replicationGroup, equalTo(oldPrimary.replicationGroup)); + + assertFalse(oldPrimary.relocated); + oldPrimary.completeRelocationHandoff(); + assertFalse(oldPrimary.primaryMode); + assertTrue(oldPrimary.relocated); + } + + public void testIllegalStateExceptionIfUnknownAllocationIdWithRemoteTranslogEnabled() { + final AllocationId active = AllocationId.newInitializing(); + final AllocationId initializing = AllocationId.newInitializing(); + Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + final ReplicationTracker tracker = newTracker(active, settings); + tracker.updateFromClusterManager( + randomNonNegativeLong(), + Collections.singleton(active.getId()), + routingTable(Collections.singleton(initializing), active) + ); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + + expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(randomAlphaOfLength(10))); + expectThrows(IllegalStateException.class, () -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(10), randomNonNegativeLong())); + } + } diff --git a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java index 54a88d57b2b69..2e058b6dab560 100644 --- a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseBackgroundSyncActionTests.java @@ -37,6 +37,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.ActionTestUtils; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; @@ -209,4 +210,30 @@ public void testBlocks() { assertNull(action.indexBlockLevel()); } + public void testGetReplicationModeWithRemoteTranslog() { + final RetentionLeaseBackgroundSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final RetentionLeaseBackgroundSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private RetentionLeaseBackgroundSyncAction createAction() { + return new RetentionLeaseBackgroundSyncAction( + Settings.EMPTY, + transportService, + clusterService, + mock(IndicesService.class), + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()) + ); + } + } diff --git a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java index 60ee3360ff235..b07b740fe3744 100644 --- a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseSyncActionTests.java @@ -36,6 +36,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.ActionTestUtils; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; @@ -206,4 +207,32 @@ public void testBlocks() { assertNull(action.indexBlockLevel()); } + public void testGetReplicationModeWithRemoteTranslog() { + final RetentionLeaseSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final RetentionLeaseSyncAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.NO_REPLICATION, action.getReplicationMode(indexShard)); + } + + private RetentionLeaseSyncAction createAction() { + return new RetentionLeaseSyncAction( + Settings.EMPTY, + transportService, + clusterService, + mock(IndicesService.class), + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + new IndexingPressureService(Settings.EMPTY, clusterService), + new SystemIndices(emptyMap()) + ); + } + } diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 121da28d783f1..39030efae878a 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; @@ -36,73 +37,6 @@ public class ReplicaRecoveryWithRemoteTranslogOnPrimaryTests extends OpenSearchI .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") .build(); - public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { - - // Step1 - Start primary, index docs and flush - shards.startPrimary(); - final IndexShard primary = shards.getPrimary(); - int numDocs = shards.indexDocs(randomIntBetween(10, 100)); - shards.flush(); - - // Step 2 - Start replica for recovery to happen, check both has same number of docs - final IndexShard replica1 = shards.addReplica(); - shards.startAll(); - assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); - - // Step 3 - Index more docs, run segment replication, check both have same number of docs - int moreDocs = shards.indexDocs(randomIntBetween(10, 100)); - primary.refresh("test"); - replicateSegments(primary, shards.getReplicas()); - assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); - - // Step 4 - Check both shard has expected number of doc count - assertDocCount(primary, numDocs + moreDocs); - assertDocCount(replica1, numDocs + moreDocs); - - // Step 5 - Start new replica, recovery happens, and check that new replica has docs upto last flush - final IndexShard replica2 = shards.addReplica(); - shards.startAll(); - assertDocCount(replica2, numDocs); - - // Step 6 - Segment replication, check all shards have same number of docs - replicateSegments(primary, shards.getReplicas()); - shards.assertAllEqual(numDocs + moreDocs); - } - } - - public void testNoTranslogHistoryTransferred() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { - - // Step1 - Start primary, index docs, flush, index more docs, check translog in primary as expected - shards.startPrimary(); - final IndexShard primary = shards.getPrimary(); - int numDocs = shards.indexDocs(randomIntBetween(10, 100)); - shards.flush(); - List docIdAndSeqNosAfterFlush = getDocIdAndSeqNos(primary); - int moreDocs = shards.indexDocs(randomIntBetween(20, 100)); - assertEquals(moreDocs, getTranslog(primary).totalOperations()); - - // Step 2 - Start replica, recovery happens, check docs recovered till last flush - final IndexShard replica = shards.addReplica(); - shards.startAll(); - assertEquals(docIdAndSeqNosAfterFlush, getDocIdAndSeqNos(replica)); - assertDocCount(replica, numDocs); - assertEquals(NRTReplicationEngine.class, replica.getEngine().getClass()); - - // Step 3 - Check replica's translog has no operations - assertEquals(WriteOnlyTranslogManager.class, ((NRTReplicationEngine) replica.getEngine()).translogManager().getClass()); - WriteOnlyTranslogManager replicaTranslogManager = (WriteOnlyTranslogManager) ((NRTReplicationEngine) replica.getEngine()) - .translogManager(); - assertEquals(0, replicaTranslogManager.getTranslog().totalOperations()); - - // Adding this for close to succeed - shards.flush(); - replicateSegments(primary, shards.getReplicas()); - shards.assertAllEqual(numDocs + moreDocs); - } - } - public void testStartSequenceForReplicaRecovery() throws Exception { try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { @@ -147,20 +81,23 @@ public void testStartSequenceForReplicaRecovery() throws Exception { null ); shards.addReplica(newReplicaShard); + AtomicBoolean assertDone = new AtomicBoolean(false); shards.recoverReplica(newReplicaShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener) { @Override public IndexShard indexShard() { IndexShard idxShard = super.indexShard(); - // verify the starting sequence number while recovering a failed shard which has a valid last commit - long startingSeqNo = -1; - try { - startingSeqNo = Long.parseLong( - idxShard.store().readLastCommittedSegmentsInfo().getUserData().get(SequenceNumbers.MAX_SEQ_NO) - ); - } catch (IOException e) { - Assert.fail(); + if (assertDone.compareAndSet(false, true)) { + // verify the starting sequence number while recovering a failed shard which has a valid last commit + long startingSeqNo = -1; + try { + startingSeqNo = Long.parseLong( + idxShard.store().readLastCommittedSegmentsInfo().getUserData().get(SequenceNumbers.MAX_SEQ_NO) + ); + } catch (IOException e) { + Assert.fail(); + } + assertEquals(numDocs - 1, startingSeqNo); } - assertEquals(numDocs - 1, startingSeqNo); return idxShard; } }); @@ -170,4 +107,36 @@ public IndexShard indexShard() { shards.assertAllEqual(numDocs + moreDocs); } } + + public void testNoTranslogHistoryTransferred() throws Exception { + try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { + + // Step1 - Start primary, index docs, flush, index more docs, check translog in primary as expected + shards.startPrimary(); + final IndexShard primary = shards.getPrimary(); + int numDocs = shards.indexDocs(randomIntBetween(10, 100)); + shards.flush(); + List docIdAndSeqNosAfterFlush = getDocIdAndSeqNos(primary); + int moreDocs = shards.indexDocs(randomIntBetween(20, 100)); + assertEquals(moreDocs, getTranslog(primary).totalOperations()); + + // Step 2 - Start replica, recovery happens, check docs recovered till last flush + final IndexShard replica = shards.addReplica(); + shards.startAll(); + assertEquals(docIdAndSeqNosAfterFlush, getDocIdAndSeqNos(replica)); + assertDocCount(replica, numDocs); + assertEquals(NRTReplicationEngine.class, replica.getEngine().getClass()); + + // Step 3 - Check replica's translog has no operations + assertEquals(WriteOnlyTranslogManager.class, ((NRTReplicationEngine) replica.getEngine()).translogManager().getClass()); + WriteOnlyTranslogManager replicaTranslogManager = (WriteOnlyTranslogManager) ((NRTReplicationEngine) replica.getEngine()) + .translogManager(); + assertEquals(0, replicaTranslogManager.getTranslog().totalOperations()); + + // Adding this for close to succeed + shards.flush(); + replicateSegments(primary, shards.getReplicas()); + shards.assertAllEqual(numDocs + moreDocs); + } + } } diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java similarity index 97% rename from server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java rename to server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java index 2b5550b71a627..7761f97769440 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java @@ -143,7 +143,10 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -public class RecoverySourceHandlerTests extends OpenSearchTestCase { +/** + * This covers test cases for {@link RecoverySourceHandler} and {@link LocalStorePeerRecoverySourceHandler}. + */ +public class LocalStorePeerRecoverySourceHandlerTests extends OpenSearchTestCase { private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings( "index", Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build() @@ -215,7 +218,7 @@ public void writeFileChunk( }); } }; - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, @@ -296,7 +299,7 @@ public void indexTranslogOperations( listener.onResponse(checkpointOnTarget.get()); } }; - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool, @@ -359,7 +362,7 @@ public void indexTranslogOperations( } } }; - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool, @@ -433,7 +436,7 @@ public void indexTranslogOperations( Randomness.shuffle(operations); List skipOperations = randomSubsetOf(operations); Translog.Snapshot snapshot = newTranslogSnapshot(operations, skipOperations); - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, @@ -552,7 +555,7 @@ public void writeFileChunk( failedEngine.set(true); return null; }).when(mockShard).failShard(any(), any()); - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( mockShard, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, @@ -627,7 +630,7 @@ public void writeFileChunk( failedEngine.set(true); return null; }).when(mockShard).failShard(any(), any()); - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( mockShard, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, @@ -680,7 +683,7 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final AtomicBoolean phase1Called = new AtomicBoolean(); final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean(); final AtomicBoolean phase2Called = new AtomicBoolean(); - final RecoverySourceHandler handler = new RecoverySourceHandler( + final RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, mock(RecoveryTargetHandler.class), threadPool, @@ -691,9 +694,15 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE ) { @Override - void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, ActionListener listener) { + void phase1( + IndexCommit snapshot, + long startingSeqNo, + IntSupplier translogOps, + ActionListener listener, + boolean skipCreateRetentionLeaseStep + ) { phase1Called.set(true); - super.phase1(snapshot, startingSeqNo, translogOps, listener); + super.phase1(snapshot, startingSeqNo, translogOps, listener, skipCreateRetentionLeaseStep); } @Override @@ -786,7 +795,7 @@ public void writeFileChunk( }; final int maxConcurrentChunks = between(1, 8); final int chunkSize = between(1, 32); - final RecoverySourceHandler handler = new RecoverySourceHandler( + final RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, recoveryTarget, threadPool, @@ -859,7 +868,7 @@ public void writeFileChunk( }; final int maxConcurrentChunks = between(1, 4); final int chunkSize = between(1, 16); - final RecoverySourceHandler handler = new RecoverySourceHandler( + final RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( null, new AsyncRecoveryTarget(recoveryTarget, recoveryExecutor), threadPool, @@ -967,7 +976,7 @@ public void cleanFiles( } }; final StartRecoveryRequest startRecoveryRequest = getStartRecoveryRequest(); - final RecoverySourceHandler handler = new RecoverySourceHandler( + final RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, recoveryTarget, threadPool, @@ -993,7 +1002,7 @@ void createRetentionLease(long startingSeqNo, ActionListener lis final StepListener phase1Listener = new StepListener<>(); try { final CountDownLatch latch = new CountDownLatch(1); - handler.phase1(DirectoryReader.listCommits(dir).get(0), 0, () -> 0, new LatchedActionListener<>(phase1Listener, latch)); + handler.phase1(DirectoryReader.listCommits(dir).get(0), 0, () -> 0, new LatchedActionListener<>(phase1Listener, latch), false); latch.await(); phase1Listener.result(); } catch (Exception e) { @@ -1006,7 +1015,7 @@ void createRetentionLease(long startingSeqNo, ActionListener lis public void testVerifySeqNoStatsWhenRecoverWithSyncId() throws Exception { IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); - RecoverySourceHandler handler = new RecoverySourceHandler( + RecoverySourceHandler handler = new LocalStorePeerRecoverySourceHandler( shard, new TestRecoveryTargetHandler(), threadPool, @@ -1061,7 +1070,7 @@ private Store newStore(Path path) throws IOException { } private Store newStore(Path path, boolean checkIndex) throws IOException { - BaseDirectoryWrapper baseDirectoryWrapper = RecoverySourceHandlerTests.newFSDirectory(path); + BaseDirectoryWrapper baseDirectoryWrapper = LocalStorePeerRecoverySourceHandlerTests.newFSDirectory(path); if (checkIndex == false) { baseDirectoryWrapper.setCheckIndexOnClose(false); // don't run checkindex we might corrupt the index in these tests } diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java new file mode 100644 index 0000000000000..91953d4db3495 --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -0,0 +1,69 @@ +/* + * 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.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; +import org.opensearch.index.seqno.ReplicationTracker; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.replication.common.ReplicationType; + +public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLevelReplicationTestCase { + + private static final Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") + .build(); + + public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { + try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { + + // Step1 - Start primary, index docs and flush + shards.startPrimary(); + final IndexShard primary = shards.getPrimary(); + int numDocs = shards.indexDocs(randomIntBetween(10, 100)); + shards.flush(); + + // Step 2 - Start replica for recovery to happen, check both has same number of docs + final IndexShard replica1 = shards.addReplica(); + shards.startAll(); + assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); + + // Step 3 - Index more docs, run segment replication, check both have same number of docs + int moreDocs = shards.indexDocs(randomIntBetween(10, 100)); + primary.refresh("test"); + replicateSegments(primary, shards.getReplicas()); + assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); + + // Step 4 - Check both shard has expected number of doc count + assertDocCount(primary, numDocs + moreDocs); + assertDocCount(replica1, numDocs + moreDocs); + + // Step 5 - Check retention lease does not exist for the replica shard + assertEquals(1, primary.getRetentionLeases().leases().size()); + assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica1.routingEntry()))); + + // Step 6 - Start new replica, recovery happens, and check that new replica has docs upto last flush + final IndexShard replica2 = shards.addReplica(); + shards.startAll(); + assertDocCount(replica2, numDocs); + + // Step 7 - Segment replication, check all shards have same number of docs + replicateSegments(primary, shards.getReplicas()); + shards.assertAllEqual(numDocs + moreDocs); + + // Step 8 - Check retention lease does not exist for the replica shard + assertEquals(1, primary.getRetentionLeases().leases().size()); + assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); + } + } +} diff --git a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointActionTests.java b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointActionTests.java index f1f55c67bfca6..88b9512eedaba 100644 --- a/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointActionTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/checkpoint/PublishCheckpointActionTests.java @@ -12,6 +12,7 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.ActionTestUtils; import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.action.support.replication.ReplicationMode; import org.opensearch.action.support.replication.TransportReplicationAction; import org.opensearch.cluster.action.shard.ShardStateAction; import org.opensearch.cluster.service.ClusterService; @@ -158,4 +159,31 @@ public void testPublishCheckpointActionOnReplica() { } + public void testGetReplicationModeWithRemoteTranslog() { + final PublishCheckpointAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(true); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + public void testGetReplicationModeWithLocalTranslog() { + final PublishCheckpointAction action = createAction(); + final IndexShard indexShard = mock(IndexShard.class); + when(indexShard.isRemoteTranslogEnabled()).thenReturn(false); + assertEquals(ReplicationMode.FULL_REPLICATION, action.getReplicationMode(indexShard)); + } + + private PublishCheckpointAction createAction() { + return new PublishCheckpointAction( + Settings.EMPTY, + transportService, + clusterService, + mock(IndicesService.class), + threadPool, + shardStateAction, + new ActionFilters(Collections.emptySet()), + mock(SegmentReplicationTargetService.class) + ); + } + } diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index 45a07dc2800d0..20fe47c1d4cc0 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -52,6 +52,7 @@ import org.opensearch.action.support.ActionTestUtils; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.support.replication.FanoutReplicationProxy; import org.opensearch.action.support.replication.PendingReplicationActions; import org.opensearch.action.support.replication.ReplicatedWriteRequest; import org.opensearch.action.support.replication.ReplicationOperation; @@ -758,7 +759,8 @@ public void execute() { opType, primaryTerm, TimeValue.timeValueMillis(20), - TimeValue.timeValueSeconds(60) + TimeValue.timeValueSeconds(60), + new FanoutReplicationProxy<>(new ReplicasRef()) ).execute(); } catch (Exception e) { listener.onFailure(e); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 34d09d5bdebfb..bc7780ffed131 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -68,6 +68,7 @@ import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.util.BigArrays; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.internal.io.IOUtils; @@ -104,6 +105,7 @@ import org.opensearch.indices.recovery.RecoveryResponse; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoverySourceHandler; +import org.opensearch.indices.recovery.RecoverySourceHandlerFactory; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.recovery.StartRecoveryRequest; @@ -131,8 +133,8 @@ import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.ArrayList; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashSet; @@ -864,17 +866,23 @@ protected final void recoverUnstartedReplica( recoveryTarget, startingSeqNo ); - int fileChunkSizeInBytes = Math.toIntExact( - randomBoolean() ? RecoverySettings.DEFAULT_CHUNK_SIZE.getBytes() : randomIntBetween(1, 10 * 1024 * 1024) + long fileChunkSizeInBytes = randomBoolean() + ? RecoverySettings.DEFAULT_CHUNK_SIZE.getBytes() + : randomIntBetween(1, 10 * 1024 * 1024); + final Settings settings = Settings.builder() + .put("indices.recovery.max_concurrent_file_chunks", Integer.toString(between(1, 4))) + .put("indices.recovery.max_concurrent_operations", Integer.toString(between(1, 4))) + .build(); + RecoverySettings recoverySettings = new RecoverySettings( + settings, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) ); - final RecoverySourceHandler recovery = new RecoverySourceHandler( + recoverySettings.setChunkSize(new ByteSizeValue(fileChunkSizeInBytes)); + final RecoverySourceHandler recovery = RecoverySourceHandlerFactory.create( primary, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - threadPool, request, - fileChunkSizeInBytes, - between(1, 8), - between(1, 8) + recoverySettings ); primary.updateShardState( primary.routingEntry(),