Let primary own its replication group (#25692)
Currently replication and recovery are both coordinated through the latest cluster state available on the ClusterService as well as through the GlobalCheckpointTracker (to have consistent local/global checkpoint information), making it difficult to understand the relation between recovery and replication, and requiring some tricky checks in the recovery code to coordinate between the two. This commit makes the primary the single owner of its replication group, which simplifies the replication model and allows to clean up corner cases we have in our recovery code. It also reduces the dependencies in the code, so that neither RecoverySourceXXX nor ReplicationOperation need access to the latest state on ClusterService anymore. Finally, it gives us the property that in-sync shard copies won't receive global checkpoint updates which are above their local checkpoint (relates #25485).
This commit is contained in:
parent
f809a12493
commit
8f0b357651
|
@ -27,15 +27,13 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.UnavailableShardsException;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
||||
import org.elasticsearch.action.support.TransportActions;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
|
@ -49,8 +47,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class ReplicationOperation<
|
||||
Request extends ReplicationRequest<Request>,
|
||||
|
@ -59,7 +57,6 @@ public class ReplicationOperation<
|
|||
> {
|
||||
private final Logger logger;
|
||||
private final Request request;
|
||||
private final Supplier<ClusterState> clusterStateSupplier;
|
||||
private final String opType;
|
||||
private final AtomicInteger totalShards = new AtomicInteger();
|
||||
/**
|
||||
|
@ -86,13 +83,12 @@ public class ReplicationOperation<
|
|||
public ReplicationOperation(Request request, Primary<Request, ReplicaRequest, PrimaryResultT> primary,
|
||||
ActionListener<PrimaryResultT> listener,
|
||||
Replicas<ReplicaRequest> replicas,
|
||||
Supplier<ClusterState> clusterStateSupplier, Logger logger, String opType) {
|
||||
Logger logger, String opType) {
|
||||
this.replicasProxy = replicas;
|
||||
this.primary = primary;
|
||||
this.resultListener = listener;
|
||||
this.logger = logger;
|
||||
this.request = request;
|
||||
this.clusterStateSupplier = clusterStateSupplier;
|
||||
this.opType = opType;
|
||||
}
|
||||
|
||||
|
@ -117,51 +113,45 @@ public class ReplicationOperation<
|
|||
logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request);
|
||||
}
|
||||
|
||||
// we have to get a new state after successfully indexing into the primary in order to honour recovery semantics.
|
||||
// we have to get the replication group after successfully indexing into the primary in order to honour recovery semantics.
|
||||
// we have to make sure that every operation indexed into the primary after recovery start will also be replicated
|
||||
// to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then.
|
||||
ClusterState clusterState = clusterStateSupplier.get();
|
||||
final List<ShardRouting> shards = getShards(primaryId, clusterState);
|
||||
Set<String> inSyncAllocationIds = getInSyncAllocationIds(primaryId, clusterState);
|
||||
|
||||
markUnavailableShardsAsStale(replicaRequest, inSyncAllocationIds, shards);
|
||||
|
||||
performOnReplicas(replicaRequest, primary.globalCheckpoint(), shards);
|
||||
// to the recovery target. If we used an old replication group, we may miss a recovery that has started since then.
|
||||
// we also have to make sure to get the global checkpoint before the replication group, to ensure that the global checkpoint
|
||||
// is valid for this replication group. If we would sample in the reverse, the global checkpoint might be based on a subset
|
||||
// of the sampled replication group, and advanced further than what the given replication group would allow it to.
|
||||
// This would entail that some shards could learn about a global checkpoint that would be higher than its local checkpoint.
|
||||
final long globalCheckpoint = primary.globalCheckpoint();
|
||||
final ReplicationGroup replicationGroup = primary.getReplicationGroup();
|
||||
markUnavailableShardsAsStale(replicaRequest, replicationGroup.getInSyncAllocationIds(), replicationGroup.getRoutingTable());
|
||||
performOnReplicas(replicaRequest, globalCheckpoint, replicationGroup.getRoutingTable());
|
||||
}
|
||||
|
||||
successfulShards.incrementAndGet(); // mark primary as successful
|
||||
decPendingAndFinishIfNeeded();
|
||||
}
|
||||
|
||||
private void markUnavailableShardsAsStale(ReplicaRequest replicaRequest, Set<String> inSyncAllocationIds, List<ShardRouting> shards) {
|
||||
if (inSyncAllocationIds.isEmpty() == false && shards.isEmpty() == false) {
|
||||
Set<String> availableAllocationIds = shards.stream()
|
||||
.map(ShardRouting::allocationId)
|
||||
.filter(Objects::nonNull)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toSet());
|
||||
|
||||
// if inSyncAllocationIds contains allocation ids of shards that don't exist in RoutingTable, mark copies as stale
|
||||
for (String allocationId : Sets.difference(inSyncAllocationIds, availableAllocationIds)) {
|
||||
// mark copy as stale
|
||||
pendingActions.incrementAndGet();
|
||||
replicasProxy.markShardCopyAsStaleIfNeeded(replicaRequest.shardId(), allocationId, replicaRequest.primaryTerm(),
|
||||
ReplicationOperation.this::decPendingAndFinishIfNeeded,
|
||||
ReplicationOperation.this::onPrimaryDemoted,
|
||||
throwable -> decPendingAndFinishIfNeeded()
|
||||
);
|
||||
}
|
||||
private void markUnavailableShardsAsStale(ReplicaRequest replicaRequest, Set<String> inSyncAllocationIds,
|
||||
IndexShardRoutingTable indexShardRoutingTable) {
|
||||
// if inSyncAllocationIds contains allocation ids of shards that don't exist in RoutingTable, mark copies as stale
|
||||
for (String allocationId : Sets.difference(inSyncAllocationIds, indexShardRoutingTable.getAllAllocationIds())) {
|
||||
// mark copy as stale
|
||||
pendingActions.incrementAndGet();
|
||||
replicasProxy.markShardCopyAsStaleIfNeeded(replicaRequest.shardId(), allocationId, replicaRequest.primaryTerm(),
|
||||
ReplicationOperation.this::decPendingAndFinishIfNeeded,
|
||||
ReplicationOperation.this::onPrimaryDemoted,
|
||||
throwable -> decPendingAndFinishIfNeeded()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private void performOnReplicas(final ReplicaRequest replicaRequest, final long globalCheckpoint, final List<ShardRouting> shards) {
|
||||
private void performOnReplicas(final ReplicaRequest replicaRequest, final long globalCheckpoint,
|
||||
final IndexShardRoutingTable indexShardRoutingTable) {
|
||||
final String localNodeId = primary.routingEntry().currentNodeId();
|
||||
// If the index gets deleted after primary operation, we skip replication
|
||||
for (final ShardRouting shard : shards) {
|
||||
for (final ShardRouting shard : indexShardRoutingTable) {
|
||||
if (shard.unassigned()) {
|
||||
if (shard.primary() == false) {
|
||||
totalShards.incrementAndGet();
|
||||
}
|
||||
assert shard.primary() == false : "primary shard should not be unassigned in a replication group: " + shard;
|
||||
totalShards.incrementAndGet();
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -238,23 +228,11 @@ public class ReplicationOperation<
|
|||
*/
|
||||
protected String checkActiveShardCount() {
|
||||
final ShardId shardId = primary.routingEntry().shardId();
|
||||
final String indexName = shardId.getIndexName();
|
||||
final ClusterState state = clusterStateSupplier.get();
|
||||
assert state != null : "replication operation must have access to the cluster state";
|
||||
final ActiveShardCount waitForActiveShards = request.waitForActiveShards();
|
||||
if (waitForActiveShards == ActiveShardCount.NONE) {
|
||||
return null; // not waiting for any shards
|
||||
}
|
||||
IndexRoutingTable indexRoutingTable = state.getRoutingTable().index(indexName);
|
||||
if (indexRoutingTable == null) {
|
||||
logger.trace("[{}] index not found in the routing table", shardId);
|
||||
return "Index " + indexName + " not found in the routing table";
|
||||
}
|
||||
IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shardId.getId());
|
||||
if (shardRoutingTable == null) {
|
||||
logger.trace("[{}] shard not found in the routing table", shardId);
|
||||
return "Shard " + shardId + " not found in the routing table";
|
||||
}
|
||||
final IndexShardRoutingTable shardRoutingTable = primary.getReplicationGroup().getRoutingTable();
|
||||
if (waitForActiveShards.enoughShardsActive(shardRoutingTable)) {
|
||||
return null;
|
||||
} else {
|
||||
|
@ -268,21 +246,6 @@ public class ReplicationOperation<
|
|||
}
|
||||
}
|
||||
|
||||
protected Set<String> getInSyncAllocationIds(ShardId shardId, ClusterState clusterState) {
|
||||
IndexMetaData indexMetaData = clusterState.metaData().index(shardId.getIndex());
|
||||
if (indexMetaData != null) {
|
||||
return indexMetaData.inSyncAllocationIds(shardId.id());
|
||||
}
|
||||
return Collections.emptySet();
|
||||
}
|
||||
|
||||
protected List<ShardRouting> getShards(ShardId shardId, ClusterState state) {
|
||||
// can be null if the index is deleted / closed on us..
|
||||
final IndexShardRoutingTable shardRoutingTable = state.getRoutingTable().shardRoutingTableOrNull(shardId);
|
||||
List<ShardRouting> shards = shardRoutingTable == null ? Collections.emptyList() : shardRoutingTable.shards();
|
||||
return shards;
|
||||
}
|
||||
|
||||
private void decPendingAndFinishIfNeeded() {
|
||||
assert pendingActions.get() > 0 : "pending action count goes below 0 for request [" + request + "]";
|
||||
if (pendingActions.decrementAndGet() == 0) {
|
||||
|
@ -371,6 +334,12 @@ public class ReplicationOperation<
|
|||
*/
|
||||
long globalCheckpoint();
|
||||
|
||||
/**
|
||||
* Returns the current replication group on the primary shard
|
||||
*
|
||||
* @return the replication group
|
||||
*/
|
||||
ReplicationGroup getReplicationGroup();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.elasticsearch.index.IndexService;
|
|||
import org.elasticsearch.index.seqno.SequenceNumbersService;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.ShardNotFoundException;
|
||||
import org.elasticsearch.indices.IndexClosedException;
|
||||
|
@ -383,7 +384,7 @@ public abstract class TransportReplicationAction<
|
|||
Request request, ActionListener<PrimaryResult<ReplicaRequest, Response>> listener,
|
||||
PrimaryShardReference primaryShardReference) {
|
||||
return new ReplicationOperation<>(request, primaryShardReference, listener,
|
||||
replicasProxy, clusterService::state, logger, actionName);
|
||||
replicasProxy, logger, actionName);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -629,7 +630,7 @@ public abstract class TransportReplicationAction<
|
|||
}
|
||||
}
|
||||
|
||||
private IndexShard getIndexShard(ShardId shardId) {
|
||||
protected IndexShard getIndexShard(ShardId shardId) {
|
||||
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
|
||||
return indexService.getShard(shardId.id());
|
||||
}
|
||||
|
@ -1006,6 +1007,11 @@ public abstract class TransportReplicationAction<
|
|||
return indexShard.getGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
return indexShard.getReplicationGroup();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.cluster.routing;
|
|||
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
|
@ -61,6 +62,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
final List<ShardRouting> shards;
|
||||
final List<ShardRouting> activeShards;
|
||||
final List<ShardRouting> assignedShards;
|
||||
final Set<String> allAllocationIds;
|
||||
static final List<ShardRouting> NO_SHARDS = Collections.emptyList();
|
||||
final boolean allShardsStarted;
|
||||
|
||||
|
@ -84,6 +86,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
List<ShardRouting> activeShards = new ArrayList<>();
|
||||
List<ShardRouting> assignedShards = new ArrayList<>();
|
||||
List<ShardRouting> allInitializingShards = new ArrayList<>();
|
||||
Set<String> allAllocationIds = new HashSet<>();
|
||||
boolean allShardsStarted = true;
|
||||
for (ShardRouting shard : shards) {
|
||||
if (shard.primary()) {
|
||||
|
@ -100,9 +103,11 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
if (shard.relocating()) {
|
||||
// create the target initializing shard routing on the node the shard is relocating to
|
||||
allInitializingShards.add(shard.getTargetRelocatingShard());
|
||||
allAllocationIds.add(shard.getTargetRelocatingShard().allocationId().getId());
|
||||
}
|
||||
if (shard.assignedToNode()) {
|
||||
assignedShards.add(shard);
|
||||
allAllocationIds.add(shard.allocationId().getId());
|
||||
}
|
||||
if (shard.state() != ShardRoutingState.STARTED) {
|
||||
allShardsStarted = false;
|
||||
|
@ -119,6 +124,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
this.activeShards = Collections.unmodifiableList(activeShards);
|
||||
this.assignedShards = Collections.unmodifiableList(assignedShards);
|
||||
this.allInitializingShards = Collections.unmodifiableList(allInitializingShards);
|
||||
this.allAllocationIds = Collections.unmodifiableSet(allAllocationIds);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -435,6 +441,25 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
return allShardsStarted;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public ShardRouting getByAllocationId(String allocationId) {
|
||||
for (ShardRouting shardRouting : assignedShards()) {
|
||||
if (shardRouting.allocationId().getId().equals(allocationId)) {
|
||||
return shardRouting;
|
||||
}
|
||||
if (shardRouting.relocating()) {
|
||||
if (shardRouting.getTargetRelocatingShard().allocationId().getId().equals(allocationId)) {
|
||||
return shardRouting.getTargetRelocatingShard();
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public Set<String> getAllAllocationIds() {
|
||||
return allAllocationIds;
|
||||
}
|
||||
|
||||
static class AttributesKey {
|
||||
|
||||
final String[] attributes;
|
||||
|
@ -634,7 +659,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
}
|
||||
|
||||
public static void writeTo(IndexShardRoutingTable indexShard, StreamOutput out) throws IOException {
|
||||
out.writeString(indexShard.shardId().getIndex().getName());
|
||||
indexShard.shardId().getIndex().writeTo(out);
|
||||
writeToThin(indexShard, out);
|
||||
}
|
||||
|
||||
|
@ -648,4 +673,19 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
|
|||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("IndexShardRoutingTable(").append(shardId()).append("){");
|
||||
final int numShards = shards.size();
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
sb.append(shards.get(i).shortSummary());
|
||||
if (i < numShards - 1) {
|
||||
sb.append(", ");
|
||||
}
|
||||
}
|
||||
sb.append("}");
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -156,17 +156,7 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
|
|||
if (shardRoutingTable == null) {
|
||||
return null;
|
||||
}
|
||||
for (ShardRouting shardRouting : shardRoutingTable.assignedShards()) {
|
||||
if (shardRouting.allocationId().getId().equals(allocationId)) {
|
||||
return shardRouting;
|
||||
}
|
||||
if (shardRouting.relocating()) {
|
||||
if (shardRouting.getTargetRelocatingShard().allocationId().getId().equals(allocationId)) {
|
||||
return shardRouting.getTargetRelocatingShard();
|
||||
}
|
||||
}
|
||||
}
|
||||
return null;
|
||||
return shardRoutingTable.getByAllocationId(allocationId);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -82,6 +82,7 @@ public final class ShardRouting implements Writeable, ToXContent {
|
|||
assert !(state == ShardRoutingState.UNASSIGNED && unassignedInfo == null) : "unassigned shard must be created with meta";
|
||||
assert (state == ShardRoutingState.UNASSIGNED || state == ShardRoutingState.INITIALIZING) == (recoverySource != null) : "recovery source only available on unassigned or initializing shard but was " + state;
|
||||
assert recoverySource == null || recoverySource == PeerRecoverySource.INSTANCE || primary : "replica shards always recover from primary";
|
||||
assert (currentNodeId == null) == (state == ShardRoutingState.UNASSIGNED) : "unassigned shard must not be assigned to a node " + this;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
|
|
|
@ -19,12 +19,16 @@
|
|||
|
||||
package org.elasticsearch.index.seqno;
|
||||
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -33,6 +37,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* This class is responsible of tracking the global checkpoint. The global checkpoint is the highest sequence number for which all lower (or
|
||||
|
@ -91,6 +96,8 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
*/
|
||||
long appliedClusterStateVersion;
|
||||
|
||||
IndexShardRoutingTable routingTable;
|
||||
|
||||
/**
|
||||
* Local checkpoint information for all shard copies that are tracked. Has an entry for all shard copies that are either initializing
|
||||
* and / or in-sync, possibly also containing information about unassigned in-sync shard copies. The information that is tracked for
|
||||
|
@ -109,7 +116,12 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
* - computed based on local checkpoints, if the tracker is in primary mode
|
||||
* - received from the primary, if the tracker is in replica mode
|
||||
*/
|
||||
long globalCheckpoint;
|
||||
volatile long globalCheckpoint;
|
||||
|
||||
/**
|
||||
* Cached value for the last replication group that was computed
|
||||
*/
|
||||
volatile ReplicationGroup replicationGroup;
|
||||
|
||||
public static class LocalCheckpointState implements Writeable {
|
||||
|
||||
|
@ -192,6 +204,10 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
// there is at least one in-sync shard copy when the global checkpoint tracker operates in primary mode (i.e. the shard itself)
|
||||
assert !primaryMode || localCheckpoints.values().stream().anyMatch(lcps -> lcps.inSync);
|
||||
|
||||
// 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 + " and replication group is " + replicationGroup;
|
||||
|
||||
// during relocation handoff there are no entries blocking global checkpoint advancement
|
||||
assert !handoffInProgress || pendingInSync.isEmpty() :
|
||||
"entries blocking global checkpoint advancement during relocation handoff: " + pendingInSync;
|
||||
|
@ -204,6 +220,17 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
"global checkpoint is not up-to-date, expected: " +
|
||||
computeGlobalCheckpoint(pendingInSync, localCheckpoints.values(), globalCheckpoint) + " but was: " + globalCheckpoint;
|
||||
|
||||
// we have a routing table iff we have a replication group
|
||||
assert (routingTable == null) == (replicationGroup == null) :
|
||||
"routing table is " + routingTable + " but replication group is " + replicationGroup;
|
||||
|
||||
assert replicationGroup == null || replicationGroup.equals(calculateReplicationGroup()) :
|
||||
"cached replication group out of sync: expected: " + calculateReplicationGroup() + " but was: " + replicationGroup;
|
||||
|
||||
// all assigned shards from the routing table are tracked
|
||||
assert routingTable == null || localCheckpoints.keySet().containsAll(routingTable.getAllAllocationIds()) :
|
||||
"local checkpoints " + localCheckpoints + " not in-sync with routing table " + routingTable;
|
||||
|
||||
for (Map.Entry<String, LocalCheckpointState> entry : localCheckpoints.entrySet()) {
|
||||
// blocking global checkpoint advancement only happens for shards that are not in-sync
|
||||
assert !pendingInSync.contains(entry.getKey()) || !entry.getValue().inSync :
|
||||
|
@ -230,15 +257,32 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
this.globalCheckpoint = globalCheckpoint;
|
||||
this.localCheckpoints = new HashMap<>(1 + indexSettings.getNumberOfReplicas());
|
||||
this.pendingInSync = new HashSet<>();
|
||||
this.routingTable = null;
|
||||
this.replicationGroup = null;
|
||||
assert invariant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current replication group for the shard.
|
||||
*
|
||||
* @return the replication group
|
||||
*/
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
assert primaryMode;
|
||||
return replicationGroup;
|
||||
}
|
||||
|
||||
private ReplicationGroup calculateReplicationGroup() {
|
||||
return new ReplicationGroup(routingTable,
|
||||
localCheckpoints.entrySet().stream().filter(e -> e.getValue().inSync).map(Map.Entry::getKey).collect(Collectors.toSet()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the global checkpoint for the shard.
|
||||
*
|
||||
* @return the global checkpoint
|
||||
*/
|
||||
public synchronized long getGlobalCheckpoint() {
|
||||
public long getGlobalCheckpoint() {
|
||||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
|
@ -284,11 +328,11 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
*
|
||||
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
|
||||
* @param inSyncAllocationIds the allocation IDs of the currently in-sync shard copies
|
||||
* @param initializingAllocationIds the allocation IDs of the currently initializing shard copies
|
||||
* @param routingTable the shard routing table
|
||||
* @param pre60AllocationIds the allocation IDs of shards that are allocated to pre-6.0 nodes
|
||||
*/
|
||||
public synchronized void updateFromMaster(final long applyingClusterStateVersion, final Set<String> inSyncAllocationIds,
|
||||
final Set<String> initializingAllocationIds, final Set<String> pre60AllocationIds) {
|
||||
final IndexShardRoutingTable routingTable, final Set<String> pre60AllocationIds) {
|
||||
assert invariant();
|
||||
if (applyingClusterStateVersion > appliedClusterStateVersion) {
|
||||
// check that the master does not fabricate new in-sync entries out of thin air once we are in primary mode
|
||||
|
@ -297,6 +341,8 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
"update from master in primary mode contains in-sync ids " + inSyncAllocationIds +
|
||||
" that have no matching entries in " + localCheckpoints;
|
||||
// remove entries which don't exist on master
|
||||
Set<String> initializingAllocationIds = routingTable.getAllInitializingShards().stream()
|
||||
.map(ShardRouting::allocationId).map(AllocationId::getId).collect(Collectors.toSet());
|
||||
boolean removedEntries = localCheckpoints.keySet().removeIf(
|
||||
aid -> !inSyncAllocationIds.contains(aid) && !initializingAllocationIds.contains(aid));
|
||||
|
||||
|
@ -325,6 +371,8 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
}
|
||||
}
|
||||
appliedClusterStateVersion = applyingClusterStateVersion;
|
||||
this.routingTable = routingTable;
|
||||
replicationGroup = calculateReplicationGroup();
|
||||
if (primaryMode && removedEntries) {
|
||||
updateGlobalCheckpointOnPrimary();
|
||||
}
|
||||
|
@ -389,6 +437,7 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
}
|
||||
} else {
|
||||
lcps.inSync = true;
|
||||
replicationGroup = calculateReplicationGroup();
|
||||
logger.trace("marked [{}] as in-sync", allocationId);
|
||||
updateGlobalCheckpointOnPrimary();
|
||||
}
|
||||
|
@ -434,6 +483,7 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
pendingInSync.remove(allocationId);
|
||||
pending = false;
|
||||
lcps.inSync = true;
|
||||
replicationGroup = calculateReplicationGroup();
|
||||
logger.trace("marked [{}] as in-sync", allocationId);
|
||||
notifyAllWaiters();
|
||||
}
|
||||
|
@ -502,7 +552,7 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
localCheckpointsCopy.put(entry.getKey(), entry.getValue().copy());
|
||||
}
|
||||
assert invariant();
|
||||
return new PrimaryContext(appliedClusterStateVersion, localCheckpointsCopy);
|
||||
return new PrimaryContext(appliedClusterStateVersion, localCheckpointsCopy, routingTable);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -552,6 +602,8 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
for (Map.Entry<String, LocalCheckpointState> entry : primaryContext.localCheckpoints.entrySet()) {
|
||||
localCheckpoints.put(entry.getKey(), entry.getValue().copy());
|
||||
}
|
||||
routingTable = primaryContext.getRoutingTable();
|
||||
replicationGroup = calculateReplicationGroup();
|
||||
updateGlobalCheckpointOnPrimary();
|
||||
// reapply missed cluster state update
|
||||
// note that if there was no cluster state update between start of the engine of this shard and the call to
|
||||
|
@ -564,19 +616,17 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
assert primaryMode == false;
|
||||
final long lastAppliedClusterStateVersion = appliedClusterStateVersion;
|
||||
final Set<String> inSyncAllocationIds = new HashSet<>();
|
||||
final Set<String> initializingAllocationIds = new HashSet<>();
|
||||
final Set<String> pre60AllocationIds = new HashSet<>();
|
||||
localCheckpoints.entrySet().forEach(entry -> {
|
||||
if (entry.getValue().inSync) {
|
||||
inSyncAllocationIds.add(entry.getKey());
|
||||
} else {
|
||||
initializingAllocationIds.add(entry.getKey());
|
||||
}
|
||||
if (entry.getValue().getLocalCheckpoint() == SequenceNumbersService.PRE_60_NODE_LOCAL_CHECKPOINT) {
|
||||
pre60AllocationIds.add(entry.getKey());
|
||||
}
|
||||
});
|
||||
return () -> updateFromMaster(lastAppliedClusterStateVersion, inSyncAllocationIds, initializingAllocationIds, pre60AllocationIds);
|
||||
final IndexShardRoutingTable lastAppliedRoutingTable = routingTable;
|
||||
return () -> updateFromMaster(lastAppliedClusterStateVersion, inSyncAllocationIds, lastAppliedRoutingTable, pre60AllocationIds);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -622,15 +672,19 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
|
||||
private final long clusterStateVersion;
|
||||
private final Map<String, LocalCheckpointState> localCheckpoints;
|
||||
private final IndexShardRoutingTable routingTable;
|
||||
|
||||
public PrimaryContext(long clusterStateVersion, Map<String, LocalCheckpointState> localCheckpoints) {
|
||||
public PrimaryContext(long clusterStateVersion, Map<String, LocalCheckpointState> localCheckpoints,
|
||||
IndexShardRoutingTable routingTable) {
|
||||
this.clusterStateVersion = clusterStateVersion;
|
||||
this.localCheckpoints = localCheckpoints;
|
||||
this.routingTable = routingTable;
|
||||
}
|
||||
|
||||
public PrimaryContext(StreamInput in) throws IOException {
|
||||
clusterStateVersion = in.readVLong();
|
||||
localCheckpoints = in.readMap(StreamInput::readString, LocalCheckpointState::new);
|
||||
routingTable = IndexShardRoutingTable.Builder.readFrom(in);
|
||||
}
|
||||
|
||||
public long clusterStateVersion() {
|
||||
|
@ -641,10 +695,15 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
return localCheckpoints;
|
||||
}
|
||||
|
||||
public IndexShardRoutingTable getRoutingTable() {
|
||||
return routingTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeVLong(clusterStateVersion);
|
||||
out.writeMap(localCheckpoints, (streamOutput, s) -> out.writeString(s), (streamOutput, lcps) -> lcps.writeTo(out));
|
||||
IndexShardRoutingTable.Builder.writeTo(routingTable, out);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -652,6 +711,7 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
return "PrimaryContext{" +
|
||||
"clusterStateVersion=" + clusterStateVersion +
|
||||
", localCheckpoints=" + localCheckpoints +
|
||||
", routingTable=" + routingTable +
|
||||
'}';
|
||||
}
|
||||
|
||||
|
@ -663,13 +723,15 @@ public class GlobalCheckpointTracker extends AbstractIndexShardComponent {
|
|||
PrimaryContext that = (PrimaryContext) o;
|
||||
|
||||
if (clusterStateVersion != that.clusterStateVersion) return false;
|
||||
return localCheckpoints.equals(that.localCheckpoints);
|
||||
if (routingTable.equals(that.routingTable)) return false;
|
||||
return routingTable.equals(that.routingTable);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = (int) (clusterStateVersion ^ (clusterStateVersion >>> 32));
|
||||
result = 31 * result + localCheckpoints.hashCode();
|
||||
result = 31 * result + routingTable.hashCode();
|
||||
return result;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,10 @@
|
|||
|
||||
package org.elasticsearch.index.seqno;
|
||||
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.util.Set;
|
||||
|
@ -169,6 +171,15 @@ public class SequenceNumbersService extends AbstractIndexShardComponent {
|
|||
return localCheckpointTracker.getCheckpoint();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current replication group for the shard.
|
||||
*
|
||||
* @return the replication group
|
||||
*/
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
return globalCheckpointTracker.getReplicationGroup();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the global checkpoint for the shard.
|
||||
*
|
||||
|
@ -205,17 +216,17 @@ public class SequenceNumbersService extends AbstractIndexShardComponent {
|
|||
|
||||
/**
|
||||
* Notifies the service of the current allocation IDs in the cluster state. See
|
||||
* {@link GlobalCheckpointTracker#updateFromMaster(long, Set, Set, Set)} for details.
|
||||
* {@link GlobalCheckpointTracker#updateFromMaster(long, Set, IndexShardRoutingTable, Set)} for details.
|
||||
*
|
||||
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
|
||||
* @param inSyncAllocationIds the allocation IDs of the currently in-sync shard copies
|
||||
* @param initializingAllocationIds the allocation IDs of the currently initializing shard copies
|
||||
* @param routingTable the shard routing table
|
||||
* @param pre60AllocationIds the allocation IDs of shards that are allocated to pre-6.0 nodes
|
||||
*/
|
||||
public void updateAllocationIdsFromMaster(
|
||||
final long applyingClusterStateVersion, final Set<String> inSyncAllocationIds, final Set<String> initializingAllocationIds,
|
||||
final long applyingClusterStateVersion, final Set<String> inSyncAllocationIds, final IndexShardRoutingTable routingTable,
|
||||
final Set<String> pre60AllocationIds) {
|
||||
globalCheckpointTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, initializingAllocationIds,
|
||||
globalCheckpointTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable,
|
||||
pre60AllocationIds);
|
||||
}
|
||||
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
|
|||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.MappingMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
|
@ -365,7 +366,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
final CheckedBiConsumer<IndexShard, ActionListener<ResyncTask>, IOException> primaryReplicaSyncer,
|
||||
final long applyingClusterStateVersion,
|
||||
final Set<String> inSyncAllocationIds,
|
||||
final Set<String> initializingAllocationIds,
|
||||
final IndexShardRoutingTable routingTable,
|
||||
final Set<String> pre60AllocationIds) throws IOException {
|
||||
final ShardRouting currentRouting;
|
||||
synchronized (mutex) {
|
||||
|
@ -385,7 +386,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
if (newRouting.primary()) {
|
||||
final Engine engine = getEngineOrNull();
|
||||
if (engine != null) {
|
||||
engine.seqNoService().updateAllocationIdsFromMaster(applyingClusterStateVersion, inSyncAllocationIds, initializingAllocationIds, pre60AllocationIds);
|
||||
engine.seqNoService().updateAllocationIdsFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable, pre60AllocationIds);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1721,6 +1722,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
return getEngine().seqNoService().getGlobalCheckpoint();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current replication group for the shard.
|
||||
*
|
||||
* @return the replication group
|
||||
*/
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
verifyPrimary();
|
||||
return getEngine().seqNoService().getReplicationGroup();
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the global checkpoint on a replica shard after it has been updated by the primary.
|
||||
*
|
||||
|
@ -1742,6 +1753,9 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
* while the global checkpoint update may have emanated from the primary when we were in that state, we could subsequently move
|
||||
* to recovery finalization, or even finished recovery before the update arrives here.
|
||||
*/
|
||||
assert state() != IndexShardState.POST_RECOVERY && state() != IndexShardState.STARTED && state() != IndexShardState.RELOCATED :
|
||||
"supposedly in-sync shard copy received a global checkpoint [" + globalCheckpoint + "] " +
|
||||
"that is higher than its local checkpoint [" + localCheckpoint + "]";
|
||||
return;
|
||||
}
|
||||
seqNoService.updateGlobalCheckpointOnReplica(globalCheckpoint, reason);
|
||||
|
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.shard;
|
||||
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Replication group for a shard. Used by a primary shard to coordinate replication and recoveries.
|
||||
*/
|
||||
public class ReplicationGroup {
|
||||
private final IndexShardRoutingTable routingTable;
|
||||
private final Set<String> inSyncAllocationIds;
|
||||
|
||||
public ReplicationGroup(IndexShardRoutingTable routingTable, Set<String> inSyncAllocationIds) {
|
||||
this.routingTable = routingTable;
|
||||
this.inSyncAllocationIds = inSyncAllocationIds;
|
||||
}
|
||||
|
||||
public IndexShardRoutingTable getRoutingTable() {
|
||||
return routingTable;
|
||||
}
|
||||
|
||||
public Set<String> getInSyncAllocationIds() {
|
||||
return inSyncAllocationIds;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
|
||||
ReplicationGroup that = (ReplicationGroup) o;
|
||||
|
||||
if (!routingTable.equals(that.routingTable)) return false;
|
||||
return inSyncAllocationIds.equals(that.inSyncAllocationIds);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
int result = routingTable.hashCode();
|
||||
result = 31 * result + inSyncAllocationIds.hashCode();
|
||||
return result;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "ReplicationGroup{" +
|
||||
"routingTable=" + routingTable +
|
||||
", inSyncAllocationIds=" + inSyncAllocationIds +
|
||||
'}';
|
||||
}
|
||||
|
||||
}
|
|
@ -89,6 +89,7 @@ import java.util.concurrent.ConcurrentMap;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.CLOSED;
|
||||
import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED;
|
||||
|
@ -558,19 +559,21 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
|||
final long primaryTerm = indexMetaData.primaryTerm(shard.shardId().id());
|
||||
final Set<String> inSyncIds = indexMetaData.inSyncAllocationIds(shard.shardId().id());
|
||||
final IndexShardRoutingTable indexShardRoutingTable = routingTable.shardRoutingTable(shardRouting.shardId());
|
||||
final Set<String> initializingIds = indexShardRoutingTable.getAllInitializingShards()
|
||||
.stream()
|
||||
.map(ShardRouting::allocationId)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toSet());
|
||||
final Set<String> pre60AllocationIds = indexShardRoutingTable.assignedShards()
|
||||
.stream()
|
||||
.flatMap(shr -> {
|
||||
if (shr.relocating()) {
|
||||
return Stream.of(shr, shr.getTargetRelocatingShard());
|
||||
} else {
|
||||
return Stream.of(shr);
|
||||
}
|
||||
})
|
||||
.filter(shr -> nodes.get(shr.currentNodeId()).getVersion().before(Version.V_6_0_0_alpha1))
|
||||
.map(ShardRouting::allocationId)
|
||||
.map(AllocationId::getId)
|
||||
.collect(Collectors.toSet());
|
||||
shard.updateShardState(shardRouting, primaryTerm, primaryReplicaSyncer::resync, clusterState.version(),
|
||||
inSyncIds, initializingIds, pre60AllocationIds);
|
||||
inSyncIds, indexShardRoutingTable, pre60AllocationIds);
|
||||
} catch (Exception e) {
|
||||
failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e, clusterState);
|
||||
return;
|
||||
|
@ -730,14 +733,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
|||
* - Updates and persists the new routing value.
|
||||
* - Updates the primary term if this shard is a primary.
|
||||
* - Updates the allocation ids that are tracked by the shard if it is a primary.
|
||||
* See {@link GlobalCheckpointTracker#updateFromMaster(long, Set, Set, Set)} for details.
|
||||
* See {@link GlobalCheckpointTracker#updateFromMaster(long, Set, IndexShardRoutingTable, Set)} for details.
|
||||
*
|
||||
* @param shardRouting the new routing entry
|
||||
* @param primaryTerm the new primary term
|
||||
* @param primaryReplicaSyncer the primary-replica resync action to trigger when a term is increased on a primary
|
||||
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
|
||||
* @param inSyncAllocationIds the allocation ids of the currently in-sync shard copies
|
||||
* @param initializingAllocationIds the allocation ids of the currently initializing shard copies
|
||||
* @param routingTable the shard routing table
|
||||
* @throws IndexShardRelocatedException if shard is marked as relocated and relocation aborted
|
||||
* @throws IOException if shard state could not be persisted
|
||||
*/
|
||||
|
@ -746,7 +749,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
|||
CheckedBiConsumer<IndexShard, ActionListener<ResyncTask>, IOException> primaryReplicaSyncer,
|
||||
long applyingClusterStateVersion,
|
||||
Set<String> inSyncAllocationIds,
|
||||
Set<String> initializingAllocationIds,
|
||||
IndexShardRoutingTable routingTable,
|
||||
Set<String> pre60AllocationIds) throws IOException;
|
||||
}
|
||||
|
||||
|
|
|
@ -20,13 +20,10 @@
|
|||
package org.elasticsearch.indices.recovery;
|
||||
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.shard.IndexEventListener;
|
||||
|
@ -45,7 +42,6 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* The source recovery accepts recovery requests from other peer shards and start the recovery process from this
|
||||
|
@ -61,17 +57,14 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
private final IndicesService indicesService;
|
||||
private final RecoverySettings recoverySettings;
|
||||
|
||||
private final ClusterService clusterService;
|
||||
|
||||
final OngoingRecoveries ongoingRecoveries = new OngoingRecoveries();
|
||||
|
||||
@Inject
|
||||
public PeerRecoverySourceService(Settings settings, TransportService transportService, IndicesService indicesService,
|
||||
RecoverySettings recoverySettings, ClusterService clusterService) {
|
||||
RecoverySettings recoverySettings) {
|
||||
super(settings);
|
||||
this.transportService = transportService;
|
||||
this.indicesService = indicesService;
|
||||
this.clusterService = clusterService;
|
||||
this.recoverySettings = recoverySettings;
|
||||
transportService.registerRequestHandler(Actions.START_RECOVERY, StartRecoveryRequest::new, ThreadPool.Names.GENERIC, new StartRecoveryTransportRequestHandler());
|
||||
}
|
||||
|
@ -88,38 +81,17 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||
final IndexShard shard = indexService.getShard(request.shardId().id());
|
||||
|
||||
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
|
||||
// the index operations will not be routed to it properly
|
||||
RoutingNode node = clusterService.state().getRoutingNodes().node(request.targetNode().getId());
|
||||
if (node == null) {
|
||||
logger.debug("delaying recovery of {} as source node {} is unknown", request.shardId(), request.targetNode());
|
||||
throw new DelayRecoveryException("source node does not have the node [" + request.targetNode() + "] in its state yet..");
|
||||
final ShardRouting routingEntry = shard.routingEntry();
|
||||
|
||||
if (routingEntry.primary() == false || routingEntry.active() == false) {
|
||||
throw new DelayRecoveryException("source shard [" + routingEntry + "] is not an active primary");
|
||||
}
|
||||
|
||||
ShardRouting routingEntry = shard.routingEntry();
|
||||
if (request.isPrimaryRelocation() && (routingEntry.relocating() == false || routingEntry.relocatingNodeId().equals(request.targetNode().getId()) == false)) {
|
||||
logger.debug("delaying recovery of {} as source shard is not marked yet as relocating to {}", request.shardId(), request.targetNode());
|
||||
throw new DelayRecoveryException("source shard is not marked yet as relocating to [" + request.targetNode() + "]");
|
||||
}
|
||||
|
||||
ShardRouting targetShardRouting = node.getByShardId(request.shardId());
|
||||
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");
|
||||
}
|
||||
if (!targetShardRouting.initializing()) {
|
||||
logger.debug("delaying recovery of {} as it is not listed as initializing on the target node {}. known shards state is [{}]",
|
||||
request.shardId(), request.targetNode(), targetShardRouting.state());
|
||||
throw new DelayRecoveryException("source node has the state of the target shard to be [" + targetShardRouting.state() + "], expecting to be [initializing]");
|
||||
}
|
||||
|
||||
if (request.targetAllocationId().equals(targetShardRouting.allocationId().getId()) == false) {
|
||||
logger.debug("delaying recovery of {} due to target allocation id mismatch (expected: [{}], but was: [{}])",
|
||||
request.shardId(), request.targetAllocationId(), targetShardRouting.allocationId().getId());
|
||||
throw new DelayRecoveryException("source node has the state of the target shard to have allocation id [" +
|
||||
targetShardRouting.allocationId().getId() + "], expecting to be [" + request.targetAllocationId() + "]");
|
||||
}
|
||||
|
||||
RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard);
|
||||
logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(), request.targetNode());
|
||||
try {
|
||||
|
@ -157,7 +129,6 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
}
|
||||
if (shardRecoveryContext.recoveryHandlers.isEmpty()) {
|
||||
ongoingRecoveries.remove(shard);
|
||||
assert shardRecoveryContext.onNewRecoveryException == null;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -181,17 +152,10 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
private final class ShardRecoveryContext {
|
||||
final Set<RecoverySourceHandler> recoveryHandlers = new HashSet<>();
|
||||
|
||||
@Nullable
|
||||
private DelayRecoveryException onNewRecoveryException;
|
||||
|
||||
/**
|
||||
* Adds recovery source handler if recoveries are not delayed from starting (see also {@link #delayNewRecoveries}.
|
||||
* Throws {@link DelayRecoveryException} if new recoveries are delayed from starting.
|
||||
* Adds recovery source handler.
|
||||
*/
|
||||
synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, IndexShard shard) {
|
||||
if (onNewRecoveryException != null) {
|
||||
throw onNewRecoveryException;
|
||||
}
|
||||
for (RecoverySourceHandler existingHandler : recoveryHandlers) {
|
||||
if (existingHandler.getRequest().targetAllocationId().equals(request.targetAllocationId())) {
|
||||
throw new DelayRecoveryException("recovery with same target already registered, waiting for " +
|
||||
|
@ -208,29 +172,9 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
|||
final RemoteRecoveryTargetHandler recoveryTarget =
|
||||
new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService,
|
||||
request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime));
|
||||
Supplier<Long> currentClusterStateVersionSupplier = () -> clusterService.state().getVersion();
|
||||
handler = new RecoverySourceHandler(shard, recoveryTarget, request, currentClusterStateVersionSupplier,
|
||||
this::delayNewRecoveries, recoverySettings.getChunkSize().bytesAsInt(), settings);
|
||||
handler = new RecoverySourceHandler(shard, recoveryTarget, request, recoverySettings.getChunkSize().bytesAsInt(), settings);
|
||||
return handler;
|
||||
}
|
||||
|
||||
/**
|
||||
* Makes new recoveries throw a {@link DelayRecoveryException} with the provided message.
|
||||
*
|
||||
* Throws {@link IllegalStateException} if new recoveries are already being delayed.
|
||||
*/
|
||||
synchronized Releasable delayNewRecoveries(String exceptionMessage) throws IllegalStateException {
|
||||
if (onNewRecoveryException != null) {
|
||||
throw new IllegalStateException("already delaying recoveries");
|
||||
}
|
||||
onNewRecoveryException = new DelayRecoveryException(exceptionMessage);
|
||||
return this::unblockNewRecoveries;
|
||||
}
|
||||
|
||||
|
||||
private synchronized void unblockNewRecoveries() {
|
||||
onNewRecoveryException = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.lucene.util.ArrayUtil;
|
|||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.StopWatch;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
|
@ -64,7 +66,6 @@ import java.util.Comparator;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
/**
|
||||
|
@ -86,8 +87,6 @@ public class RecoverySourceHandler {
|
|||
private final int shardId;
|
||||
// Request containing source and target node information
|
||||
private final StartRecoveryRequest request;
|
||||
private final Supplier<Long> currentClusterStateVersionSupplier;
|
||||
private final Function<String, Releasable> delayNewRecoveries;
|
||||
private final int chunkSizeInBytes;
|
||||
private final RecoveryTargetHandler recoveryTarget;
|
||||
|
||||
|
@ -111,15 +110,11 @@ public class RecoverySourceHandler {
|
|||
|
||||
public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget,
|
||||
final StartRecoveryRequest request,
|
||||
final Supplier<Long> currentClusterStateVersionSupplier,
|
||||
Function<String, Releasable> delayNewRecoveries,
|
||||
final int fileChunkSizeInBytes,
|
||||
final Settings nodeSettings) {
|
||||
this.shard = shard;
|
||||
this.recoveryTarget = recoveryTarget;
|
||||
this.request = request;
|
||||
this.currentClusterStateVersionSupplier = currentClusterStateVersionSupplier;
|
||||
this.delayNewRecoveries = delayNewRecoveries;
|
||||
this.shardId = this.request.shardId().id();
|
||||
this.logger = Loggers.getLogger(getClass(), nodeSettings, request.shardId(), "recover to " + request.targetNode().getName());
|
||||
this.chunkSizeInBytes = fileChunkSizeInBytes;
|
||||
|
@ -134,6 +129,17 @@ public class RecoverySourceHandler {
|
|||
* performs the recovery from the local engine to the target
|
||||
*/
|
||||
public RecoveryResponse recoverToTarget() throws IOException {
|
||||
runUnderPrimaryPermit(() -> {
|
||||
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;
|
||||
});
|
||||
|
||||
try (Translog.View translogView = shard.acquireTranslogView()) {
|
||||
|
||||
final long startingSeqNo;
|
||||
|
@ -168,7 +174,7 @@ public class RecoverySourceHandler {
|
|||
}
|
||||
}
|
||||
|
||||
cancellableThreads.execute(() -> runUnderOperationPermit(() -> shard.initiateTracking(request.targetAllocationId())));
|
||||
runUnderPrimaryPermit(() -> shard.initiateTracking(request.targetAllocationId()));
|
||||
|
||||
try {
|
||||
prepareTargetForTranslog(translogView.estimateTotalOperations(startingSeqNo));
|
||||
|
@ -176,24 +182,6 @@ public class RecoverySourceHandler {
|
|||
throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e);
|
||||
}
|
||||
|
||||
// engine was just started at the end of phase1
|
||||
if (shard.state() == IndexShardState.RELOCATED) {
|
||||
assert request.isPrimaryRelocation() == false :
|
||||
"recovery target should not retry primary relocation if previous attempt made it past finalization step";
|
||||
/*
|
||||
* The primary shard has been relocated while we copied files. This means that we can't guarantee any more that all
|
||||
* operations that were replicated during the file copy (when the target engine was not yet opened) will be present in the
|
||||
* local translog and thus will be resent on phase2. The reason is that an operation replicated by the target primary is
|
||||
* sent to the recovery target and the local shard (old primary) concurrently, meaning it may have arrived at the recovery
|
||||
* target before we opened the engine and is still in-flight on the local shard.
|
||||
*
|
||||
* Checking the relocated status here, after we opened the engine on the target, is safe because primary relocation waits
|
||||
* for all ongoing operations to complete and be fully replicated. Therefore all future operation by the new primary are
|
||||
* guaranteed to reach the target shard when its engine is open.
|
||||
*/
|
||||
throw new IndexShardRelocatedException(request.shardId());
|
||||
}
|
||||
|
||||
logger.trace("snapshot translog for recovery; current size is [{}]", translogView.estimateTotalOperations(startingSeqNo));
|
||||
final long targetLocalCheckpoint;
|
||||
try {
|
||||
|
@ -207,15 +195,19 @@ public class RecoverySourceHandler {
|
|||
return response;
|
||||
}
|
||||
|
||||
private void runUnderOperationPermit(CancellableThreads.Interruptable runnable) throws InterruptedException {
|
||||
final PlainActionFuture<Releasable> onAcquired = new PlainActionFuture<>();
|
||||
shard.acquirePrimaryOperationPermit(onAcquired, ThreadPool.Names.SAME);
|
||||
try (Releasable ignored = onAcquired.actionGet()) {
|
||||
if (shard.state() == IndexShardState.RELOCATED) {
|
||||
throw new IndexShardRelocatedException(shard.shardId());
|
||||
private void runUnderPrimaryPermit(CancellableThreads.Interruptable runnable) {
|
||||
cancellableThreads.execute(() -> {
|
||||
final PlainActionFuture<Releasable> onAcquired = new PlainActionFuture<>();
|
||||
shard.acquirePrimaryOperationPermit(onAcquired, ThreadPool.Names.SAME);
|
||||
try (Releasable ignored = onAcquired.actionGet()) {
|
||||
// check that the IndexShard still has the primary authority. This needs to be checked under operation permit to prevent
|
||||
// races, as IndexShard will change to RELOCATED only when it holds all operation permits, see IndexShard.relocated()
|
||||
if (shard.state() == IndexShardState.RELOCATED) {
|
||||
throw new IndexShardRelocatedException(shard.shardId());
|
||||
}
|
||||
runnable.run();
|
||||
}
|
||||
runnable.run();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -468,30 +460,19 @@ public class RecoverySourceHandler {
|
|||
cancellableThreads.checkForCancel();
|
||||
StopWatch stopWatch = new StopWatch().start();
|
||||
logger.trace("finalizing recovery");
|
||||
cancellableThreads.execute(() -> {
|
||||
/*
|
||||
* Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a
|
||||
* shard as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done
|
||||
* marking the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire
|
||||
* the permit then the state of the shard will be relocated and this recovery will fail.
|
||||
*/
|
||||
runUnderOperationPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint));
|
||||
recoveryTarget.finalizeRecovery(shard.getGlobalCheckpoint());
|
||||
});
|
||||
/*
|
||||
* Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a
|
||||
* shard as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done
|
||||
* marking the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire
|
||||
* the permit then the state of the shard will be relocated and this recovery will fail.
|
||||
*/
|
||||
runUnderPrimaryPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint));
|
||||
cancellableThreads.execute(() -> recoveryTarget.finalizeRecovery(shard.getGlobalCheckpoint()));
|
||||
|
||||
if (request.isPrimaryRelocation()) {
|
||||
// in case of primary relocation we have to ensure that the cluster state on the primary relocation target has all
|
||||
// replica shards that have recovered or are still recovering from the current primary, otherwise replication actions
|
||||
// will not be send to these replicas. To accomplish this, first block new recoveries, then take version of latest cluster
|
||||
// state. This means that no new recovery can be completed based on information of a newer cluster state than the current one.
|
||||
try (Releasable ignored = delayNewRecoveries.apply("primary relocation hand-off in progress or completed for " + shardId)) {
|
||||
final long currentClusterStateVersion = currentClusterStateVersionSupplier.get();
|
||||
logger.trace("waiting on remote node to have cluster state with version [{}]", currentClusterStateVersion);
|
||||
cancellableThreads.execute(() -> recoveryTarget.ensureClusterStateVersion(currentClusterStateVersion));
|
||||
|
||||
logger.trace("performing relocation hand-off");
|
||||
cancellableThreads.execute(() -> shard.relocated("to " + request.targetNode(), recoveryTarget::handoffPrimaryContext));
|
||||
}
|
||||
logger.trace("performing relocation hand-off");
|
||||
// this acquires all IndexShard operation permits and will thus delay new recoveries until it is done
|
||||
cancellableThreads.execute(() -> shard.relocated("to " + request.targetNode(), recoveryTarget::handoffPrimaryContext));
|
||||
/*
|
||||
* if the recovery process fails after setting the shard state to RELOCATED, both relocation source and
|
||||
* target are failed (see {@link IndexShard#updateRoutingEntry}).
|
||||
|
|
|
@ -485,7 +485,7 @@ public class Node implements Closeable {
|
|||
RecoverySettings recoverySettings = new RecoverySettings(settings, settingsModule.getClusterSettings());
|
||||
processRecoverySettings(settingsModule.getClusterSettings(), recoverySettings);
|
||||
b.bind(PeerRecoverySourceService.class).toInstance(new PeerRecoverySourceService(settings, transportService,
|
||||
indicesService, recoverySettings, clusterService));
|
||||
indicesService, recoverySettings));
|
||||
b.bind(PeerRecoveryTargetService.class).toInstance(new PeerRecoveryTargetService(settings, threadPool,
|
||||
transportService, recoverySettings, clusterService));
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
|||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.index.shard.IndexShardNotStartedException;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
|
@ -57,9 +58,7 @@ import java.util.function.Supplier;
|
|||
import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state;
|
||||
import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.stateWithActivePrimary;
|
||||
import static org.hamcrest.Matchers.arrayWithSize;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.hasToString;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
@ -109,11 +108,11 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
|
||||
Request request = new Request(shardId);
|
||||
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
|
||||
final ClusterState finalState = state;
|
||||
final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures);
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm);
|
||||
final ClusterState finalState = state;
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm, () -> finalState);
|
||||
final TestReplicationOperation op = new TestReplicationOperation(request,
|
||||
primary, listener, replicasProxy, () -> finalState);
|
||||
primary, listener, replicasProxy);
|
||||
op.execute();
|
||||
|
||||
assertThat(request.primaryTerm(), equalTo(primaryTerm));
|
||||
|
@ -188,13 +187,13 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
}
|
||||
};
|
||||
AtomicBoolean primaryFailed = new AtomicBoolean();
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) {
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm, () -> finalState) {
|
||||
@Override
|
||||
public void failShard(String message, Exception exception) {
|
||||
assertTrue(primaryFailed.compareAndSet(false, true));
|
||||
}
|
||||
};
|
||||
final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy, () -> finalState);
|
||||
final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, replicasProxy);
|
||||
op.execute();
|
||||
|
||||
assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
|
||||
|
@ -215,30 +214,15 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
} else {
|
||||
stateWithAddedReplicas = state(index, true, ShardRoutingState.RELOCATING);
|
||||
}
|
||||
testClusterStateChangeAfterPrimaryOperation(shardId, initialState, stateWithAddedReplicas);
|
||||
}
|
||||
|
||||
public void testIndexDeletedAfterPrimaryOperation() throws Exception {
|
||||
final String index = "test";
|
||||
final ShardId shardId = new ShardId(index, "_na_", 0);
|
||||
final ClusterState initialState = state(index, true, ShardRoutingState.STARTED, ShardRoutingState.STARTED);
|
||||
final ClusterState stateWithDeletedIndex = state(index + "_new", true, ShardRoutingState.STARTED, ShardRoutingState.RELOCATING);
|
||||
testClusterStateChangeAfterPrimaryOperation(shardId, initialState, stateWithDeletedIndex);
|
||||
}
|
||||
|
||||
|
||||
private void testClusterStateChangeAfterPrimaryOperation(final ShardId shardId,
|
||||
final ClusterState initialState,
|
||||
final ClusterState changedState) throws Exception {
|
||||
AtomicReference<ClusterState> state = new AtomicReference<>(initialState);
|
||||
logger.debug("--> using initial state:\n{}", state.get());
|
||||
final long primaryTerm = initialState.getMetaData().index(shardId.getIndexName()).primaryTerm(shardId.id());
|
||||
final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard();
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) {
|
||||
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm, state::get) {
|
||||
@Override
|
||||
public Result perform(Request request) throws Exception {
|
||||
Result result = super.perform(request);
|
||||
state.set(changedState);
|
||||
state.set(stateWithAddedReplicas);
|
||||
logger.debug("--> state after primary operation:\n{}", state.get());
|
||||
return result;
|
||||
}
|
||||
|
@ -247,7 +231,7 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
Request request = new Request(shardId);
|
||||
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
|
||||
final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener,
|
||||
new TestReplicaProxy(), state::get);
|
||||
new TestReplicaProxy());
|
||||
op.execute();
|
||||
|
||||
assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
|
||||
|
@ -284,8 +268,8 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
|
||||
final ShardRouting primaryShard = shardRoutingTable.primaryShard();
|
||||
final TestReplicationOperation op = new TestReplicationOperation(request,
|
||||
new TestPrimary(primaryShard, primaryTerm),
|
||||
listener, new TestReplicaProxy(), () -> state, logger, "test");
|
||||
new TestPrimary(primaryShard, primaryTerm, () -> state),
|
||||
listener, new TestReplicaProxy(), logger, "test");
|
||||
|
||||
if (passesActiveShardCheck) {
|
||||
assertThat(op.checkActiveShardCount(), nullValue());
|
||||
|
@ -314,7 +298,8 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
|
||||
final boolean fatal = randomBoolean();
|
||||
final AtomicBoolean primaryFailed = new AtomicBoolean();
|
||||
final ReplicationOperation.Primary<Request, Request, TestPrimary.Result> primary = new TestPrimary(primaryRouting, primaryTerm) {
|
||||
final ReplicationOperation.Primary<Request, Request, TestPrimary.Result> primary =
|
||||
new TestPrimary(primaryRouting, primaryTerm, () -> state) {
|
||||
|
||||
@Override
|
||||
public void failShard(String message, Exception exception) {
|
||||
|
@ -338,7 +323,7 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
|
||||
final PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
|
||||
final ReplicationOperation.Replicas<Request> replicas = new TestReplicaProxy(Collections.emptyMap());
|
||||
TestReplicationOperation operation = new TestReplicationOperation(request, primary, listener, replicas, () -> state);
|
||||
TestReplicationOperation operation = new TestReplicationOperation(request, primary, listener, replicas);
|
||||
operation.execute();
|
||||
|
||||
assertThat(primaryFailed.get(), equalTo(fatal));
|
||||
|
@ -395,11 +380,13 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
final long term;
|
||||
final long localCheckpoint;
|
||||
final long globalCheckpoint;
|
||||
final Supplier<ClusterState> clusterStateSupplier;
|
||||
final Map<String, Long> knownLocalCheckpoints = new HashMap<>();
|
||||
|
||||
TestPrimary(ShardRouting routing, long term) {
|
||||
TestPrimary(ShardRouting routing, long term, Supplier<ClusterState> clusterStateSupplier) {
|
||||
this.routing = routing;
|
||||
this.term = term;
|
||||
this.clusterStateSupplier = clusterStateSupplier;
|
||||
this.localCheckpoint = random().nextLong();
|
||||
this.globalCheckpoint = randomNonNegativeLong();
|
||||
}
|
||||
|
@ -461,6 +448,13 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
ClusterState clusterState = clusterStateSupplier.get();
|
||||
return new ReplicationGroup(clusterState.routingTable().shardRoutingTable(routing.shardId()),
|
||||
clusterState.metaData().index(routing.index()).inSyncAllocationIds(routing.id()));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class ReplicaResponse implements ReplicationOperation.ReplicaResponse {
|
||||
|
@ -545,15 +539,14 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
|
||||
class TestReplicationOperation extends ReplicationOperation<Request, Request, TestPrimary.Result> {
|
||||
TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
|
||||
ActionListener<TestPrimary.Result> listener, Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier) {
|
||||
this(request, primary, listener, replicas, clusterStateSupplier, ReplicationOperationTests.this.logger, "test");
|
||||
ActionListener<TestPrimary.Result> listener, Replicas<Request> replicas) {
|
||||
this(request, primary, listener, replicas, ReplicationOperationTests.this.logger, "test");
|
||||
}
|
||||
|
||||
TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
|
||||
ActionListener<TestPrimary.Result> listener,
|
||||
Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier,
|
||||
Logger logger, String opType) {
|
||||
super(request, primary, listener, replicas, clusterStateSupplier, logger, opType);
|
||||
Replicas<Request> replicas, Logger logger, String opType) {
|
||||
super(request, primary, listener, replicas, logger, opType);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -63,6 +63,7 @@ import org.elasticsearch.index.IndexService;
|
|||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.index.shard.ReplicationGroup;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.ShardNotFoundException;
|
||||
import org.elasticsearch.indices.IndexClosedException;
|
||||
|
@ -614,8 +615,11 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
|
||||
// check that at unknown node fails
|
||||
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
||||
ShardRoutingState routingState = randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.STARTED,
|
||||
ShardRoutingState.RELOCATING);
|
||||
proxy.performOn(
|
||||
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
||||
TestShardRouting.newShardRouting(shardId, "NOT THERE",
|
||||
routingState == ShardRoutingState.RELOCATING ? state.nodes().iterator().next().getId() : null, false, routingState),
|
||||
new Request(),
|
||||
randomNonNegativeLong(),
|
||||
listener);
|
||||
|
@ -675,6 +679,13 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
long primaryTerm = clusterService.state().getMetaData().index(index).primaryTerm(0);
|
||||
when(shard.getPrimaryTerm()).thenReturn(primaryTerm);
|
||||
when(shard.routingEntry()).thenReturn(routingEntry);
|
||||
when(shard.getReplicationGroup()).thenReturn(
|
||||
new ReplicationGroup(clusterService.state().routingTable().shardRoutingTable(shardId),
|
||||
clusterService.state().metaData().index(index).inSyncAllocationIds(0)));
|
||||
doAnswer(invocation -> {
|
||||
((ActionListener<Releasable>)invocation.getArguments()[0]).onResponse(() -> {});
|
||||
return null;
|
||||
}).when(shard).acquirePrimaryOperationPermit(any(), anyString());
|
||||
|
||||
AtomicBoolean closed = new AtomicBoolean();
|
||||
Releasable releasable = () -> {
|
||||
|
@ -684,7 +695,12 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
};
|
||||
|
||||
TestAction action =
|
||||
new TestAction(Settings.EMPTY, "testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction, threadPool);
|
||||
new TestAction(Settings.EMPTY, "testSeqNoIsSetOnPrimary", transportService, clusterService, shardStateAction, threadPool) {
|
||||
@Override
|
||||
protected IndexShard getIndexShard(ShardId shardId) {
|
||||
return shard;
|
||||
}
|
||||
};
|
||||
|
||||
TransportReplicationAction<Request, Request, TestResponse>.PrimaryOperationTransportHandler primaryPhase =
|
||||
action.new PrimaryOperationTransportHandler();
|
||||
|
@ -1185,7 +1201,7 @@ public class TransportReplicationActionTests extends ESTestCase {
|
|||
class NoopReplicationOperation extends ReplicationOperation<Request, Request, TestAction.PrimaryResult<Request, TestResponse>> {
|
||||
|
||||
NoopReplicationOperation(Request request, ActionListener<TestAction.PrimaryResult<Request, TestResponse>> listener) {
|
||||
super(request, null, listener, null, null, TransportReplicationActionTests.this.logger, "noop");
|
||||
super(request, null, listener, null, TransportReplicationActionTests.this.logger, "noop");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -258,7 +258,7 @@ public class TransportWriteActionTests extends ESTestCase {
|
|||
transportService.start();
|
||||
transportService.acceptIncomingRequests();
|
||||
ShardStateAction shardStateAction = new ShardStateAction(Settings.EMPTY, clusterService, transportService, null, null, threadPool);
|
||||
TestAction action = action = new TestAction(Settings.EMPTY, "testAction", transportService,
|
||||
TestAction action = new TestAction(Settings.EMPTY, "testAction", transportService,
|
||||
clusterService, shardStateAction, threadPool);
|
||||
ReplicationOperation.Replicas proxy = action.newReplicasProxy();
|
||||
final String index = "test";
|
||||
|
@ -269,8 +269,11 @@ public class TransportWriteActionTests extends ESTestCase {
|
|||
|
||||
// check that at unknown node fails
|
||||
PlainActionFuture<ReplicaResponse> listener = new PlainActionFuture<>();
|
||||
ShardRoutingState routingState = randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.STARTED,
|
||||
ShardRoutingState.RELOCATING);
|
||||
proxy.performOn(
|
||||
TestShardRouting.newShardRouting(shardId, "NOT THERE", false, randomFrom(ShardRoutingState.values())),
|
||||
TestShardRouting.newShardRouting(shardId, "NOT THERE",
|
||||
routingState == ShardRoutingState.RELOCATING ? state.nodes().iterator().next().getId() : null, false, routingState),
|
||||
new TestRequest(),
|
||||
randomNonNegativeLong(), listener);
|
||||
assertTrue(listener.isDone());
|
||||
|
|
|
@ -208,7 +208,9 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
|
|||
}
|
||||
|
||||
private ShardRouting nonExistentShardRouting(Index index, List<String> nodeIds, boolean primary) {
|
||||
return TestShardRouting.newShardRouting(new ShardId(index, 0), randomFrom(nodeIds), primary, randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.RELOCATING, ShardRoutingState.STARTED));
|
||||
ShardRoutingState state = randomFrom(ShardRoutingState.INITIALIZING, ShardRoutingState.RELOCATING, ShardRoutingState.STARTED);
|
||||
return TestShardRouting.newShardRouting(new ShardId(index, 0), randomFrom(nodeIds),
|
||||
state == ShardRoutingState.RELOCATING ? randomFrom(nodeIds) : null, primary, state);
|
||||
}
|
||||
|
||||
private static void assertTasksSuccessful(
|
||||
|
|
|
@ -58,7 +58,8 @@ public class ShardRoutingTests extends ESTestCase {
|
|||
|
||||
private ShardRouting randomShardRouting(String index, int shard) {
|
||||
ShardRoutingState state = randomFrom(ShardRoutingState.values());
|
||||
return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1", state != ShardRoutingState.UNASSIGNED && randomBoolean(), state);
|
||||
return TestShardRouting.newShardRouting(index, shard, state == ShardRoutingState.UNASSIGNED ? null : "1",
|
||||
state == ShardRoutingState.RELOCATING ? "2" : null, state != ShardRoutingState.UNASSIGNED && randomBoolean(), state);
|
||||
}
|
||||
|
||||
public void testIsSourceTargetRelocation() {
|
||||
|
@ -131,16 +132,23 @@ public class ShardRoutingTests extends ESTestCase {
|
|||
break;
|
||||
case 2:
|
||||
// change current node
|
||||
otherRouting = new ShardRouting(otherRouting.shardId(), otherRouting.currentNodeId() == null ? "1" : otherRouting.currentNodeId() + "_1", otherRouting.relocatingNodeId(),
|
||||
if (otherRouting.assignedToNode() == false) {
|
||||
unchanged = true;
|
||||
} else {
|
||||
otherRouting = new ShardRouting(otherRouting.shardId(), otherRouting.currentNodeId() + "_1", otherRouting.relocatingNodeId(),
|
||||
otherRouting.primary(), otherRouting.state(), otherRouting.recoverySource(), otherRouting.unassignedInfo(),
|
||||
otherRouting.allocationId(), otherRouting.getExpectedShardSize());
|
||||
}
|
||||
break;
|
||||
case 3:
|
||||
// change relocating node
|
||||
otherRouting = new ShardRouting(otherRouting.shardId(), otherRouting.currentNodeId(),
|
||||
otherRouting.relocatingNodeId() == null ? "1" : otherRouting.relocatingNodeId() + "_1",
|
||||
if (otherRouting.relocating() == false) {
|
||||
unchanged = true;
|
||||
} else {
|
||||
otherRouting = new ShardRouting(otherRouting.shardId(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId() + "_1",
|
||||
otherRouting.primary(), otherRouting.state(), otherRouting.recoverySource(), otherRouting.unassignedInfo(),
|
||||
otherRouting.allocationId(), otherRouting.getExpectedShardSize());
|
||||
}
|
||||
break;
|
||||
case 4:
|
||||
// change recovery source (only works for inactive primaries)
|
||||
|
@ -170,7 +178,9 @@ public class ShardRoutingTests extends ESTestCase {
|
|||
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test");
|
||||
}
|
||||
|
||||
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(), otherRouting.currentNodeId(), otherRouting.relocatingNodeId(),
|
||||
otherRouting = TestShardRouting.newShardRouting(otherRouting.getIndexName(), otherRouting.id(),
|
||||
newState == ShardRoutingState.UNASSIGNED ? null : (otherRouting.currentNodeId() == null ? "1" : otherRouting.currentNodeId()),
|
||||
newState == ShardRoutingState.RELOCATING ? "2" : null,
|
||||
otherRouting.primary(), newState, unassignedInfo);
|
||||
break;
|
||||
}
|
||||
|
|
|
@ -853,7 +853,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
|
|||
// Two shards consuming each 80% of disk space while 70% is allowed, but one is relocating, so shard 0 can stay
|
||||
firstRouting = TestShardRouting.newShardRouting("test", 0, "node1", null, true, ShardRoutingState.STARTED);
|
||||
secondRouting = TestShardRouting.newShardRouting("test", 1, "node1", "node2", true, ShardRoutingState.RELOCATING);
|
||||
ShardRouting fooRouting = TestShardRouting.newShardRouting("foo", 0, "node1", null, true, ShardRoutingState.UNASSIGNED);
|
||||
ShardRouting fooRouting = TestShardRouting.newShardRouting("foo", 0, null, true, ShardRoutingState.UNASSIGNED);
|
||||
firstRoutingNode = new RoutingNode("node1", discoveryNode1, firstRouting, secondRouting);
|
||||
builder = RoutingTable.builder().add(
|
||||
IndexRoutingTable.builder(firstRouting.index())
|
||||
|
|
|
@ -72,6 +72,10 @@ import org.elasticsearch.Version;
|
|||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.support.TransportActions;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -2020,9 +2024,12 @@ public class InternalEngineTests extends ESTestCase {
|
|||
|
||||
try {
|
||||
initialEngine = engine;
|
||||
initialEngine.seqNoService().updateAllocationIdsFromMaster(1L, new HashSet<>(Arrays.asList("primary", "replica")),
|
||||
Collections.emptySet(), Collections.emptySet());
|
||||
initialEngine.seqNoService().activatePrimaryMode("primary", primarySeqNo);
|
||||
final ShardRouting primary = TestShardRouting.newShardRouting(shardId, "node1", true, ShardRoutingState.STARTED);
|
||||
final ShardRouting replica = TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED);
|
||||
initialEngine.seqNoService().updateAllocationIdsFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(),
|
||||
replica.allocationId().getId())),
|
||||
new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build(), Collections.emptySet());
|
||||
initialEngine.seqNoService().activatePrimaryMode(primary.allocationId().getId(), primarySeqNo);
|
||||
for (int op = 0; op < opCount; op++) {
|
||||
final String id;
|
||||
// mostly index, sometimes delete
|
||||
|
@ -2066,8 +2073,9 @@ public class InternalEngineTests extends ESTestCase {
|
|||
// only update rarely as we do it every doc
|
||||
replicaLocalCheckpoint = randomIntBetween(Math.toIntExact(replicaLocalCheckpoint), Math.toIntExact(primarySeqNo));
|
||||
}
|
||||
initialEngine.seqNoService().updateLocalCheckpointForShard("primary", initialEngine.seqNoService().getLocalCheckpoint());
|
||||
initialEngine.seqNoService().updateLocalCheckpointForShard("replica", replicaLocalCheckpoint);
|
||||
initialEngine.seqNoService().updateLocalCheckpointForShard(primary.allocationId().getId(),
|
||||
initialEngine.seqNoService().getLocalCheckpoint());
|
||||
initialEngine.seqNoService().updateLocalCheckpointForShard(replica.allocationId().getId(), replicaLocalCheckpoint);
|
||||
|
||||
if (rarely()) {
|
||||
localCheckpoint = primarySeqNo;
|
||||
|
|
|
@ -41,10 +41,10 @@ import org.elasticsearch.action.support.replication.ReplicationResponse;
|
|||
import org.elasticsearch.action.support.replication.TransportReplicationAction.ReplicaResponse;
|
||||
import org.elasticsearch.action.support.replication.TransportWriteAction;
|
||||
import org.elasticsearch.action.support.replication.TransportWriteActionTestHelper;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||
|
@ -82,6 +82,7 @@ import java.util.concurrent.FutureTask;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
|
||||
|
@ -225,11 +226,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
HashSet<String> activeIds = new HashSet<>();
|
||||
activeIds.addAll(activeIds());
|
||||
activeIds.add(primary.routingEntry().allocationId().getId());
|
||||
HashSet<String> initializingIds = new HashSet<>();
|
||||
initializingIds.addAll(initializingIds());
|
||||
initializingIds.remove(primary.routingEntry().allocationId().getId());
|
||||
primary.updateShardState(ShardRoutingHelper.moveToStarted(primary.routingEntry()), primary.getPrimaryTerm(), null,
|
||||
currentClusterStateVersion.incrementAndGet(), activeIds, initializingIds, Collections.emptySet());
|
||||
ShardRouting startedRoutingEntry = ShardRoutingHelper.moveToStarted(primary.routingEntry());
|
||||
IndexShardRoutingTable routingTable = routingTable(shr -> shr == primary.routingEntry() ? startedRoutingEntry : shr);
|
||||
primary.updateShardState(startedRoutingEntry, primary.getPrimaryTerm(), null,
|
||||
currentClusterStateVersion.incrementAndGet(), activeIds, routingTable, Collections.emptySet());
|
||||
for (final IndexShard replica : replicas) {
|
||||
recoverReplica(replica);
|
||||
}
|
||||
|
@ -282,7 +282,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
primary = replica;
|
||||
assert primary.routingEntry().active() : "only active replicas can be promoted to primary: " + primary.routingEntry();
|
||||
PlainActionFuture<PrimaryReplicaSyncer.ResyncTask> fut = new PlainActionFuture<>();
|
||||
primary.updateShardState(replica.routingEntry().moveActiveReplicaToPrimary(),
|
||||
ShardRouting primaryRouting = replica.routingEntry().moveActiveReplicaToPrimary();
|
||||
IndexShardRoutingTable routingTable = routingTable(shr -> shr == replica.routingEntry() ? primaryRouting : shr);
|
||||
|
||||
primary.updateShardState(primaryRouting,
|
||||
newTerm, (shard, listener) -> primaryReplicaSyncer.resync(shard,
|
||||
new ActionListener<PrimaryReplicaSyncer.ResyncTask>() {
|
||||
@Override
|
||||
|
@ -296,7 +299,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
listener.onFailure(e);
|
||||
fut.onFailure(e);
|
||||
}
|
||||
}), currentClusterStateVersion.incrementAndGet(), activeIds(), initializingIds(), Collections.emptySet());
|
||||
}), currentClusterStateVersion.incrementAndGet(), activeIds(), routingTable, Collections.emptySet());
|
||||
|
||||
return fut;
|
||||
}
|
||||
|
@ -306,9 +309,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
.filter(ShardRouting::active).map(ShardRouting::allocationId).map(AllocationId::getId).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
private synchronized Set<String> initializingIds() {
|
||||
return shardRoutings().stream()
|
||||
.filter(ShardRouting::initializing).map(ShardRouting::allocationId).map(AllocationId::getId).collect(Collectors.toSet());
|
||||
private synchronized IndexShardRoutingTable routingTable(Function<ShardRouting, ShardRouting> transformer) {
|
||||
IndexShardRoutingTable.Builder routingTable = new IndexShardRoutingTable.Builder(primary.shardId());
|
||||
shardRoutings().stream().map(transformer).forEach(routingTable::addShard);
|
||||
return routingTable.build();
|
||||
}
|
||||
|
||||
synchronized boolean removeReplica(IndexShard replica) throws IOException {
|
||||
|
@ -333,7 +337,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
|
||||
boolean markAsRecovering) throws IOException {
|
||||
ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering, activeIds(),
|
||||
initializingIds());
|
||||
routingTable(Function.identity()));
|
||||
}
|
||||
|
||||
public synchronized DiscoveryNode getPrimaryNode() {
|
||||
|
@ -411,8 +415,9 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
}
|
||||
|
||||
private void updateAllocationIDsOnPrimary() throws IOException {
|
||||
|
||||
primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(),
|
||||
activeIds(), initializingIds(), Collections.emptySet());
|
||||
activeIds(), routingTable(Function.identity()), Collections.emptySet());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -433,7 +438,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
|
||||
public void execute() {
|
||||
try {
|
||||
new ReplicationOperation<Request, ReplicaRequest, PrimaryResult>(request, new PrimaryRef(),
|
||||
new ReplicationOperation<>(request, new PrimaryRef(),
|
||||
new ActionListener<PrimaryResult>() {
|
||||
@Override
|
||||
public void onResponse(PrimaryResult result) {
|
||||
|
@ -444,24 +449,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
public void onFailure(Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}, new ReplicasRef(), () -> null, logger, opType) {
|
||||
|
||||
@Override
|
||||
protected List<ShardRouting> getShards(ShardId shardId, ClusterState state) {
|
||||
return replicationGroup.shardRoutings();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String checkActiveShardCount() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<String> getInSyncAllocationIds(ShardId shardId, ClusterState clusterState) {
|
||||
return replicationGroup.shardRoutings().stream().filter(ShardRouting::active).map(r -> r.allocationId().getId())
|
||||
.collect(Collectors.toSet());
|
||||
}
|
||||
}.execute();
|
||||
}, new ReplicasRef(), logger, opType).execute();
|
||||
} catch (Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
|
@ -505,6 +493,12 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
return replicationGroup.getPrimary().getGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
public org.elasticsearch.index.shard.ReplicationGroup getReplicationGroup() {
|
||||
return new org.elasticsearch.index.shard.ReplicationGroup(replicationGroup.routingTable(Function.identity()),
|
||||
replicationGroup.activeIds());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ReplicasRef implements ReplicationOperation.Replicas<ReplicaRequest> {
|
||||
|
|
|
@ -19,6 +19,11 @@
|
|||
|
||||
package org.elasticsearch.index.seqno;
|
||||
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
|
@ -75,22 +80,36 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
|
||||
private final AtomicInteger aIdGenerator = new AtomicInteger();
|
||||
|
||||
private Map<String, Long> randomAllocationsWithLocalCheckpoints(int min, int max) {
|
||||
Map<String, Long> allocations = new HashMap<>();
|
||||
private Map<AllocationId, Long> randomAllocationsWithLocalCheckpoints(int min, int max) {
|
||||
Map<AllocationId, Long> allocations = new HashMap<>();
|
||||
for (int i = randomIntBetween(min, max); i > 0; i--) {
|
||||
allocations.put("id_" + aIdGenerator.incrementAndGet(), (long) randomInt(1000));
|
||||
allocations.put(AllocationId.newInitializing(), (long) randomInt(1000));
|
||||
}
|
||||
return allocations;
|
||||
}
|
||||
|
||||
private static IndexShardRoutingTable routingTable(Set<AllocationId> initializingIds) {
|
||||
ShardId shardId = new ShardId("test", "_na_", 0);
|
||||
IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId);
|
||||
for (AllocationId initializingId : initializingIds) {
|
||||
builder.addShard(TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(10), null, false, ShardRoutingState.INITIALIZING,
|
||||
initializingId));
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
private static Set<String> ids(Set<AllocationId> allocationIds) {
|
||||
return allocationIds.stream().map(AllocationId::getId).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
public void testGlobalCheckpointUpdate() {
|
||||
final long initialClusterStateVersion = randomNonNegativeLong();
|
||||
Map<String, Long> allocations = new HashMap<>();
|
||||
Map<String, Long> activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
Set<String> active = new HashSet<>(activeWithCheckpoints.keySet());
|
||||
Map<AllocationId, Long> allocations = new HashMap<>();
|
||||
Map<AllocationId, Long> activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
Set<AllocationId> active = new HashSet<>(activeWithCheckpoints.keySet());
|
||||
allocations.putAll(activeWithCheckpoints);
|
||||
Map<String, Long> initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||
Set<String> initializing = new HashSet<>(initializingWithCheckpoints.keySet());
|
||||
Map<AllocationId, Long> initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||
Set<AllocationId> initializing = new HashSet<>(initializingWithCheckpoints.keySet());
|
||||
allocations.putAll(initializingWithCheckpoints);
|
||||
assertThat(allocations.size(), equalTo(active.size() + initializing.size()));
|
||||
|
||||
|
@ -113,41 +132,41 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type);
|
||||
});
|
||||
|
||||
tracker.updateFromMaster(initialClusterStateVersion, active, initializing, emptySet());
|
||||
tracker.activatePrimaryMode(active.iterator().next(), NO_OPS_PERFORMED);
|
||||
initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, NO_OPS_PERFORMED));
|
||||
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing), emptySet());
|
||||
tracker.activatePrimaryMode(active.iterator().next().getId(), NO_OPS_PERFORMED);
|
||||
initializing.forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
||||
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId.getId(), allocations.get(aId)));
|
||||
|
||||
assertThat(tracker.getGlobalCheckpoint(), equalTo(minLocalCheckpoint));
|
||||
|
||||
// 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 -> tracker.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||
allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId.getId(), allocations.get(aId)));
|
||||
|
||||
final long minLocalCheckpointAfterUpdates =
|
||||
allocations.entrySet().stream().map(Map.Entry::getValue).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 String extraId = "extra_" + randomAlphaOfLength(5);
|
||||
final AllocationId extraId = AllocationId.newInitializing();
|
||||
|
||||
// first check that adding it without the master blessing doesn't change anything.
|
||||
tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
tracker.updateLocalCheckpoint(extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
assertNull(tracker.localCheckpoints.get(extraId));
|
||||
expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(extraId));
|
||||
expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(extraId.getId()));
|
||||
|
||||
Set<String> newInitializing = new HashSet<>(initializing);
|
||||
Set<AllocationId> newInitializing = new HashSet<>(initializing);
|
||||
newInitializing.add(extraId);
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, active, newInitializing, emptySet());
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing), emptySet());
|
||||
|
||||
tracker.initiateTracking(extraId);
|
||||
tracker.initiateTracking(extraId.getId());
|
||||
|
||||
// now notify for the new id
|
||||
if (randomBoolean()) {
|
||||
tracker.updateLocalCheckpoint(extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
markAllocationIdAsInSyncQuietly(tracker, extraId, randomInt((int) minLocalCheckpointAfterUpdates));
|
||||
tracker.updateLocalCheckpoint(extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
markAllocationIdAsInSyncQuietly(tracker, extraId.getId(), randomInt((int) minLocalCheckpointAfterUpdates));
|
||||
} else {
|
||||
markAllocationIdAsInSyncQuietly(tracker, extraId, minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
markAllocationIdAsInSyncQuietly(tracker, extraId.getId(), minLocalCheckpointAfterUpdates + 1 + randomInt(4));
|
||||
}
|
||||
|
||||
// now it should be incremented
|
||||
|
@ -155,75 +174,75 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testMissingActiveIdsPreventAdvance() {
|
||||
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(2, 5);
|
||||
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||
final Map<String, Long> assigned = new HashMap<>();
|
||||
final Map<AllocationId, Long> active = randomAllocationsWithLocalCheckpoints(2, 5);
|
||||
final Map<AllocationId, Long> initializing = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||
final Map<AllocationId, Long> assigned = new HashMap<>();
|
||||
assigned.putAll(active);
|
||||
assigned.putAll(initializing);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
||||
String primary = active.keySet().iterator().next();
|
||||
tracker.activatePrimaryMode(primary, NO_OPS_PERFORMED);
|
||||
randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
||||
final String missingActiveID = randomFrom(active.keySet());
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet()), emptySet());
|
||||
AllocationId primary = active.keySet().iterator().next();
|
||||
tracker.activatePrimaryMode(primary.getId(), NO_OPS_PERFORMED);
|
||||
randomSubsetOf(initializing.keySet()).forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
||||
final AllocationId missingActiveID = randomFrom(active.keySet());
|
||||
assigned
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> !e.getKey().equals(missingActiveID))
|
||||
.forEach(e -> tracker.updateLocalCheckpoint(e.getKey(), e.getValue()));
|
||||
.forEach(e -> tracker.updateLocalCheckpoint(e.getKey().getId(), e.getValue()));
|
||||
|
||||
if (missingActiveID.equals(primary) == false) {
|
||||
assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||
}
|
||||
// now update all knowledge of all shards
|
||||
assigned.forEach(tracker::updateLocalCheckpoint);
|
||||
assigned.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
||||
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||
}
|
||||
|
||||
public void testMissingInSyncIdsPreventAdvance() {
|
||||
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(2, 5);
|
||||
final Map<AllocationId, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> initializing = randomAllocationsWithLocalCheckpoints(2, 5);
|
||||
logger.info("active: {}, initializing: {}", active, initializing);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
||||
String primary = active.keySet().iterator().next();
|
||||
tracker.activatePrimaryMode(primary, NO_OPS_PERFORMED);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet()), emptySet());
|
||||
AllocationId primary = active.keySet().iterator().next();
|
||||
tracker.activatePrimaryMode(primary.getId(), NO_OPS_PERFORMED);
|
||||
randomSubsetOf(randomIntBetween(1, initializing.size() - 1),
|
||||
initializing.keySet()).forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId, NO_OPS_PERFORMED));
|
||||
initializing.keySet()).forEach(aId -> markAllocationIdAsInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED));
|
||||
|
||||
active.forEach(tracker::updateLocalCheckpoint);
|
||||
active.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
||||
|
||||
assertThat(tracker.getGlobalCheckpoint(), equalTo(NO_OPS_PERFORMED));
|
||||
|
||||
// update again
|
||||
initializing.forEach(tracker::updateLocalCheckpoint);
|
||||
initializing.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
||||
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||
}
|
||||
|
||||
public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() {
|
||||
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), active.keySet(), initializing.keySet(), emptySet());
|
||||
tracker.activatePrimaryMode(active.keySet().iterator().next(), NO_OPS_PERFORMED);
|
||||
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
||||
final Map<AllocationId, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> nonApproved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet()), emptySet());
|
||||
tracker.activatePrimaryMode(active.keySet().iterator().next().getId(), NO_OPS_PERFORMED);
|
||||
initializing.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
||||
nonApproved.keySet().forEach(k ->
|
||||
expectThrows(IllegalStateException.class, () -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED)));
|
||||
expectThrows(IllegalStateException.class, () -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)));
|
||||
|
||||
List<Map<String, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
||||
List<Map<AllocationId, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
||||
Collections.shuffle(allocations, random());
|
||||
allocations.forEach(a -> a.forEach(tracker::updateLocalCheckpoint));
|
||||
allocations.forEach(a -> a.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP)));
|
||||
|
||||
assertThat(tracker.getGlobalCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||
}
|
||||
|
||||
public void testInSyncIdsAreRemovedIfNotValidatedByMaster() {
|
||||
final long initialClusterStateVersion = randomNonNegativeLong();
|
||||
final Map<String, Long> activeToStay = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> initializingToStay = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> activeToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<String, Long> initializingToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Set<String> active = Sets.union(activeToStay.keySet(), activeToBeRemoved.keySet());
|
||||
final Set<String> initializing = Sets.union(initializingToStay.keySet(), initializingToBeRemoved.keySet());
|
||||
final Map<String, Long> allocations = new HashMap<>();
|
||||
final Map<AllocationId, Long> activeToStay = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> initializingToStay = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> activeToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Map<AllocationId, Long> initializingToBeRemoved = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||
final Set<AllocationId> active = Sets.union(activeToStay.keySet(), activeToBeRemoved.keySet());
|
||||
final Set<AllocationId> initializing = Sets.union(initializingToStay.keySet(), initializingToBeRemoved.keySet());
|
||||
final Map<AllocationId, Long> allocations = new HashMap<>();
|
||||
allocations.putAll(activeToStay);
|
||||
if (randomBoolean()) {
|
||||
allocations.putAll(activeToBeRemoved);
|
||||
|
@ -232,25 +251,25 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
if (randomBoolean()) {
|
||||
allocations.putAll(initializingToBeRemoved);
|
||||
}
|
||||
tracker.updateFromMaster(initialClusterStateVersion, active, initializing, emptySet());
|
||||
tracker.activatePrimaryMode(active.iterator().next(), NO_OPS_PERFORMED);
|
||||
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing), emptySet());
|
||||
tracker.activatePrimaryMode(active.iterator().next().getId(), NO_OPS_PERFORMED);
|
||||
if (randomBoolean()) {
|
||||
initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
||||
initializingToStay.keySet().forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
||||
} else {
|
||||
initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k, NO_OPS_PERFORMED));
|
||||
initializing.forEach(k -> markAllocationIdAsInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
allocations.forEach(tracker::updateLocalCheckpoint);
|
||||
allocations.forEach((aid, localCP) -> tracker.updateLocalCheckpoint(aid.getId(), localCP));
|
||||
}
|
||||
|
||||
// now remove shards
|
||||
if (randomBoolean()) {
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, activeToStay.keySet(), initializingToStay.keySet(),
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, ids(activeToStay.keySet()), routingTable(initializingToStay.keySet()),
|
||||
emptySet());
|
||||
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L));
|
||||
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid.getId(), ckp + 10L));
|
||||
} else {
|
||||
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L));
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 2, activeToStay.keySet(), initializingToStay.keySet(),
|
||||
allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid.getId(), ckp + 10L));
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 2, ids(activeToStay.keySet()), routingTable(initializingToStay.keySet()),
|
||||
emptySet());
|
||||
}
|
||||
|
||||
|
@ -265,16 +284,16 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final int globalCheckpoint = randomIntBetween(localCheckpoint + 1, 64);
|
||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||
final AtomicBoolean complete = new AtomicBoolean();
|
||||
final String inSyncAllocationId =randomAlphaOfLength(16);
|
||||
final String trackingAllocationId = randomAlphaOfLength(16);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId),
|
||||
Collections.singleton(trackingAllocationId), emptySet());
|
||||
tracker.activatePrimaryMode(inSyncAllocationId, globalCheckpoint);
|
||||
final AllocationId inSyncAllocationId = AllocationId.newInitializing();
|
||||
final AllocationId trackingAllocationId = AllocationId.newInitializing();
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
||||
routingTable(Collections.singleton(trackingAllocationId)), emptySet());
|
||||
tracker.activatePrimaryMode(inSyncAllocationId.getId(), globalCheckpoint);
|
||||
final Thread thread = new Thread(() -> {
|
||||
try {
|
||||
// synchronize starting with the test thread
|
||||
barrier.await();
|
||||
tracker.markAllocationIdAsInSync(trackingAllocationId, localCheckpoint);
|
||||
tracker.markAllocationIdAsInSync(trackingAllocationId.getId(), localCheckpoint);
|
||||
complete.set(true);
|
||||
// synchronize with the test thread checking if we are no longer waiting
|
||||
barrier.await();
|
||||
|
@ -291,18 +310,18 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final List<Integer> elements = IntStream.rangeClosed(0, globalCheckpoint - 1).boxed().collect(Collectors.toList());
|
||||
Randomness.shuffle(elements);
|
||||
for (int i = 0; i < elements.size(); i++) {
|
||||
tracker.updateLocalCheckpoint(trackingAllocationId, elements.get(i));
|
||||
tracker.updateLocalCheckpoint(trackingAllocationId.getId(), elements.get(i));
|
||||
assertFalse(complete.get());
|
||||
assertFalse(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId).inSync);
|
||||
assertBusy(() -> assertTrue(tracker.pendingInSync.contains(trackingAllocationId)));
|
||||
assertFalse(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId.getId()).inSync);
|
||||
assertBusy(() -> assertTrue(tracker.pendingInSync.contains(trackingAllocationId.getId())));
|
||||
}
|
||||
|
||||
tracker.updateLocalCheckpoint(trackingAllocationId, randomIntBetween(globalCheckpoint, 64));
|
||||
tracker.updateLocalCheckpoint(trackingAllocationId.getId(), randomIntBetween(globalCheckpoint, 64));
|
||||
// synchronize with the waiting thread to mark that it is complete
|
||||
barrier.await();
|
||||
assertTrue(complete.get());
|
||||
assertTrue(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId).inSync);
|
||||
assertFalse(tracker.pendingInSync.contains(trackingAllocationId));
|
||||
assertTrue(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId.getId()).inSync);
|
||||
assertFalse(tracker.pendingInSync.contains(trackingAllocationId.getId()));
|
||||
|
||||
thread.join();
|
||||
}
|
||||
|
@ -312,11 +331,11 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final int globalCheckpoint = randomIntBetween(localCheckpoint + 1, 64);
|
||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||
final AtomicBoolean interrupted = new AtomicBoolean();
|
||||
final String inSyncAllocationId = randomAlphaOfLength(16);
|
||||
final String trackingAllocationId = randomAlphaOfLength(32);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId),
|
||||
Collections.singleton(trackingAllocationId), emptySet());
|
||||
tracker.activatePrimaryMode(inSyncAllocationId, globalCheckpoint);
|
||||
final AllocationId inSyncAllocationId = AllocationId.newInitializing();
|
||||
final AllocationId trackingAllocationId = AllocationId.newInitializing();
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
||||
routingTable(Collections.singleton(trackingAllocationId)), emptySet());
|
||||
tracker.activatePrimaryMode(inSyncAllocationId.getId(), globalCheckpoint);
|
||||
final Thread thread = new Thread(() -> {
|
||||
try {
|
||||
// synchronize starting with the test thread
|
||||
|
@ -325,7 +344,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
throw new RuntimeException(e);
|
||||
}
|
||||
try {
|
||||
tracker.markAllocationIdAsInSync(trackingAllocationId, localCheckpoint);
|
||||
tracker.markAllocationIdAsInSync(trackingAllocationId.getId(), localCheckpoint);
|
||||
} catch (final InterruptedException e) {
|
||||
interrupted.set(true);
|
||||
// synchronize with the test thread checking if we are interrupted
|
||||
|
@ -356,81 +375,89 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final long initialClusterStateVersion = randomNonNegativeLong();
|
||||
final int numberOfActiveAllocationsIds = randomIntBetween(2, 16);
|
||||
final int numberOfInitializingIds = randomIntBetween(2, 16);
|
||||
final Tuple<Set<String>, Set<String>> activeAndInitializingAllocationIds =
|
||||
final Tuple<Set<AllocationId>, Set<AllocationId>> activeAndInitializingAllocationIds =
|
||||
randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
||||
final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
||||
final Set<String> initializingIds = activeAndInitializingAllocationIds.v2();
|
||||
tracker.updateFromMaster(initialClusterStateVersion, activeAllocationIds, initializingIds, emptySet());
|
||||
String primaryId = activeAllocationIds.iterator().next();
|
||||
tracker.activatePrimaryMode(primaryId, NO_OPS_PERFORMED);
|
||||
final Set<AllocationId> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
||||
final Set<AllocationId> initializingIds = activeAndInitializingAllocationIds.v2();
|
||||
IndexShardRoutingTable routingTable = routingTable(initializingIds);
|
||||
tracker.updateFromMaster(initialClusterStateVersion, ids(activeAllocationIds), routingTable, emptySet());
|
||||
AllocationId primaryId = activeAllocationIds.iterator().next();
|
||||
tracker.activatePrimaryMode(primaryId.getId(), 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).inSync));
|
||||
assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync));
|
||||
assertTrue(
|
||||
activeAllocationIds
|
||||
.stream()
|
||||
.filter(a -> a.equals(primaryId) == false)
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint()
|
||||
== SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
||||
assertTrue(initializingIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
||||
assertTrue(initializingIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync));
|
||||
assertTrue(
|
||||
initializingIds
|
||||
.stream()
|
||||
.filter(a -> a.equals(primaryId) == false)
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint()
|
||||
== SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
||||
|
||||
// now we will remove some allocation IDs from these and ensure that they propagate through
|
||||
final List<String> removingActiveAllocationIds = randomSubsetOf(activeAllocationIds);
|
||||
final Set<String> newActiveAllocationIds =
|
||||
final Set<AllocationId> removingActiveAllocationIds = new HashSet<>(randomSubsetOf(activeAllocationIds));
|
||||
final Set<AllocationId> newActiveAllocationIds =
|
||||
activeAllocationIds.stream().filter(a -> !removingActiveAllocationIds.contains(a)).collect(Collectors.toSet());
|
||||
final List<String> removingInitializingAllocationIds = randomSubsetOf(initializingIds);
|
||||
final Set<String> newInitializingAllocationIds =
|
||||
final List<AllocationId> removingInitializingAllocationIds = randomSubsetOf(initializingIds);
|
||||
final Set<AllocationId> newInitializingAllocationIds =
|
||||
initializingIds.stream().filter(a -> !removingInitializingAllocationIds.contains(a)).collect(Collectors.toSet());
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, newActiveAllocationIds, newInitializingAllocationIds,
|
||||
routingTable = routingTable(newInitializingAllocationIds);
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 1, ids(newActiveAllocationIds), routingTable,
|
||||
emptySet());
|
||||
assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
||||
assertTrue(removingActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a) == null));
|
||||
assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
||||
assertTrue(removingInitializingAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a) == null));
|
||||
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(randomAlphaOfLength(64));
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 2, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
||||
assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
||||
newInitializingAllocationIds.add(AllocationId.newInitializing());
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 2, ids(newActiveAllocationIds), routingTable(newInitializingAllocationIds),
|
||||
emptySet());
|
||||
assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync));
|
||||
assertTrue(
|
||||
newActiveAllocationIds
|
||||
.stream()
|
||||
.filter(a -> a.equals(primaryId) == false)
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint()
|
||||
== SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
||||
assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).inSync));
|
||||
assertTrue(newInitializingAllocationIds.stream().noneMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync));
|
||||
assertTrue(
|
||||
newInitializingAllocationIds
|
||||
.stream()
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a).getLocalCheckpoint()
|
||||
.allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).getLocalCheckpoint()
|
||||
== SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
||||
|
||||
// the tracking allocation IDs should play no role in determining the global checkpoint
|
||||
final Map<String, Integer> activeLocalCheckpoints =
|
||||
final Map<AllocationId, Integer> activeLocalCheckpoints =
|
||||
newActiveAllocationIds.stream().collect(Collectors.toMap(Function.identity(), a -> randomIntBetween(1, 1024)));
|
||||
activeLocalCheckpoints.forEach((a, l) -> tracker.updateLocalCheckpoint(a, l));
|
||||
final Map<String, Integer> initializingLocalCheckpoints =
|
||||
activeLocalCheckpoints.forEach((a, l) -> tracker.updateLocalCheckpoint(a.getId(), l));
|
||||
final Map<AllocationId, Integer> initializingLocalCheckpoints =
|
||||
newInitializingAllocationIds.stream().collect(Collectors.toMap(Function.identity(), a -> randomIntBetween(1, 1024)));
|
||||
initializingLocalCheckpoints.forEach((a, l) -> tracker.updateLocalCheckpoint(a, l));
|
||||
initializingLocalCheckpoints.forEach((a, l) -> tracker.updateLocalCheckpoint(a.getId(), l));
|
||||
assertTrue(
|
||||
activeLocalCheckpoints
|
||||
.entrySet()
|
||||
.stream()
|
||||
.allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey()).getLocalCheckpoint() == e.getValue()));
|
||||
.allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey().getId()).getLocalCheckpoint() == e.getValue()));
|
||||
assertTrue(
|
||||
initializingLocalCheckpoints
|
||||
.entrySet()
|
||||
.stream()
|
||||
.allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey()).getLocalCheckpoint() == e.getValue()));
|
||||
.allMatch(e -> tracker.getTrackedLocalCheckpointForShard(e.getKey().getId()).getLocalCheckpoint() == e.getValue()));
|
||||
final long minimumActiveLocalCheckpoint = (long) activeLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
||||
assertThat(tracker.getGlobalCheckpoint(), equalTo(minimumActiveLocalCheckpoint));
|
||||
final long minimumInitailizingLocalCheckpoint = (long) initializingLocalCheckpoints.values().stream().min(Integer::compareTo).get();
|
||||
|
@ -440,14 +467,15 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
randomIntBetween(0, Math.toIntExact(Math.min(minimumActiveLocalCheckpoint, minimumInitailizingLocalCheckpoint) - 1));
|
||||
|
||||
// using a different length than we have been using above ensures that we can not collide with a previous allocation ID
|
||||
final String newSyncingAllocationId = randomAlphaOfLength(128);
|
||||
final AllocationId newSyncingAllocationId = AllocationId.newInitializing();
|
||||
newInitializingAllocationIds.add(newSyncingAllocationId);
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 3, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 3, ids(newActiveAllocationIds), routingTable(newInitializingAllocationIds),
|
||||
emptySet());
|
||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||
final Thread thread = new Thread(() -> {
|
||||
try {
|
||||
barrier.await();
|
||||
tracker.markAllocationIdAsInSync(newSyncingAllocationId, localCheckpoint);
|
||||
tracker.markAllocationIdAsInSync(newSyncingAllocationId.getId(), localCheckpoint);
|
||||
barrier.await();
|
||||
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
|
@ -459,25 +487,27 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
barrier.await();
|
||||
|
||||
assertBusy(() -> {
|
||||
assertTrue(tracker.pendingInSync.contains(newSyncingAllocationId));
|
||||
assertFalse(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
||||
assertTrue(tracker.pendingInSync.contains(newSyncingAllocationId.getId()));
|
||||
assertFalse(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync);
|
||||
});
|
||||
|
||||
tracker.updateLocalCheckpoint(newSyncingAllocationId, randomIntBetween(Math.toIntExact(minimumActiveLocalCheckpoint), 1024));
|
||||
tracker.updateLocalCheckpoint(newSyncingAllocationId.getId(),
|
||||
randomIntBetween(Math.toIntExact(minimumActiveLocalCheckpoint), 1024));
|
||||
|
||||
barrier.await();
|
||||
|
||||
assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId));
|
||||
assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
||||
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 master 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.updateFromMaster(initialClusterStateVersion + 4, newActiveAllocationIds, newInitializingAllocationIds, emptySet());
|
||||
assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId).inSync);
|
||||
assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId));
|
||||
tracker.updateFromMaster(initialClusterStateVersion + 4, ids(newActiveAllocationIds), routingTable(newInitializingAllocationIds),
|
||||
emptySet());
|
||||
assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync);
|
||||
assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -494,13 +524,14 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
*/
|
||||
public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, BrokenBarrierException {
|
||||
|
||||
final String active = randomAlphaOfLength(16);
|
||||
final String initializing = randomAlphaOfLength(32);
|
||||
final AllocationId active = AllocationId.newInitializing();
|
||||
final AllocationId initializing = AllocationId.newInitializing();
|
||||
final CyclicBarrier barrier = new CyclicBarrier(4);
|
||||
|
||||
final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing), emptySet());
|
||||
tracker.activatePrimaryMode(active, activeLocalCheckpoint);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active.getId()),
|
||||
routingTable(Collections.singleton(initializing)), emptySet());
|
||||
tracker.activatePrimaryMode(active.getId(), activeLocalCheckpoint);
|
||||
final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE);
|
||||
final Thread activeThread = new Thread(() -> {
|
||||
try {
|
||||
|
@ -508,7 +539,7 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
tracker.updateLocalCheckpoint(active, nextActiveLocalCheckpoint);
|
||||
tracker.updateLocalCheckpoint(active.getId(), nextActiveLocalCheckpoint);
|
||||
});
|
||||
|
||||
final int initializingLocalCheckpoint = randomIntBetween(0, nextActiveLocalCheckpoint - 1);
|
||||
|
@ -518,13 +549,13 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
tracker.updateLocalCheckpoint(initializing, nextActiveLocalCheckpoint);
|
||||
tracker.updateLocalCheckpoint(initializing.getId(), nextActiveLocalCheckpoint);
|
||||
});
|
||||
|
||||
final Thread markingThread = new Thread(() -> {
|
||||
try {
|
||||
barrier.await();
|
||||
tracker.markAllocationIdAsInSync(initializing, initializingLocalCheckpoint - 1);
|
||||
tracker.markAllocationIdAsInSync(initializing.getId(), initializingLocalCheckpoint - 1);
|
||||
} catch (final BrokenBarrierException | InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
@ -645,16 +676,19 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
assertThat(newPrimary.appliedClusterStateVersion, equalTo(oldPrimary.appliedClusterStateVersion));
|
||||
assertThat(newPrimary.localCheckpoints, equalTo(oldPrimary.localCheckpoints));
|
||||
assertThat(newPrimary.globalCheckpoint, equalTo(oldPrimary.globalCheckpoint));
|
||||
assertThat(newPrimary.routingTable, equalTo(oldPrimary.routingTable));
|
||||
assertThat(newPrimary.replicationGroup, equalTo(oldPrimary.replicationGroup));
|
||||
|
||||
oldPrimary.completeRelocationHandoff();
|
||||
assertFalse(oldPrimary.primaryMode);
|
||||
}
|
||||
|
||||
public void testIllegalStateExceptionIfUnknownAllocationId() {
|
||||
final String active = randomAlphaOfLength(16);
|
||||
final String initializing = randomAlphaOfLength(32);
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active), Collections.singleton(initializing), emptySet());
|
||||
tracker.activatePrimaryMode(active, NO_OPS_PERFORMED);
|
||||
final AllocationId active = AllocationId.newInitializing();
|
||||
final AllocationId initializing = AllocationId.newInitializing();
|
||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active.getId()),
|
||||
routingTable(Collections.singleton(initializing)), emptySet());
|
||||
tracker.activatePrimaryMode(active.getId(), NO_OPS_PERFORMED);
|
||||
|
||||
expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(randomAlphaOfLength(10)));
|
||||
expectThrows(IllegalStateException.class, () -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(10), randomNonNegativeLong()));
|
||||
|
@ -662,21 +696,26 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
|
||||
private static class FakeClusterState {
|
||||
final long version;
|
||||
final Set<String> inSyncIds;
|
||||
final Set<String> initializingIds;
|
||||
final Set<AllocationId> inSyncIds;
|
||||
final IndexShardRoutingTable routingTable;
|
||||
|
||||
private FakeClusterState(long version, Set<String> inSyncIds, Set<String> initializingIds) {
|
||||
private FakeClusterState(long version, Set<AllocationId> inSyncIds, IndexShardRoutingTable routingTable) {
|
||||
this.version = version;
|
||||
this.inSyncIds = Collections.unmodifiableSet(inSyncIds);
|
||||
this.initializingIds = Collections.unmodifiableSet(initializingIds);
|
||||
this.routingTable = routingTable;
|
||||
}
|
||||
|
||||
public Set<String> allIds() {
|
||||
return Sets.union(initializingIds, inSyncIds);
|
||||
public Set<AllocationId> allIds() {
|
||||
return Sets.union(initializingIds(), inSyncIds);
|
||||
}
|
||||
|
||||
public Set<AllocationId> initializingIds() {
|
||||
return routingTable.getAllInitializingShards().stream()
|
||||
.map(ShardRouting::allocationId).collect(Collectors.toSet());
|
||||
}
|
||||
|
||||
public void apply(GlobalCheckpointTracker gcp) {
|
||||
gcp.updateFromMaster(version, inSyncIds, initializingIds, Collections.emptySet());
|
||||
gcp.updateFromMaster(version, ids(inSyncIds), routingTable, Collections.emptySet());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -684,15 +723,15 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
final long initialClusterStateVersion = randomIntBetween(1, Integer.MAX_VALUE);
|
||||
final int numberOfActiveAllocationsIds = randomIntBetween(1, 8);
|
||||
final int numberOfInitializingIds = randomIntBetween(0, 8);
|
||||
final Tuple<Set<String>, Set<String>> activeAndInitializingAllocationIds =
|
||||
final Tuple<Set<AllocationId>, Set<AllocationId>> activeAndInitializingAllocationIds =
|
||||
randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
||||
final Set<String> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
||||
final Set<String> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
||||
return new FakeClusterState(initialClusterStateVersion, activeAllocationIds, initializingAllocationIds);
|
||||
final Set<AllocationId> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
||||
final Set<AllocationId> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
||||
return new FakeClusterState(initialClusterStateVersion, activeAllocationIds, routingTable(initializingAllocationIds));
|
||||
}
|
||||
|
||||
private static void activatePrimary(FakeClusterState clusterState, GlobalCheckpointTracker gcp) {
|
||||
gcp.activatePrimaryMode(randomFrom(clusterState.inSyncIds), randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
||||
gcp.activatePrimaryMode(randomFrom(ids(clusterState.inSyncIds)), randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
||||
}
|
||||
|
||||
private static void randomLocalCheckpointUpdate(GlobalCheckpointTracker gcp) {
|
||||
|
@ -708,34 +747,34 @@ public class GlobalCheckpointTrackerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
private static FakeClusterState randomUpdateClusterState(FakeClusterState clusterState) {
|
||||
final Set<String> initializingIdsToAdd = randomAllocationIdsExcludingExistingIds(clusterState.allIds(), randomInt(2));
|
||||
final Set<String> initializingIdsToRemove = new HashSet<>(
|
||||
randomSubsetOf(randomInt(clusterState.initializingIds.size()), clusterState.initializingIds));
|
||||
final Set<String> inSyncIdsToRemove = new HashSet<>(
|
||||
final Set<AllocationId> initializingIdsToAdd = randomAllocationIdsExcludingExistingIds(clusterState.allIds(), randomInt(2));
|
||||
final Set<AllocationId> initializingIdsToRemove = new HashSet<>(
|
||||
randomSubsetOf(randomInt(clusterState.initializingIds().size()), clusterState.initializingIds()));
|
||||
final Set<AllocationId> inSyncIdsToRemove = new HashSet<>(
|
||||
randomSubsetOf(randomInt(clusterState.inSyncIds.size()), clusterState.inSyncIds));
|
||||
final Set<String> remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove);
|
||||
final Set<AllocationId> remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove);
|
||||
return new FakeClusterState(clusterState.version + randomIntBetween(1, 5),
|
||||
remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds,
|
||||
Sets.difference(Sets.union(clusterState.initializingIds, initializingIdsToAdd), initializingIdsToRemove));
|
||||
routingTable(Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove)));
|
||||
}
|
||||
|
||||
private static Tuple<Set<String>, Set<String>> randomActiveAndInitializingAllocationIds(
|
||||
private static Tuple<Set<AllocationId>, Set<AllocationId>> randomActiveAndInitializingAllocationIds(
|
||||
final int numberOfActiveAllocationsIds,
|
||||
final int numberOfInitializingIds) {
|
||||
final Set<String> activeAllocationIds =
|
||||
IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> randomAlphaOfLength(16) + i).collect(Collectors.toSet());
|
||||
final Set<String> initializingIds = randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfInitializingIds);
|
||||
final Set<AllocationId> activeAllocationIds =
|
||||
IntStream.range(0, numberOfActiveAllocationsIds).mapToObj(i -> AllocationId.newInitializing()).collect(Collectors.toSet());
|
||||
final Set<AllocationId> initializingIds = randomAllocationIdsExcludingExistingIds(activeAllocationIds, numberOfInitializingIds);
|
||||
return Tuple.tuple(activeAllocationIds, initializingIds);
|
||||
}
|
||||
|
||||
private static Set<String> randomAllocationIdsExcludingExistingIds(final Set<String> existingAllocationIds,
|
||||
final int numberOfAllocationIds) {
|
||||
private static Set<AllocationId> randomAllocationIdsExcludingExistingIds(final Set<AllocationId> existingAllocationIds,
|
||||
final int numberOfAllocationIds) {
|
||||
return IntStream.range(0, numberOfAllocationIds).mapToObj(i -> {
|
||||
do {
|
||||
final String newAllocationId = randomAlphaOfLength(16);
|
||||
final AllocationId newAllocationId = AllocationId.newInitializing();
|
||||
// ensure we do not duplicate an allocation ID
|
||||
if (!existingAllocationIds.contains(newAllocationId)) {
|
||||
return newAllocationId + i;
|
||||
return newAllocationId;
|
||||
}
|
||||
} while (true);
|
||||
}).collect(Collectors.toSet());
|
||||
|
|
|
@ -47,6 +47,7 @@ import org.elasticsearch.cluster.metadata.MetaData;
|
|||
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.AllocationId;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||
|
@ -346,7 +347,9 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
ShardRoutingState.STARTED,
|
||||
replicaRouting.allocationId());
|
||||
indexShard.updateShardState(primaryRouting, indexShard.getPrimaryTerm() + 1, (shard, listener) -> {},
|
||||
0L, Collections.singleton(primaryRouting.allocationId().getId()), Collections.emptySet(), Collections.emptySet());
|
||||
0L, Collections.singleton(primaryRouting.allocationId().getId()),
|
||||
new IndexShardRoutingTable.Builder(primaryRouting.shardId()).addShard(primaryRouting).build(),
|
||||
Collections.emptySet());
|
||||
|
||||
final int delayedOperations = scaledRandomIntBetween(1, 64);
|
||||
final CyclicBarrier delayedOperationsBarrier = new CyclicBarrier(1 + delayedOperations);
|
||||
|
@ -421,7 +424,8 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
ShardRoutingState.STARTED,
|
||||
replicaRouting.allocationId());
|
||||
indexShard.updateShardState(primaryRouting, indexShard.getPrimaryTerm() + 1, (shard, listener) -> {},
|
||||
0L, Collections.singleton(primaryRouting.allocationId().getId()), Collections.emptySet(), Collections.emptySet());
|
||||
0L, Collections.singleton(primaryRouting.allocationId().getId()),
|
||||
new IndexShardRoutingTable.Builder(primaryRouting.shardId()).addShard(primaryRouting).build(), Collections.emptySet());
|
||||
|
||||
/*
|
||||
* This operation completing means that the delay operation executed as part of increasing the primary term has completed and the
|
||||
|
@ -463,7 +467,9 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
ShardRouting primaryRouting = TestShardRouting.newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), null,
|
||||
true, ShardRoutingState.STARTED, replicaRouting.allocationId());
|
||||
indexShard.updateShardState(primaryRouting, indexShard.getPrimaryTerm() + 1, (shard, listener) -> {}, 0L,
|
||||
Collections.singleton(indexShard.routingEntry().allocationId().getId()), Collections.emptySet(), Collections.emptySet());
|
||||
Collections.singleton(indexShard.routingEntry().allocationId().getId()),
|
||||
new IndexShardRoutingTable.Builder(indexShard.shardId()).addShard(primaryRouting).build(),
|
||||
Collections.emptySet());
|
||||
} else {
|
||||
indexShard = newStartedShard(true);
|
||||
}
|
||||
|
@ -744,7 +750,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
(s, r) -> resyncLatch.countDown(),
|
||||
1L,
|
||||
Collections.singleton(newRouting.allocationId().getId()),
|
||||
Collections.emptySet(),
|
||||
new IndexShardRoutingTable.Builder(newRouting.shardId()).addShard(newRouting).build(),
|
||||
Collections.emptySet());
|
||||
resyncLatch.await();
|
||||
assertThat(indexShard.getLocalCheckpoint(), equalTo(maxSeqNo));
|
||||
|
@ -1405,7 +1411,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
for (int i = 0; i < totalOps; i++) {
|
||||
indexDoc(primarySource, "test", Integer.toString(i));
|
||||
}
|
||||
IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate("n2", -1));
|
||||
IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1));
|
||||
final IndexShard primaryTarget = newShard(primarySource.routingEntry().getTargetRelocatingShard());
|
||||
updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetaData());
|
||||
recoverReplica(primaryTarget, primarySource);
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.elasticsearch.index.shard;
|
|||
|
||||
import org.elasticsearch.action.resync.ResyncReplicationResponse;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -63,7 +64,7 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase {
|
|||
|
||||
String allocationId = shard.routingEntry().allocationId().getId();
|
||||
shard.updateShardState(shard.routingEntry(), shard.getPrimaryTerm(), null, 1000L, Collections.singleton(allocationId),
|
||||
Collections.emptySet(), Collections.emptySet());
|
||||
new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build(), Collections.emptySet());
|
||||
shard.updateLocalCheckpointForShard(allocationId, globalCheckPoint);
|
||||
assertEquals(globalCheckPoint, shard.getGlobalCheckpoint());
|
||||
|
||||
|
|
|
@ -142,12 +142,10 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
|||
IndexShardRoutingTable shardRoutingTable = state.routingTable().shardRoutingTable(shard.shardId());
|
||||
Set<String> inSyncIds = state.metaData().index(shard.shardId().getIndex())
|
||||
.inSyncAllocationIds(shard.shardId().id());
|
||||
Set<String> initializingIds = shardRoutingTable.getAllInitializingShards().stream()
|
||||
.map(r -> r.allocationId().getId()).collect(Collectors.toSet());
|
||||
assertThat(shard.routingEntry() + " isn't updated with in-sync aIDs", shard.inSyncAllocationIds,
|
||||
equalTo(inSyncIds));
|
||||
assertThat(shard.routingEntry() + " isn't updated with init aIDs", shard.initializingAllocationIds,
|
||||
equalTo(initializingIds));
|
||||
assertThat(shard.routingEntry() + " isn't updated with routing table", shard.routingTable,
|
||||
equalTo(shardRoutingTable));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -326,7 +324,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
|||
private volatile ShardRouting shardRouting;
|
||||
private volatile RecoveryState recoveryState;
|
||||
private volatile Set<String> inSyncAllocationIds;
|
||||
private volatile Set<String> initializingAllocationIds;
|
||||
private volatile IndexShardRoutingTable routingTable;
|
||||
private volatile long term;
|
||||
|
||||
public MockIndexShard(ShardRouting shardRouting, long term) {
|
||||
|
@ -350,7 +348,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
|||
CheckedBiConsumer<IndexShard, ActionListener<ResyncTask>, IOException> primaryReplicaSyncer,
|
||||
long applyingClusterStateVersion,
|
||||
Set<String> inSyncAllocationIds,
|
||||
Set<String> initializingAllocationIds,
|
||||
IndexShardRoutingTable routingTable,
|
||||
Set<String> pre60AllocationIds) throws IOException {
|
||||
failRandomly();
|
||||
assertThat(this.shardId(), equalTo(shardRouting.shardId()));
|
||||
|
@ -364,7 +362,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
|||
term = newPrimaryTerm;
|
||||
this.clusterStateVersion = applyingClusterStateVersion;
|
||||
this.inSyncAllocationIds = inSyncAllocationIds;
|
||||
this.initializingAllocationIds = initializingAllocationIds;
|
||||
this.routingTable = routingTable;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.indices.recovery;
|
||||
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
|
@ -38,8 +37,7 @@ public class PeerRecoverySourceServiceTests extends IndexShardTestCase {
|
|||
IndexShard primary = newStartedShard(true);
|
||||
PeerRecoverySourceService peerRecoverySourceService = new PeerRecoverySourceService(Settings.EMPTY,
|
||||
mock(TransportService.class), mock(IndicesService.class),
|
||||
new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)),
|
||||
mock(ClusterService.class));
|
||||
new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)));
|
||||
StartRecoveryRequest startRecoveryRequest = new StartRecoveryRequest(primary.shardId(), randomAlphaOfLength(10),
|
||||
getFakeDiscoNode("source"), getFakeDiscoNode("target"), null, randomBoolean(), randomLong(), randomLong());
|
||||
RecoverySourceHandler handler = peerRecoverySourceService.ongoingRecoveries.addNewRecovery(startRecoveryRequest, primary);
|
||||
|
|
|
@ -77,9 +77,6 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.emptySet;
|
||||
|
@ -110,7 +107,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
randomNonNegativeLong(),
|
||||
randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : randomNonNegativeLong());
|
||||
Store store = newStore(createTempDir());
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, () -> 0L, e -> () -> {},
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY);
|
||||
Directory dir = store.directory();
|
||||
RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig());
|
||||
|
@ -170,7 +167,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
when(shard.state()).thenReturn(IndexShardState.STARTED);
|
||||
final RecoveryTargetHandler recoveryTarget = mock(RecoveryTargetHandler.class);
|
||||
final RecoverySourceHandler handler =
|
||||
new RecoverySourceHandler(shard, recoveryTarget, request, () -> 0L, e -> () -> {}, fileChunkSizeInBytes, Settings.EMPTY);
|
||||
new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, Settings.EMPTY);
|
||||
final List<Translog.Operation> operations = new ArrayList<>();
|
||||
final int initialNumberOfDocs = randomIntBetween(16, 64);
|
||||
for (int i = 0; i < initialNumberOfDocs; i++) {
|
||||
|
@ -238,7 +235,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
Path tempDir = createTempDir();
|
||||
Store store = newStore(tempDir, false);
|
||||
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, () -> 0L, e -> () -> {},
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) {
|
||||
@Override
|
||||
protected void failEngine(IOException cause) {
|
||||
|
@ -308,7 +305,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
Path tempDir = createTempDir();
|
||||
Store store = newStore(tempDir, false);
|
||||
AtomicBoolean failedEngine = new AtomicBoolean(false);
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, () -> 0L, e -> () -> {},
|
||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request,
|
||||
recoverySettings.getChunkSize().bytesAsInt(), Settings.EMPTY) {
|
||||
@Override
|
||||
protected void failEngine(IOException cause) {
|
||||
|
@ -358,10 +355,9 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
IOUtils.close(store, targetStore);
|
||||
}
|
||||
|
||||
public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws IOException {
|
||||
public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOException {
|
||||
final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service);
|
||||
final boolean attemptSequenceNumberBasedRecovery = randomBoolean();
|
||||
final boolean isTranslogReadyForSequenceNumberBasedRecovery = attemptSequenceNumberBasedRecovery && randomBoolean();
|
||||
final StartRecoveryRequest request =
|
||||
new StartRecoveryRequest(
|
||||
shardId,
|
||||
|
@ -384,25 +380,18 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
return null;
|
||||
}).when(shard).acquirePrimaryOperationPermit(any(), anyString());
|
||||
final AtomicBoolean phase1Called = new AtomicBoolean();
|
||||
// final Engine.IndexCommitRef indexCommitRef = mock(Engine.IndexCommitRef.class);
|
||||
// when(shard.acquireIndexCommit(anyBoolean())).thenReturn(indexCommitRef);
|
||||
// final IndexCommit indexCommit = mock(IndexCommit.class);
|
||||
// when(indexCommitRef.getIndexCommit()).thenReturn(indexCommit);
|
||||
// when(indexCommit.getUserData()).thenReturn(Collections.emptyMap());final AtomicBoolean phase1Called = new AtomicBoolean();
|
||||
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
||||
final AtomicBoolean phase2Called = new AtomicBoolean();
|
||||
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
||||
shard,
|
||||
mock(RecoveryTargetHandler.class),
|
||||
request,
|
||||
() -> 0L,
|
||||
e -> () -> {},
|
||||
recoverySettings.getChunkSize().bytesAsInt(),
|
||||
Settings.EMPTY) {
|
||||
|
||||
@Override
|
||||
boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) {
|
||||
return isTranslogReadyForSequenceNumberBasedRecovery;
|
||||
return randomBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -423,120 +412,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
|
||||
};
|
||||
expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget);
|
||||
// phase1 should only be attempted if we are not doing a sequence-number-based recovery
|
||||
assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery));
|
||||
assertFalse(phase1Called.get());
|
||||
assertFalse(prepareTargetForTranslogCalled.get());
|
||||
assertFalse(phase2Called.get());
|
||||
}
|
||||
|
||||
public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, InterruptedException {
|
||||
final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service);
|
||||
final boolean attemptSequenceNumberBasedRecovery = randomBoolean();
|
||||
final boolean isTranslogReadyForSequenceNumberBasedRecovery = attemptSequenceNumberBasedRecovery && randomBoolean();
|
||||
final StartRecoveryRequest request =
|
||||
new StartRecoveryRequest(
|
||||
shardId,
|
||||
null,
|
||||
new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT),
|
||||
new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT),
|
||||
null,
|
||||
true,
|
||||
randomNonNegativeLong(),
|
||||
attemptSequenceNumberBasedRecovery ? randomNonNegativeLong(): SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
||||
final AtomicBoolean phase1Called = new AtomicBoolean();
|
||||
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
||||
final AtomicBoolean phase2Called = new AtomicBoolean();
|
||||
final AtomicBoolean ensureClusterStateVersionCalled = new AtomicBoolean();
|
||||
final AtomicBoolean recoveriesDelayed = new AtomicBoolean();
|
||||
final AtomicBoolean relocated = new AtomicBoolean();
|
||||
|
||||
final IndexShard shard = mock(IndexShard.class);
|
||||
when(shard.seqNoStats()).thenReturn(mock(SeqNoStats.class));
|
||||
when(shard.segmentStats(anyBoolean())).thenReturn(mock(SegmentsStats.class));
|
||||
final Translog.View translogView = mock(Translog.View.class);
|
||||
when(shard.acquireTranslogView()).thenReturn(translogView);
|
||||
when(shard.state()).then(i -> relocated.get() ? IndexShardState.RELOCATED : IndexShardState.STARTED);
|
||||
doAnswer(i -> {
|
||||
relocated.set(true);
|
||||
assertTrue(recoveriesDelayed.get());
|
||||
return null;
|
||||
}).when(shard).relocated(any(String.class), any(Consumer.class));
|
||||
when(shard.acquireIndexCommit(anyBoolean())).thenReturn(mock(Engine.IndexCommitRef.class));
|
||||
doAnswer(invocationOnMock -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
final ActionListener<Releasable> listener = (ActionListener<Releasable>)invocationOnMock.getArguments()[0];
|
||||
listener.onResponse(() -> {});
|
||||
return null;
|
||||
}).when(shard).acquirePrimaryOperationPermit(any(ActionListener.class), any(String.class));
|
||||
|
||||
// final Engine.IndexCommitRef indexCommitRef = mock(Engine.IndexCommitRef.class);
|
||||
// when(shard.acquireIndexCommit(anyBoolean())).thenReturn(indexCommitRef);
|
||||
// final IndexCommit indexCommit = mock(IndexCommit.class);
|
||||
// when(indexCommitRef.getIndexCommit()).thenReturn(indexCommit);
|
||||
// when(indexCommit.getUserData()).thenReturn(Collections.emptyMap());
|
||||
final Supplier<Long> currentClusterStateVersionSupplier = () -> {
|
||||
assertFalse(ensureClusterStateVersionCalled.get());
|
||||
assertTrue(recoveriesDelayed.get());
|
||||
ensureClusterStateVersionCalled.set(true);
|
||||
return 0L;
|
||||
};
|
||||
|
||||
final Function<String, Releasable> delayNewRecoveries = s -> {
|
||||
// phase1 should only be attempted if we are not doing a sequence-number-based recovery
|
||||
assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery));
|
||||
assertTrue(prepareTargetForTranslogCalled.get());
|
||||
assertTrue(phase2Called.get());
|
||||
|
||||
assertFalse(recoveriesDelayed.get());
|
||||
recoveriesDelayed.set(true);
|
||||
return () -> {
|
||||
assertTrue(recoveriesDelayed.get());
|
||||
recoveriesDelayed.set(false);
|
||||
};
|
||||
};
|
||||
|
||||
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
||||
shard,
|
||||
mock(RecoveryTargetHandler.class),
|
||||
request,
|
||||
currentClusterStateVersionSupplier,
|
||||
delayNewRecoveries,
|
||||
recoverySettings.getChunkSize().bytesAsInt(),
|
||||
Settings.EMPTY) {
|
||||
|
||||
@Override
|
||||
boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) {
|
||||
return isTranslogReadyForSequenceNumberBasedRecovery;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void phase1(final IndexCommit snapshot, final Translog.View translogView, final long startSeqNo) {
|
||||
phase1Called.set(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
void prepareTargetForTranslog(final int totalTranslogOps) throws IOException {
|
||||
prepareTargetForTranslogCalled.set(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
long phase2(long startingSeqNo, Translog.Snapshot snapshot) throws IOException {
|
||||
phase2Called.set(true);
|
||||
return SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
handler.recoverToTarget();
|
||||
assertTrue(ensureClusterStateVersionCalled.get());
|
||||
// phase1 should only be attempted if we are not doing a sequence-number-based recovery
|
||||
assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery));
|
||||
assertTrue(prepareTargetForTranslogCalled.get());
|
||||
assertTrue(phase2Called.get());
|
||||
assertTrue(relocated.get());
|
||||
assertFalse(recoveriesDelayed.get());
|
||||
}
|
||||
|
||||
private Store newStore(Path path) throws IOException {
|
||||
return newStore(path, true);
|
||||
}
|
||||
|
|
|
@ -78,8 +78,9 @@ public class IndicesStoreTests extends ESTestCase {
|
|||
if (state == ShardRoutingState.UNASSIGNED) {
|
||||
unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null);
|
||||
}
|
||||
String currentNodeId = state == ShardRoutingState.UNASSIGNED ? null : randomAlphaOfLength(10);
|
||||
String relocatingNodeId = state == ShardRoutingState.RELOCATING ? randomAlphaOfLength(10) : null;
|
||||
routingTable.addShard(TestShardRouting.newShardRouting("test", i, randomAlphaOfLength(10), relocatingNodeId, j == 0, state, unassignedInfo));
|
||||
routingTable.addShard(TestShardRouting.newShardRouting("test", i, currentNodeId, relocatingNodeId, j == 0, state, unassignedInfo));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
|||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||
|
@ -148,7 +149,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
* (ready to recover from another shard)
|
||||
*/
|
||||
protected IndexShard newShard(boolean primary) throws IOException {
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), "n1", primary,
|
||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), primary,
|
||||
ShardRoutingState.INITIALIZING,
|
||||
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
||||
return newShard(shardRouting);
|
||||
|
@ -369,10 +370,11 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
public static void updateRoutingEntry(IndexShard shard, ShardRouting shardRouting) throws IOException {
|
||||
Set<String> inSyncIds =
|
||||
shardRouting.active() ? Collections.singleton(shardRouting.allocationId().getId()) : Collections.emptySet();
|
||||
Set<String> initializingIds =
|
||||
shardRouting.initializing() ? Collections.singleton(shardRouting.allocationId().getId()) : Collections.emptySet();
|
||||
IndexShardRoutingTable newRoutingTable = new IndexShardRoutingTable.Builder(shardRouting.shardId())
|
||||
.addShard(shardRouting)
|
||||
.build();
|
||||
shard.updateShardState(shardRouting, shard.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(),
|
||||
inSyncIds, initializingIds, Collections.emptySet());
|
||||
inSyncIds, newRoutingTable, Collections.emptySet());
|
||||
}
|
||||
|
||||
protected void recoveryEmptyReplica(IndexShard replica) throws IOException {
|
||||
|
@ -403,9 +405,14 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
final IndexShard primary,
|
||||
final BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
|
||||
final boolean markAsRecovering) throws IOException {
|
||||
IndexShardRoutingTable.Builder newRoutingTable = new IndexShardRoutingTable.Builder(replica.shardId());
|
||||
newRoutingTable.addShard(primary.routingEntry());
|
||||
if (replica.routingEntry().isRelocationTarget() == false) {
|
||||
newRoutingTable.addShard(replica.routingEntry());
|
||||
}
|
||||
recoverReplica(replica, primary, targetSupplier, markAsRecovering,
|
||||
Collections.singleton(primary.routingEntry().allocationId().getId()),
|
||||
Collections.singleton(replica.routingEntry().allocationId().getId()));
|
||||
newRoutingTable.build());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -421,7 +428,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
final BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
|
||||
final boolean markAsRecovering,
|
||||
final Set<String> inSyncIds,
|
||||
final Set<String> initializingIds) throws IOException {
|
||||
final IndexShardRoutingTable routingTable) throws IOException {
|
||||
final DiscoveryNode pNode = getFakeDiscoNode(primary.routingEntry().currentNodeId());
|
||||
final DiscoveryNode rNode = getFakeDiscoNode(replica.routingEntry().currentNodeId());
|
||||
if (markAsRecovering) {
|
||||
|
@ -447,23 +454,30 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
primary,
|
||||
recoveryTarget,
|
||||
request,
|
||||
() -> 0L,
|
||||
e -> () -> {},
|
||||
(int) ByteSizeUnit.MB.toBytes(1),
|
||||
Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), pNode.getName()).build());
|
||||
final ShardRouting initializingReplicaRouting = replica.routingEntry();
|
||||
primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(),
|
||||
inSyncIds, initializingIds, Collections.emptySet());
|
||||
inSyncIds, routingTable, Collections.emptySet());
|
||||
recovery.recoverToTarget();
|
||||
recoveryTarget.markAsDone();
|
||||
Set<String> initializingIdsWithoutReplica = new HashSet<>(initializingIds);
|
||||
initializingIdsWithoutReplica.remove(replica.routingEntry().allocationId().getId());
|
||||
IndexShardRoutingTable newRoutingTable =
|
||||
initializingReplicaRouting.isRelocationTarget() ?
|
||||
new IndexShardRoutingTable.Builder(routingTable)
|
||||
.removeShard(primary.routingEntry())
|
||||
.addShard(replica.routingEntry())
|
||||
.build() :
|
||||
new IndexShardRoutingTable.Builder(routingTable)
|
||||
.removeShard(initializingReplicaRouting)
|
||||
.addShard(replica.routingEntry())
|
||||
.build();
|
||||
Set<String> inSyncIdsWithReplica = new HashSet<>(inSyncIds);
|
||||
inSyncIdsWithReplica.add(replica.routingEntry().allocationId().getId());
|
||||
// update both primary and replica shard state
|
||||
primary.updateShardState(primary.routingEntry(), primary.getPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(),
|
||||
inSyncIdsWithReplica, initializingIdsWithoutReplica, Collections.emptySet());
|
||||
inSyncIdsWithReplica, newRoutingTable, Collections.emptySet());
|
||||
replica.updateShardState(replica.routingEntry().moveToStarted(), replica.getPrimaryTerm(), null,
|
||||
currentClusterStateVersion.get(), inSyncIdsWithReplica, initializingIdsWithoutReplica, Collections.emptySet());
|
||||
currentClusterStateVersion.get(), inSyncIdsWithReplica, newRoutingTable, Collections.emptySet());
|
||||
}
|
||||
|
||||
private Store.MetadataSnapshot getMetadataSnapshotOrEmpty(IndexShard replica) throws IOException {
|
||||
|
|
Loading…
Reference in New Issue