Skip to content

Commit

Permalink
[Backport 2.x] Support no-op replication for remote-backed indexes (#…
Browse files Browse the repository at this point in the history
…5731)

* Remove PRRL creation/deletion in peer recovery of remote store enabled replica (#4954)

* Add RecoverySourceHandlerFactory for extensibility

Signed-off-by: Ashish Singh <[email protected]>

* recoverToTarget made extensible to allow multiple implementations

Signed-off-by: Ashish Singh <[email protected]>

* Remove PRRL after SendFileStep in Peer Recovery

Signed-off-by: Ashish Singh <[email protected]>

* Incorporate PR review feedback

Signed-off-by: Ashish Singh <[email protected]>

* Empty-Commit

Signed-off-by: Ashish Singh <[email protected]>

* Incorporate PR review feedback

Signed-off-by: Ashish Singh <[email protected]>

* Empty-Commit

Signed-off-by: Ashish Singh <[email protected]>

* Empty-Commit

Signed-off-by: Ashish Singh <[email protected]>

* Remove CHANGELOG entry as this is incremental PR

Signed-off-by: Ashish Singh <[email protected]>

* Incorporate PR review feedback

Signed-off-by: Ashish Singh <[email protected]>

Signed-off-by: Ashish Singh <[email protected]>

* Enhance CheckpointState to support no-op replication (#5282)

* CheckpointState enhanced to support no-op replication

Signed-off-by: Ashish Singh <[email protected]>
Co-authored-by: Bukhtawar Khan<[email protected]>
Signed-off-by: Ashish Singh <[email protected]>

* Add transport action for primary term validation for remote-backed indices (#5616)

Add transport action for primary term validation for remote-backed indices

Signed-off-by: Ashish Singh <[email protected]>

Signed-off-by: Ashish Singh <[email protected]>
  • Loading branch information
ashking94 authored and kotwanikunal committed Jan 25, 2023
1 parent 63f3275 commit 98e4eb6
Show file tree
Hide file tree
Showing 35 changed files with 3,028 additions and 462 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -115,6 +134,15 @@ public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequ
private final UpdateHelper updateHelper;
private final MappingUpdatedAction mappingUpdatedAction;

/**
* This action is used for performing primary term validation. With remote translog enabled, the translogs would
* be durably persisted in remote store. Without remote translog, current transport replication calls does primary
* term validation as well as logically replicate the data. With remote translog, the primary would make calls to
* replicas to perform primary term validation. This make sures an isolated primary fails to ack after primary
* term validation in presence of a new primary.
*/
private final String transportPrimaryTermValidationAction;

@Inject
public TransportShardBulkAction(
Settings settings,
Expand Down Expand Up @@ -147,6 +175,212 @@ public TransportShardBulkAction(
);
this.updateHelper = updateHelper;
this.mappingUpdatedAction = mappingUpdatedAction;

this.transportPrimaryTermValidationAction = ACTION_NAME + "[validate_primary_term]";

transportService.registerRequestHandler(
transportPrimaryTermValidationAction,
executor,
true,
true,
PrimaryTermValidationRequest::new,
this::handlePrimaryTermValidationRequest
);
}

protected void handlePrimaryTermValidationRequest(
final PrimaryTermValidationRequest request,
final TransportChannel channel,
final Task task
) {
ActionListener<ReplicaResponse> 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<Releasable> {

private final ActionListener<ReplicaResponse> onCompletionListener;
private final IndexShard replica;
private final ReplicationTask task;
private final PrimaryTermValidationRequest request;

public PrimaryTermValidationReplicaAction(
ActionListener<ReplicaResponse> 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<String, String> 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<BulkShardRequest> 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<ReplicationOperation.ReplicaResponse> 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<ReplicaResponse> handler = new ActionListenerResponseHandler<>(
listener,
ReplicaResponse::new
);
transportService.sendRequest(node, transportPrimaryTermValidationAction, validationRequest, transportOptions, handler);
}
}

@Override
Expand Down Expand Up @@ -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,
Expand Down
Original file line number Diff line number Diff line change
@@ -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<ReplicaRequest extends ReplicationRequest<ReplicaRequest>> extends ReplicationProxy<ReplicaRequest> {

public FanoutReplicationProxy(Replicas<ReplicaRequest> replicasProxy) {
super(replicasProxy);
}

@Override
protected void performOnReplicaProxy(
ReplicationProxyRequest<ReplicaRequest> proxyRequest,
ReplicationMode replicationMode,
BiConsumer<Consumer<ActionListener<ReplicaResponse>>, ReplicationProxyRequest<ReplicaRequest>> 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;
}
}
Original file line number Diff line number Diff line change
@@ -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;
}
Loading

0 comments on commit 98e4eb6

Please sign in to comment.