Simplify GlobalCheckpointService and properly hook it for cluster state updates (#20720)
During a recent merge from master, we lost the bridge from IndicesClusterStateService to the GlobalCheckpointService of primary shards, notifying them of changes to the current set of active/initializing shards. This commits add the bridge back (with unit tests). It also simplifies the GlobalCheckpoint tracking to use a simpler model (which makes use the fact that the global check point sync is done periodically). The old integration CheckpointIT test is moved to IndexLevelReplicationTests. I also added similar assertions to RelocationsIT, which surfaced a bug in the primary relocation logic and how it plays with global checkpoint updates. The test is currently await-fixed and will be fixed in a follow up issue.
This commit is contained in:
parent
7c2e761c87
commit
eaa105951f
|
@ -25,15 +25,18 @@ import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
|
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A shard component that is responsible of tracking the global checkpoint. The global checkpoint
|
* A shard component that is responsible of tracking the global checkpoint. The global checkpoint
|
||||||
* is the highest seq_no for which all lower (or equal) seq_no have been processed on all shards that
|
* is the highest seq_no for which all lower (or equal) seq_no have been processed on all shards that
|
||||||
* are currently active. Since shards count as "active" when the master starts them, and before this primary shard
|
* are currently active. Since shards count as "active" when the master starts them, and before this primary shard
|
||||||
* has been notified of this fact, we also include shards in that are in the
|
* has been notified of this fact, we also include shards that have completed recovery. These shards have received
|
||||||
* {@link org.elasticsearch.index.shard.IndexShardState#POST_RECOVERY} state when checking for global checkpoint advancement.
|
* all old operations via the recovery mechanism and are kept up to date by the various replications actions. The set
|
||||||
* We call these shards "in sync" with all operations on the primary (see {@link #inSyncLocalCheckpoints}.
|
* of shards that are taken into account for the global checkpoint calculation are called the "in sync" shards.
|
||||||
*
|
*
|
||||||
* <p>
|
* <p>
|
||||||
* The global checkpoint is maintained by the primary shard and is replicated to all the replicas
|
* The global checkpoint is maintained by the primary shard and is replicated to all the replicas
|
||||||
|
@ -41,15 +44,9 @@ import java.util.Set;
|
||||||
*/
|
*/
|
||||||
public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
||||||
|
|
||||||
/**
|
|
||||||
* This map holds the last known local checkpoint for every shard copy that's active.
|
|
||||||
* All shard copies in this map participate in determining the global checkpoint
|
|
||||||
* keyed by allocation ids
|
|
||||||
*/
|
|
||||||
private final ObjectLongMap<String> activeLocalCheckpoints;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This map holds the last known local checkpoint for every initializing shard copy that's has been brought up
|
* This map holds the last known local checkpoint for every active shard and initializing shard copies that has been brought up
|
||||||
* to speed through recovery. These shards are treated as valid copies and participate in determining the global
|
* to speed through recovery. These shards are treated as valid copies and participate in determining the global
|
||||||
* checkpoint.
|
* checkpoint.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -57,22 +54,19 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
||||||
*/
|
*/
|
||||||
private final ObjectLongMap<String> inSyncLocalCheckpoints; // keyed by allocation ids
|
private final ObjectLongMap<String> inSyncLocalCheckpoints; // keyed by allocation ids
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This map holds the last known local checkpoint for every initializing shard copy that is still undergoing recovery.
|
* This set holds the last set of known valid allocation ids as received by the master. This is important to make sure
|
||||||
* These shards <strong>do not</strong> participate in determining the global checkpoint. This map is needed to make sure that when
|
* shard that are failed or relocated are cleaned up from {@link #inSyncLocalCheckpoints} and do not hold the global
|
||||||
* shards are promoted to {@link #inSyncLocalCheckpoints} we use the highest known checkpoint, even if we index concurrently
|
* checkpoint back
|
||||||
* while recovering the shard.
|
|
||||||
* Keyed by allocation ids
|
|
||||||
*/
|
*/
|
||||||
private final ObjectLongMap<String> trackingLocalCheckpoint;
|
private final Set<String> assignedAllocationIds;
|
||||||
|
|
||||||
private long globalCheckpoint;
|
private long globalCheckpoint;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Initialize the global checkpoint service. The {@code globalCheckpoint}
|
* Initialize the global checkpoint service. The {@code globalCheckpoint}
|
||||||
* should be set to the last known global checkpoint for this shard, or
|
* should be set to the last known global checkpoint for this shard, or
|
||||||
* {@link SequenceNumbersService#NO_OPS_PERFORMED}.
|
* {@link SequenceNumbersService#UNASSIGNED_SEQ_NO}.
|
||||||
*
|
*
|
||||||
* @param shardId the shard this service is providing tracking
|
* @param shardId the shard this service is providing tracking
|
||||||
* local checkpoints for
|
* local checkpoints for
|
||||||
|
@ -83,51 +77,35 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
||||||
*/
|
*/
|
||||||
GlobalCheckpointService(final ShardId shardId, final IndexSettings indexSettings, final long globalCheckpoint) {
|
GlobalCheckpointService(final ShardId shardId, final IndexSettings indexSettings, final long globalCheckpoint) {
|
||||||
super(shardId, indexSettings);
|
super(shardId, indexSettings);
|
||||||
activeLocalCheckpoints = new ObjectLongHashMap<>(1 + indexSettings.getNumberOfReplicas());
|
assert globalCheckpoint >= UNASSIGNED_SEQ_NO : "illegal initial global checkpoint:" + globalCheckpoint;
|
||||||
inSyncLocalCheckpoints = new ObjectLongHashMap<>(indexSettings.getNumberOfReplicas());
|
inSyncLocalCheckpoints = new ObjectLongHashMap<>(1 + indexSettings.getNumberOfReplicas());
|
||||||
trackingLocalCheckpoint = new ObjectLongHashMap<>(indexSettings.getNumberOfReplicas());
|
assignedAllocationIds = new HashSet<>(1 + indexSettings.getNumberOfReplicas());
|
||||||
this.globalCheckpoint = globalCheckpoint;
|
this.globalCheckpoint = globalCheckpoint;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* notifies the service of a local checkpoint. if the checkpoint is lower than the currently known one,
|
* Notifies the service of a local checkpoint. If the checkpoint is lower than the currently known one,
|
||||||
* this is a noop. Last, if the allocation id is not yet known, it is ignored. This to prevent late
|
* this is a noop. Last, if the allocation id is not in sync, it is ignored. This to prevent late
|
||||||
* arrivals from shards that are removed to be re-added.
|
* arrivals from shards that are removed to be re-added.
|
||||||
*/
|
*/
|
||||||
public synchronized void updateLocalCheckpoint(String allocationId, long localCheckpoint) {
|
public synchronized void updateLocalCheckpoint(String allocationId, long localCheckpoint) {
|
||||||
if (updateLocalCheckpointInMap(allocationId, localCheckpoint, activeLocalCheckpoints, "active")) {
|
final int indexOfKey = inSyncLocalCheckpoints.indexOf(allocationId);
|
||||||
return;
|
if (indexOfKey >= 0) {
|
||||||
}
|
final long current = inSyncLocalCheckpoints.indexGet(indexOfKey);
|
||||||
if (updateLocalCheckpointInMap(allocationId, localCheckpoint, inSyncLocalCheckpoints, "inSync")) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
if (updateLocalCheckpointInMap(allocationId, localCheckpoint, trackingLocalCheckpoint, "tracking")) {
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
logger.trace("local checkpoint of [{}] ([{}]) wasn't found in any map. ignoring.", allocationId, localCheckpoint);
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean updateLocalCheckpointInMap(String allocationId, long localCheckpoint,
|
if (current < localCheckpoint) {
|
||||||
ObjectLongMap<String> checkpointsMap, String name) {
|
inSyncLocalCheckpoints.indexReplace(indexOfKey, localCheckpoint);
|
||||||
assert Thread.holdsLock(this);
|
if (logger.isTraceEnabled()) {
|
||||||
int indexOfKey = checkpointsMap.indexOf(allocationId);
|
logger.trace("updated local checkpoint of [{}] to [{}] (was [{}])", allocationId, localCheckpoint, current);
|
||||||
if (indexOfKey < 0) {
|
}
|
||||||
return false;
|
} else {
|
||||||
}
|
logger.trace("skipping update of local checkpoint [{}], current checkpoint is higher " +
|
||||||
long current = checkpointsMap.indexGet(indexOfKey);
|
"(current [{}], incoming [{}], type [{}])",
|
||||||
// nocommit: this can change when we introduces rollback/resync
|
allocationId, current, localCheckpoint, allocationId);
|
||||||
if (current < localCheckpoint) {
|
|
||||||
checkpointsMap.indexReplace(indexOfKey, localCheckpoint);
|
|
||||||
if (logger.isTraceEnabled()) {
|
|
||||||
logger.trace("updated local checkpoint of [{}] to [{}] (type [{}])", allocationId, localCheckpoint,
|
|
||||||
name);
|
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
logger.trace("skipping update local checkpoint [{}], current check point is higher " +
|
logger.trace("[{}] isn't marked as in sync. ignoring local checkpoint of [{}].", allocationId, localCheckpoint);
|
||||||
"(current [{}], incoming [{}], type [{}])",
|
|
||||||
allocationId, current, localCheckpoint, allocationId);
|
|
||||||
}
|
}
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -138,21 +116,16 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
||||||
*/
|
*/
|
||||||
synchronized boolean updateCheckpointOnPrimary() {
|
synchronized boolean updateCheckpointOnPrimary() {
|
||||||
long minCheckpoint = Long.MAX_VALUE;
|
long minCheckpoint = Long.MAX_VALUE;
|
||||||
if (activeLocalCheckpoints.isEmpty() && inSyncLocalCheckpoints.isEmpty()) {
|
if (inSyncLocalCheckpoints.isEmpty()) {
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
for (ObjectLongCursor<String> cp : activeLocalCheckpoints) {
|
for (ObjectLongCursor<String> cp : inSyncLocalCheckpoints) {
|
||||||
if (cp.value == SequenceNumbersService.UNASSIGNED_SEQ_NO) {
|
if (cp.value == UNASSIGNED_SEQ_NO) {
|
||||||
logger.trace("unknown local checkpoint for active allocationId [{}], requesting a sync", cp.key);
|
logger.trace("unknown local checkpoint for active allocationId [{}], requesting a sync", cp.key);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
minCheckpoint = Math.min(cp.value, minCheckpoint);
|
minCheckpoint = Math.min(cp.value, minCheckpoint);
|
||||||
}
|
}
|
||||||
for (ObjectLongCursor<String> cp : inSyncLocalCheckpoints) {
|
|
||||||
assert cp.value != SequenceNumbersService.UNASSIGNED_SEQ_NO :
|
|
||||||
"in sync allocation ids can not have an unknown checkpoint (aId [" + cp.key + "])";
|
|
||||||
minCheckpoint = Math.min(cp.value, minCheckpoint);
|
|
||||||
}
|
|
||||||
if (minCheckpoint < globalCheckpoint) {
|
if (minCheckpoint < globalCheckpoint) {
|
||||||
// nocommit: if this happens - do you we fail the shard?
|
// nocommit: if this happens - do you we fail the shard?
|
||||||
throw new IllegalStateException(shardId + " new global checkpoint [" + minCheckpoint
|
throw new IllegalStateException(shardId + " new global checkpoint [" + minCheckpoint
|
||||||
|
@ -181,77 +154,53 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent {
|
||||||
this.globalCheckpoint = globalCheckpoint;
|
this.globalCheckpoint = globalCheckpoint;
|
||||||
logger.trace("global checkpoint updated from primary to [{}]", globalCheckpoint);
|
logger.trace("global checkpoint updated from primary to [{}]", globalCheckpoint);
|
||||||
} else {
|
} else {
|
||||||
// nocommit: fail the shard?
|
|
||||||
throw new IllegalArgumentException("global checkpoint from primary should never decrease. current [" +
|
throw new IllegalArgumentException("global checkpoint from primary should never decrease. current [" +
|
||||||
this.globalCheckpoint + "], got [" + globalCheckpoint + "]");
|
this.globalCheckpoint + "], got [" + globalCheckpoint + "]");
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Notifies the service of the current allocation ids in the cluster state. This method trims any shards that
|
* Notifies the service of the current allocation ids in the cluster state. This method trims any shards that
|
||||||
* have been removed and adds/promotes any active allocations to the {@link #activeLocalCheckpoints}.
|
* have been removed.
|
||||||
*
|
*
|
||||||
* @param activeAllocationIds the allocation ids of the currently active shard copies
|
* @param activeAllocationIds the allocation ids of the currently active shard copies
|
||||||
* @param initializingAllocationIds the allocation ids of the currently initializing shard copies
|
* @param initializingAllocationIds the allocation ids of the currently initializing shard copies
|
||||||
*/
|
*/
|
||||||
public synchronized void updateAllocationIdsFromMaster(Set<String> activeAllocationIds,
|
public synchronized void updateAllocationIdsFromMaster(Set<String> activeAllocationIds,
|
||||||
Set<String> initializingAllocationIds) {
|
Set<String> initializingAllocationIds) {
|
||||||
activeLocalCheckpoints.removeAll(key -> activeAllocationIds.contains(key) == false);
|
assignedAllocationIds.removeIf(
|
||||||
|
aId -> activeAllocationIds.contains(aId) == false && initializingAllocationIds.contains(aId) == false);
|
||||||
|
assignedAllocationIds.addAll(activeAllocationIds);
|
||||||
|
assignedAllocationIds.addAll(initializingAllocationIds);
|
||||||
for (String activeId : activeAllocationIds) {
|
for (String activeId : activeAllocationIds) {
|
||||||
if (activeLocalCheckpoints.containsKey(activeId) == false) {
|
if (inSyncLocalCheckpoints.containsKey(activeId) == false) {
|
||||||
long knownCheckpoint = trackingLocalCheckpoint.getOrDefault(activeId, SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
inSyncLocalCheckpoints.put(activeId, UNASSIGNED_SEQ_NO);
|
||||||
knownCheckpoint = inSyncLocalCheckpoints.getOrDefault(activeId, knownCheckpoint);
|
|
||||||
activeLocalCheckpoints.put(activeId, knownCheckpoint);
|
|
||||||
logger.trace("marking [{}] as active. known checkpoint [{}]", activeId, knownCheckpoint);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
inSyncLocalCheckpoints.removeAll(key -> initializingAllocationIds.contains(key) == false);
|
inSyncLocalCheckpoints.removeAll(key -> assignedAllocationIds.contains(key) == false);
|
||||||
trackingLocalCheckpoint.removeAll(key -> initializingAllocationIds.contains(key) == false);
|
|
||||||
// add initializing shards to tracking
|
|
||||||
for (String initID : initializingAllocationIds) {
|
|
||||||
if (inSyncLocalCheckpoints.containsKey(initID)) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
if (trackingLocalCheckpoint.containsKey(initID)) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
trackingLocalCheckpoint.put(initID, SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
|
||||||
logger.trace("added [{}] to the tracking map due to a CS update", initID);
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* marks the allocationId as "in sync" with the primary shard. This should be called at the end of recovery
|
* marks the allocationId as "in sync" with the primary shard. This should be called at the end of recovery
|
||||||
* where the primary knows all operation bellow the global checkpoint have been completed on this shard.
|
* where the primary knows all operation below the global checkpoint have been completed on this shard.
|
||||||
*
|
*
|
||||||
* @param allocationId allocationId of the recovering shard
|
* @param allocationId allocationId of the recovering shard
|
||||||
* @param localCheckpoint the local checkpoint of the shard in question
|
|
||||||
*/
|
*/
|
||||||
public synchronized void markAllocationIdAsInSync(String allocationId, long localCheckpoint) {
|
public synchronized void markAllocationIdAsInSync(String allocationId) {
|
||||||
if (trackingLocalCheckpoint.containsKey(allocationId) == false) {
|
if (assignedAllocationIds.contains(allocationId) == false) {
|
||||||
// master have change its mind and removed this allocation, ignore.
|
// master have change it's mind and removed this allocation, ignore.
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
long current = trackingLocalCheckpoint.remove(allocationId);
|
logger.trace("marked [{}] as in sync", allocationId);
|
||||||
localCheckpoint = Math.max(current, localCheckpoint);
|
inSyncLocalCheckpoints.put(allocationId, UNASSIGNED_SEQ_NO);
|
||||||
logger.trace("marked [{}] as in sync with a local checkpoint of [{}]", allocationId, localCheckpoint);
|
|
||||||
inSyncLocalCheckpoints.put(allocationId, localCheckpoint);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
// for testing
|
// for testing
|
||||||
synchronized long getLocalCheckpointForAllocation(String allocationId) {
|
synchronized long getLocalCheckpointForAllocation(String allocationId) {
|
||||||
if (activeLocalCheckpoints.containsKey(allocationId)) {
|
|
||||||
return activeLocalCheckpoints.get(allocationId);
|
|
||||||
}
|
|
||||||
if (inSyncLocalCheckpoints.containsKey(allocationId)) {
|
if (inSyncLocalCheckpoints.containsKey(allocationId)) {
|
||||||
return inSyncLocalCheckpoints.get(allocationId);
|
return inSyncLocalCheckpoints.get(allocationId);
|
||||||
}
|
}
|
||||||
if (trackingLocalCheckpoint.containsKey(allocationId)) {
|
return UNASSIGNED_SEQ_NO;
|
||||||
return trackingLocalCheckpoint.get(allocationId);
|
|
||||||
}
|
|
||||||
return SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -96,25 +96,25 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class PrimaryRequest extends ReplicationRequest<PrimaryRequest> {
|
public static final class PrimaryRequest extends ReplicationRequest<PrimaryRequest> {
|
||||||
|
|
||||||
private PrimaryRequest() {
|
private PrimaryRequest() {
|
||||||
super();
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
PrimaryRequest(ShardId shardId) {
|
public PrimaryRequest(ShardId shardId) {
|
||||||
super(shardId);
|
super(shardId);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class ReplicaRequest extends ReplicationRequest<GlobalCheckpointSyncAction.ReplicaRequest> {
|
public static final class ReplicaRequest extends ReplicationRequest<GlobalCheckpointSyncAction.ReplicaRequest> {
|
||||||
|
|
||||||
public long checkpoint;
|
private long checkpoint;
|
||||||
|
|
||||||
private ReplicaRequest() {
|
private ReplicaRequest() {
|
||||||
}
|
}
|
||||||
|
|
||||||
ReplicaRequest(PrimaryRequest primaryRequest, long checkpoint) {
|
public ReplicaRequest(PrimaryRequest primaryRequest, long checkpoint) {
|
||||||
super(primaryRequest.shardId());
|
super(primaryRequest.shardId());
|
||||||
this.checkpoint = checkpoint;
|
this.checkpoint = checkpoint;
|
||||||
}
|
}
|
||||||
|
@ -130,6 +130,9 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
|
||||||
super.writeTo(out);
|
super.writeTo(out);
|
||||||
out.writeZLong(checkpoint);
|
out.writeZLong(checkpoint);
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
|
public long getCheckpoint() {
|
||||||
|
return checkpoint;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,4 +79,13 @@ public class SeqNoStats implements ToXContent, Writeable {
|
||||||
builder.endObject();
|
builder.endObject();
|
||||||
return builder;
|
return builder;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "SeqNoStats{" +
|
||||||
|
"maxSeqNo=" + maxSeqNo +
|
||||||
|
", localCheckpoint=" + localCheckpoint +
|
||||||
|
", globalCheckpoint=" + globalCheckpoint +
|
||||||
|
'}';
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -114,13 +114,12 @@ public class SequenceNumbersService extends AbstractIndexShardComponent {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* marks the allocationId as "in sync" with the primary shard.
|
* marks the allocationId as "in sync" with the primary shard.
|
||||||
* see {@link GlobalCheckpointService#markAllocationIdAsInSync(String, long)} for details.
|
* see {@link GlobalCheckpointService#markAllocationIdAsInSync(String)} for details.
|
||||||
*
|
*
|
||||||
* @param allocationId allocationId of the recovering shard
|
* @param allocationId allocationId of the recovering shard
|
||||||
* @param localCheckpoint the local checkpoint of the shard in question
|
|
||||||
*/
|
*/
|
||||||
public void markAllocationIdAsInSync(String allocationId, long localCheckpoint) {
|
public void markAllocationIdAsInSync(String allocationId) {
|
||||||
globalCheckpointService.markAllocationIdAsInSync(allocationId, localCheckpoint);
|
globalCheckpointService.markAllocationIdAsInSync(allocationId);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getLocalCheckpoint() {
|
public long getLocalCheckpoint() {
|
||||||
|
|
|
@ -349,8 +349,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
primaryTerm = newTerm;
|
primaryTerm = newTerm;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -1366,14 +1364,14 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* marks the allocationId as "in sync" with the primary shard. see {@link GlobalCheckpointService#markAllocationIdAsInSync(String, long)} for details.
|
* marks the allocationId as "in sync" with the primary shard. see {@link GlobalCheckpointService#markAllocationIdAsInSync(String)}
|
||||||
|
* for details.
|
||||||
*
|
*
|
||||||
* @param allocationId allocationId of the recovering shard
|
* @param allocationId allocationId of the recovering shard
|
||||||
* @param localCheckpoint the local checkpoint of the shard in question
|
|
||||||
*/
|
*/
|
||||||
public void markAllocationIdAsInSync(String allocationId, long localCheckpoint) {
|
public void markAllocationIdAsInSync(String allocationId) {
|
||||||
verifyPrimary();
|
verifyPrimary();
|
||||||
getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint);
|
getEngine().seqNoService().markAllocationIdAsInSync(allocationId);
|
||||||
}
|
}
|
||||||
|
|
||||||
public long getLocalCheckpoint() {
|
public long getLocalCheckpoint() {
|
||||||
|
|
|
@ -31,6 +31,7 @@ import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
|
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource.Type;
|
import org.elasticsearch.cluster.routing.RecoverySource.Type;
|
||||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||||
|
@ -54,6 +55,7 @@ import org.elasticsearch.index.IndexService;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.IndexShardAlreadyExistsException;
|
import org.elasticsearch.index.IndexShardAlreadyExistsException;
|
||||||
import org.elasticsearch.index.NodeServicesProvider;
|
import org.elasticsearch.index.NodeServicesProvider;
|
||||||
|
import org.elasticsearch.index.seqno.GlobalCheckpointService;
|
||||||
import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
|
import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
|
||||||
import org.elasticsearch.index.shard.IndexEventListener;
|
import org.elasticsearch.index.shard.IndexEventListener;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
|
@ -85,6 +87,7 @@ import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateListener {
|
public class IndicesClusterStateService extends AbstractLifecycleComponent implements ClusterStateListener {
|
||||||
|
|
||||||
|
@ -120,7 +123,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
SearchService searchService, SyncedFlushService syncedFlushService,
|
SearchService searchService, SyncedFlushService syncedFlushService,
|
||||||
PeerRecoverySourceService peerRecoverySourceService, NodeServicesProvider nodeServicesProvider,
|
PeerRecoverySourceService peerRecoverySourceService, NodeServicesProvider nodeServicesProvider,
|
||||||
GlobalCheckpointSyncAction globalCheckpointSyncAction) {
|
GlobalCheckpointSyncAction globalCheckpointSyncAction) {
|
||||||
this(settings, (AllocatedIndices<? extends Shard, ? extends AllocatedIndex<? extends Shard>>) indicesService,
|
this(settings, indicesService,
|
||||||
clusterService, threadPool, recoveryTargetService, shardStateAction,
|
clusterService, threadPool, recoveryTargetService, shardStateAction,
|
||||||
nodeMappingRefreshAction, repositoriesService, restoreService, searchService, syncedFlushService, peerRecoverySourceService,
|
nodeMappingRefreshAction, repositoriesService, restoreService, searchService, syncedFlushService, peerRecoverySourceService,
|
||||||
nodeServicesProvider, globalCheckpointSyncAction::updateCheckpointForShard);
|
nodeServicesProvider, globalCheckpointSyncAction::updateCheckpointForShard);
|
||||||
|
@ -502,7 +505,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
assert shardRouting.initializing() : shardRouting + " should have been removed by failMissingShards";
|
assert shardRouting.initializing() : shardRouting + " should have been removed by failMissingShards";
|
||||||
createShard(nodes, routingTable, shardRouting);
|
createShard(nodes, routingTable, shardRouting);
|
||||||
} else {
|
} else {
|
||||||
updateShard(nodes, shardRouting, shard);
|
updateShard(nodes, shardRouting, shard, routingTable);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -534,7 +537,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard shard) {
|
private void updateShard(DiscoveryNodes nodes, ShardRouting shardRouting, Shard shard, RoutingTable routingTable) {
|
||||||
final ShardRouting currentRoutingEntry = shard.routingEntry();
|
final ShardRouting currentRoutingEntry = shard.routingEntry();
|
||||||
assert currentRoutingEntry.isSameAllocation(shardRouting) :
|
assert currentRoutingEntry.isSameAllocation(shardRouting) :
|
||||||
"local shard has a different allocation id but wasn't cleaning by removeShards. "
|
"local shard has a different allocation id but wasn't cleaning by removeShards. "
|
||||||
|
@ -542,6 +545,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
|
|
||||||
try {
|
try {
|
||||||
shard.updateRoutingEntry(shardRouting);
|
shard.updateRoutingEntry(shardRouting);
|
||||||
|
if (shardRouting.primary()) {
|
||||||
|
IndexShardRoutingTable indexShardRoutingTable = routingTable.shardRoutingTable(shardRouting.shardId());
|
||||||
|
Set<String> activeIds = indexShardRoutingTable.activeShards().stream().map(r -> r.allocationId().getId())
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
Set<String> initializingIds = indexShardRoutingTable.getAllInitializingShards().stream().map(r -> r.allocationId().getId())
|
||||||
|
.collect(Collectors.toSet());
|
||||||
|
shard.updateAllocationIdsFromMaster(activeIds, initializingIds);
|
||||||
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e);
|
failAndRemoveShard(shardRouting, true, "failed updating shard routing entry", e);
|
||||||
return;
|
return;
|
||||||
|
@ -720,6 +731,15 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple
|
||||||
* @throws IOException if shard state could not be persisted
|
* @throws IOException if shard state could not be persisted
|
||||||
*/
|
*/
|
||||||
void updateRoutingEntry(ShardRouting shardRouting) throws IOException;
|
void updateRoutingEntry(ShardRouting shardRouting) throws IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Notifies the service of the current allocation ids in the cluster state.
|
||||||
|
* See {@link GlobalCheckpointService#updateAllocationIdsFromMaster(Set, Set)} for details.
|
||||||
|
*
|
||||||
|
* @param activeAllocationIds the allocation ids of the currently active shard copies
|
||||||
|
* @param initializingAllocationIds the allocation ids of the currently initializing shard copies
|
||||||
|
*/
|
||||||
|
void updateAllocationIdsFromMaster(Set<String> activeAllocationIds, Set<String> initializingAllocationIds);
|
||||||
}
|
}
|
||||||
|
|
||||||
public interface AllocatedIndex<T extends Shard> extends Iterable<T>, IndexComponent {
|
public interface AllocatedIndex<T extends Shard> extends Iterable<T>, IndexComponent {
|
||||||
|
|
|
@ -113,7 +113,7 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
||||||
throw new DelayRecoveryException("source node has the state of the target shard to be [" + targetShardRouting.state() + "], expecting to be [initializing]");
|
throw new DelayRecoveryException("source node has the state of the target shard to be [" + targetShardRouting.state() + "], expecting to be [initializing]");
|
||||||
}
|
}
|
||||||
|
|
||||||
RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard);
|
RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, targetShardRouting.allocationId().getId(), shard);
|
||||||
logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(), request.targetNode());
|
logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(), request.targetNode());
|
||||||
try {
|
try {
|
||||||
return handler.recoverToTarget();
|
return handler.recoverToTarget();
|
||||||
|
@ -133,9 +133,9 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
||||||
private final class OngoingRecoveries {
|
private final class OngoingRecoveries {
|
||||||
private final Map<IndexShard, ShardRecoveryContext> ongoingRecoveries = new HashMap<>();
|
private final Map<IndexShard, ShardRecoveryContext> ongoingRecoveries = new HashMap<>();
|
||||||
|
|
||||||
synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, IndexShard shard) {
|
synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, String targetAllocationId, IndexShard shard) {
|
||||||
final ShardRecoveryContext shardContext = ongoingRecoveries.computeIfAbsent(shard, s -> new ShardRecoveryContext());
|
final ShardRecoveryContext shardContext = ongoingRecoveries.computeIfAbsent(shard, s -> new ShardRecoveryContext());
|
||||||
RecoverySourceHandler handler = shardContext.addNewRecovery(request, shard);
|
RecoverySourceHandler handler = shardContext.addNewRecovery(request, targetAllocationId, shard);
|
||||||
shard.recoveryStats().incCurrentAsSource();
|
shard.recoveryStats().incCurrentAsSource();
|
||||||
return handler;
|
return handler;
|
||||||
}
|
}
|
||||||
|
@ -181,20 +181,21 @@ public class PeerRecoverySourceService extends AbstractComponent implements Inde
|
||||||
* Adds recovery source handler if recoveries are not delayed from starting (see also {@link #delayNewRecoveries}.
|
* 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.
|
* Throws {@link DelayRecoveryException} if new recoveries are delayed from starting.
|
||||||
*/
|
*/
|
||||||
synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, IndexShard shard) {
|
synchronized RecoverySourceHandler addNewRecovery(StartRecoveryRequest request, String targetAllocationId, IndexShard shard) {
|
||||||
if (onNewRecoveryException != null) {
|
if (onNewRecoveryException != null) {
|
||||||
throw onNewRecoveryException;
|
throw onNewRecoveryException;
|
||||||
}
|
}
|
||||||
RecoverySourceHandler handler = createRecoverySourceHandler(request, shard);
|
RecoverySourceHandler handler = createRecoverySourceHandler(request, targetAllocationId, shard);
|
||||||
recoveryHandlers.add(handler);
|
recoveryHandlers.add(handler);
|
||||||
return handler;
|
return handler;
|
||||||
}
|
}
|
||||||
|
|
||||||
private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest request, IndexShard shard) {
|
private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest request, String targetAllocationId,
|
||||||
|
IndexShard shard) {
|
||||||
RecoverySourceHandler handler;
|
RecoverySourceHandler handler;
|
||||||
final RemoteRecoveryTargetHandler recoveryTarget =
|
final RemoteRecoveryTargetHandler recoveryTarget =
|
||||||
new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService, request.targetNode(),
|
new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), targetAllocationId, transportService,
|
||||||
recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime));
|
request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime));
|
||||||
Supplier<Long> currentClusterStateVersionSupplier = () -> clusterService.state().getVersion();
|
Supplier<Long> currentClusterStateVersionSupplier = () -> clusterService.state().getVersion();
|
||||||
if (shard.indexSettings().isOnSharedFilesystem()) {
|
if (shard.indexSettings().isOnSharedFilesystem()) {
|
||||||
handler = new SharedFSRecoverySourceHandler(shard, recoveryTarget, request, currentClusterStateVersionSupplier,
|
handler = new SharedFSRecoverySourceHandler(shard, recoveryTarget, request, currentClusterStateVersionSupplier,
|
||||||
|
|
|
@ -314,12 +314,11 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportChannel channel) throws Exception {
|
public void messageReceived(RecoveryFinalizeRecoveryRequest request, TransportChannel channel) throws Exception {
|
||||||
final RecoveryTargetHandler.FinalizeResponse response;
|
try (RecoveriesCollection.RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId()))
|
||||||
try (RecoveriesCollection.RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId()
|
{
|
||||||
)) {
|
recoveryRef.status().finalizeRecovery();
|
||||||
response = recoveryRef.status().finalizeRecovery();
|
|
||||||
}
|
}
|
||||||
channel.sendResponse(response);
|
channel.sendResponse(TransportResponse.Empty.INSTANCE);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -391,8 +391,8 @@ public class RecoverySourceHandler {
|
||||||
StopWatch stopWatch = new StopWatch().start();
|
StopWatch stopWatch = new StopWatch().start();
|
||||||
logger.trace("[{}][{}] finalizing recovery to {}", indexName, shardId, request.targetNode());
|
logger.trace("[{}][{}] finalizing recovery to {}", indexName, shardId, request.targetNode());
|
||||||
cancellableThreads.execute(() -> {
|
cancellableThreads.execute(() -> {
|
||||||
RecoveryTarget.FinalizeResponse response = recoveryTarget.finalizeRecovery();
|
recoveryTarget.finalizeRecovery();
|
||||||
shard.markAllocationIdAsInSync(response.getAllocationId(), response.getLocalCheckpoint());
|
shard.markAllocationIdAsInSync(recoveryTarget.getTargetAllocationId());
|
||||||
});
|
});
|
||||||
|
|
||||||
if (request.isPrimaryRelocation()) {
|
if (request.isPrimaryRelocation()) {
|
||||||
|
|
|
@ -333,10 +333,14 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FinalizeResponse finalizeRecovery() {
|
public void finalizeRecovery() {
|
||||||
final IndexShard indexShard = indexShard();
|
final IndexShard indexShard = indexShard();
|
||||||
indexShard.finalizeRecovery();
|
indexShard.finalizeRecovery();
|
||||||
return new FinalizeResponse(indexShard.routingEntry().allocationId().getId(), indexShard.getLocalCheckpoint());
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTargetAllocationId() {
|
||||||
|
return indexShard().routingEntry().allocationId().getId();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -19,12 +19,9 @@
|
||||||
package org.elasticsearch.indices.recovery;
|
package org.elasticsearch.indices.recovery;
|
||||||
|
|
||||||
import org.elasticsearch.common.bytes.BytesReference;
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetaData;
|
import org.elasticsearch.index.store.StoreFileMetaData;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.transport.TransportResponse;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -44,9 +41,9 @@ public interface RecoveryTargetHandler {
|
||||||
/**
|
/**
|
||||||
* The finalize request clears unreferenced translog files, refreshes the engine now that
|
* The finalize request clears unreferenced translog files, refreshes the engine now that
|
||||||
* new segments are available, and enables garbage collection of
|
* new segments are available, and enables garbage collection of
|
||||||
* tombstone files. The shard is also moved to the POST_RECOVERY phase during this time
|
* tombstone files.
|
||||||
**/
|
**/
|
||||||
FinalizeResponse finalizeRecovery();
|
void finalizeRecovery();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Blockingly waits for cluster state with at least clusterStateVersion to be available
|
* Blockingly waits for cluster state with at least clusterStateVersion to be available
|
||||||
|
@ -81,41 +78,8 @@ public interface RecoveryTargetHandler {
|
||||||
void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content,
|
void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content,
|
||||||
boolean lastChunk, int totalTranslogOps) throws IOException;
|
boolean lastChunk, int totalTranslogOps) throws IOException;
|
||||||
|
|
||||||
class FinalizeResponse extends TransportResponse {
|
/***
|
||||||
private long localCheckpoint;
|
* @return the allocation id of the target shard.
|
||||||
private String allocationId;
|
*/
|
||||||
|
String getTargetAllocationId();
|
||||||
public FinalizeResponse(String allocationId, long localCheckpoint) {
|
|
||||||
this.localCheckpoint = localCheckpoint;
|
|
||||||
this.allocationId = allocationId;
|
|
||||||
}
|
|
||||||
|
|
||||||
FinalizeResponse() {
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
public long getLocalCheckpoint() {
|
|
||||||
return localCheckpoint;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String getAllocationId() {
|
|
||||||
return allocationId;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void writeTo(StreamOutput out) throws IOException {
|
|
||||||
super.writeTo(out);
|
|
||||||
out.writeZLong(localCheckpoint);
|
|
||||||
out.writeString(allocationId);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void readFrom(StreamInput in) throws IOException {
|
|
||||||
super.readFrom(in);
|
|
||||||
localCheckpoint = in.readZLong();
|
|
||||||
allocationId = in.readString();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,7 +28,6 @@ import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetaData;
|
import org.elasticsearch.index.store.StoreFileMetaData;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.transport.EmptyTransportResponseHandler;
|
import org.elasticsearch.transport.EmptyTransportResponseHandler;
|
||||||
import org.elasticsearch.transport.FutureTransportResponseHandler;
|
|
||||||
import org.elasticsearch.transport.TransportRequestOptions;
|
import org.elasticsearch.transport.TransportRequestOptions;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
|
@ -51,9 +50,11 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
|
||||||
private final AtomicLong bytesSinceLastPause = new AtomicLong();
|
private final AtomicLong bytesSinceLastPause = new AtomicLong();
|
||||||
|
|
||||||
private final Consumer<Long> onSourceThrottle;
|
private final Consumer<Long> onSourceThrottle;
|
||||||
|
private String targetAllocationId;
|
||||||
|
|
||||||
public RemoteRecoveryTargetHandler(long recoveryId, ShardId shardId, TransportService transportService, DiscoveryNode targetNode,
|
public RemoteRecoveryTargetHandler(long recoveryId, ShardId shardId, String targetAllocationId, TransportService transportService,
|
||||||
RecoverySettings recoverySettings, Consumer<Long> onSourceThrottle) {
|
DiscoveryNode targetNode, RecoverySettings recoverySettings, Consumer<Long> onSourceThrottle) {
|
||||||
|
this.targetAllocationId = targetAllocationId;
|
||||||
this.transportService = transportService;
|
this.transportService = transportService;
|
||||||
|
|
||||||
|
|
||||||
|
@ -85,16 +86,11 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FinalizeResponse finalizeRecovery() {
|
public void finalizeRecovery() {
|
||||||
return transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.FINALIZE,
|
transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.FINALIZE,
|
||||||
new RecoveryFinalizeRecoveryRequest(recoveryId, shardId),
|
new RecoveryFinalizeRecoveryRequest(recoveryId, shardId),
|
||||||
TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionLongTimeout()).build(),
|
TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionLongTimeout()).build(),
|
||||||
new FutureTransportResponseHandler<FinalizeResponse>() {
|
EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||||
@Override
|
|
||||||
public FinalizeResponse newInstance() {
|
|
||||||
return new FinalizeResponse();
|
|
||||||
}
|
|
||||||
}).txGet();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -167,4 +163,9 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
|
||||||
*/
|
*/
|
||||||
throttleTimeInNanos), fileChunkRequestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
throttleTimeInNanos), fileChunkRequestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String getTargetAllocationId() {
|
||||||
|
return targetAllocationId;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -45,6 +45,7 @@ import org.elasticsearch.common.transport.LocalTransportAddress;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.mapper.Uid;
|
import org.elasticsearch.index.mapper.Uid;
|
||||||
|
import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardTestCase;
|
import org.elasticsearch.index.shard.IndexShardTestCase;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
@ -104,9 +105,10 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
boolean closed = false;
|
boolean closed = false;
|
||||||
|
|
||||||
ReplicationGroup(final IndexMetaData indexMetaData) throws IOException {
|
ReplicationGroup(final IndexMetaData indexMetaData) throws IOException {
|
||||||
primary = newShard(shardId, true, "s0", indexMetaData, null);
|
primary = newShard(shardId, true, "s0", indexMetaData, this::syncGlobalCheckpoint, null);
|
||||||
replicas = new ArrayList<>();
|
replicas = new ArrayList<>();
|
||||||
this.indexMetaData = indexMetaData;
|
this.indexMetaData = indexMetaData;
|
||||||
|
updateAllocationIDsOnPrimary();
|
||||||
for (int i = 0; i < indexMetaData.getNumberOfReplicas(); i++) {
|
for (int i = 0; i < indexMetaData.getNumberOfReplicas(); i++) {
|
||||||
addReplica();
|
addReplica();
|
||||||
}
|
}
|
||||||
|
@ -119,6 +121,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
final IndexResponse response = index(indexRequest);
|
final IndexResponse response = index(indexRequest);
|
||||||
assertEquals(DocWriteResponse.Result.CREATED, response.getResult());
|
assertEquals(DocWriteResponse.Result.CREATED, response.getResult());
|
||||||
}
|
}
|
||||||
|
primary.updateGlobalCheckpointOnPrimary();
|
||||||
return numOfDoc;
|
return numOfDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -128,6 +131,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
final IndexResponse response = index(indexRequest);
|
final IndexResponse response = index(indexRequest);
|
||||||
assertEquals(DocWriteResponse.Result.CREATED, response.getResult());
|
assertEquals(DocWriteResponse.Result.CREATED, response.getResult());
|
||||||
}
|
}
|
||||||
|
primary.updateGlobalCheckpointOnPrimary();
|
||||||
return numOfDoc;
|
return numOfDoc;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -138,18 +142,39 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void startAll() throws IOException {
|
public synchronized void startAll() throws IOException {
|
||||||
|
startReplicas(replicas.size());
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized int startReplicas(int numOfReplicasToStart) throws IOException {
|
||||||
|
if (primary.routingEntry().initializing()) {
|
||||||
|
startPrimary();
|
||||||
|
}
|
||||||
|
int started = 0;
|
||||||
|
for (IndexShard replicaShard : replicas) {
|
||||||
|
if (replicaShard.routingEntry().initializing()) {
|
||||||
|
recoverReplica(replicaShard);
|
||||||
|
started++;
|
||||||
|
if (started > numOfReplicasToStart) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return started;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void startPrimary() throws IOException {
|
||||||
final DiscoveryNode pNode = getDiscoveryNode(primary.routingEntry().currentNodeId());
|
final DiscoveryNode pNode = getDiscoveryNode(primary.routingEntry().currentNodeId());
|
||||||
primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null));
|
primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null));
|
||||||
primary.recoverFromStore();
|
primary.recoverFromStore();
|
||||||
primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry()));
|
primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry()));
|
||||||
for (IndexShard replicaShard : replicas) {
|
updateAllocationIDsOnPrimary();
|
||||||
recoverReplica(replicaShard);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized IndexShard addReplica() throws IOException {
|
public synchronized IndexShard addReplica() throws IOException {
|
||||||
final IndexShard replica = newShard(shardId, false, "s" + replicaId.incrementAndGet(), indexMetaData, null);
|
final IndexShard replica = newShard(shardId, false, "s" + replicaId.incrementAndGet(), indexMetaData,
|
||||||
|
() -> { throw new AssertionError("replicas can't sync global checkpoint"); }, null);
|
||||||
replicas.add(replica);
|
replicas.add(replica);
|
||||||
|
updateAllocationIDsOnPrimary();
|
||||||
return replica;
|
return replica;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -165,6 +190,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
public void recoverReplica(IndexShard replica, BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
|
public void recoverReplica(IndexShard replica, BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
|
||||||
boolean markAsRecovering) throws IOException {
|
boolean markAsRecovering) throws IOException {
|
||||||
ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering);
|
ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering);
|
||||||
|
updateAllocationIDsOnPrimary();
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized DiscoveryNode getPrimaryNode() {
|
public synchronized DiscoveryNode getPrimaryNode() {
|
||||||
|
@ -230,9 +256,33 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
public IndexShard getPrimary() {
|
public IndexShard getPrimary() {
|
||||||
return primary;
|
return primary;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void syncGlobalCheckpoint() {
|
||||||
|
PlainActionFuture<ReplicationResponse> listener = new PlainActionFuture<>();
|
||||||
|
try {
|
||||||
|
new GlobalCheckpointSync(listener, this).execute();
|
||||||
|
listener.get();
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new AssertionError(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateAllocationIDsOnPrimary() {
|
||||||
|
Set<String> active = new HashSet<>();
|
||||||
|
Set<String> initializing = new HashSet<>();
|
||||||
|
for (ShardRouting shard: shardRoutings()) {
|
||||||
|
if (shard.active()) {
|
||||||
|
active.add(shard.allocationId().getId());
|
||||||
|
} else {
|
||||||
|
initializing.add(shard.allocationId().getId());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
primary.updateAllocationIdsFromMaster(active, initializing);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract class ReplicationAction<Request extends ReplicationRequest<Request>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
|
abstract class ReplicationAction<Request extends ReplicationRequest<Request>,
|
||||||
|
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
|
||||||
Response extends ReplicationResponse> {
|
Response extends ReplicationResponse> {
|
||||||
private final Request request;
|
private final Request request;
|
||||||
private ActionListener<Response> listener;
|
private ActionListener<Response> listener;
|
||||||
|
@ -390,4 +440,24 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
TransportWriteActionTestHelper.performPostWriteActions(replica, request, index.getTranslogLocation(), logger);
|
TransportWriteActionTestHelper.performPostWriteActions(replica, request, index.getTranslogLocation(), logger);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class GlobalCheckpointSync extends
|
||||||
|
ReplicationAction<GlobalCheckpointSyncAction.PrimaryRequest, GlobalCheckpointSyncAction.ReplicaRequest, ReplicationResponse> {
|
||||||
|
|
||||||
|
public GlobalCheckpointSync(ActionListener<ReplicationResponse> listener, ReplicationGroup replicationGroup) {
|
||||||
|
super(new GlobalCheckpointSyncAction.PrimaryRequest(replicationGroup.getPrimary().shardId()), listener,
|
||||||
|
replicationGroup, "global_ckp");
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected PrimaryResult performOnPrimary(IndexShard primary, GlobalCheckpointSyncAction.PrimaryRequest request) throws Exception {
|
||||||
|
return new PrimaryResult(new GlobalCheckpointSyncAction.ReplicaRequest(request, primary.getGlobalCheckpoint()),
|
||||||
|
new ReplicationResponse());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void performOnReplica(GlobalCheckpointSyncAction.ReplicaRequest request, IndexShard replica) {
|
||||||
|
replica.updateGlobalCheckpointOnReplica(request.getCheckpoint());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,19 +21,29 @@ package org.elasticsearch.index.replication;
|
||||||
import org.elasticsearch.action.DocWriteResponse;
|
import org.elasticsearch.action.DocWriteResponse;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.index.IndexResponse;
|
import org.elasticsearch.action.index.IndexResponse;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.common.xcontent.ToXContent;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.InternalEngine;
|
import org.elasticsearch.index.engine.InternalEngine;
|
||||||
import org.elasticsearch.index.engine.InternalEngineTests;
|
import org.elasticsearch.index.engine.InternalEngineTests;
|
||||||
import org.elasticsearch.index.engine.SegmentsStats;
|
import org.elasticsearch.index.engine.SegmentsStats;
|
||||||
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
|
import org.elasticsearch.index.seqno.SequenceNumbersService;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardTests;
|
import org.elasticsearch.index.shard.IndexShardTests;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||||
|
import org.hamcrest.Matcher;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.anyOf;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase {
|
public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase {
|
||||||
|
|
||||||
public void testSimpleReplication() throws Exception {
|
public void testSimpleReplication() throws Exception {
|
||||||
|
@ -67,7 +77,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
||||||
try {
|
try {
|
||||||
latch.countDown();
|
latch.countDown();
|
||||||
latch.await();
|
latch.await();
|
||||||
shards.appendDocs(numDocs-1);
|
shards.appendDocs(numDocs - 1);
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new AssertionError(e);
|
throw new AssertionError(e);
|
||||||
}
|
}
|
||||||
|
@ -121,4 +131,38 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testCheckpointsAdvance() throws Exception {
|
||||||
|
try (ReplicationGroup shards = createGroup(randomInt(3))) {
|
||||||
|
shards.startPrimary();
|
||||||
|
int numDocs = 0;
|
||||||
|
int startedShards;
|
||||||
|
do {
|
||||||
|
numDocs += shards.indexDocs(randomInt(20));
|
||||||
|
startedShards = shards.startReplicas(randomIntBetween(1, 2));
|
||||||
|
} while (startedShards > 0);
|
||||||
|
|
||||||
|
if (numDocs == 0 || randomBoolean()) {
|
||||||
|
// in the case we have no indexing, we simulate the background global checkpoint sync
|
||||||
|
shards.getPrimary().updateGlobalCheckpointOnPrimary();
|
||||||
|
}
|
||||||
|
for (IndexShard shard : shards) {
|
||||||
|
final SeqNoStats shardStats = shard.seqNoStats();
|
||||||
|
final ShardRouting shardRouting = shard.routingEntry();
|
||||||
|
logger.debug("seq_no stats for {}: {}", shardRouting, XContentHelper.toString(shardStats,
|
||||||
|
new ToXContent.MapParams(Collections.singletonMap("pretty", "false"))));
|
||||||
|
assertThat(shardRouting + " local checkpoint mismatch", shardStats.getLocalCheckpoint(), equalTo(numDocs - 1L));
|
||||||
|
|
||||||
|
final Matcher<Long> globalCheckpointMatcher;
|
||||||
|
if (shardRouting.primary()) {
|
||||||
|
globalCheckpointMatcher = equalTo(numDocs - 1L);
|
||||||
|
} else {
|
||||||
|
// nocommit: removed once fixed
|
||||||
|
globalCheckpointMatcher = anyOf(equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO), equalTo(numDocs - 1L));
|
||||||
|
}
|
||||||
|
assertThat(shardRouting + " global checkpoint mismatch", shardStats.getGlobalCheckpoint(), globalCheckpointMatcher);
|
||||||
|
assertThat(shardRouting + " max seq no mismatch", shardStats.getMaxSeqNo(), equalTo(numDocs - 1L));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -109,14 +109,13 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FinalizeResponse finalizeRecovery() {
|
public void finalizeRecovery() {
|
||||||
if (hasBlocked() == false) {
|
if (hasBlocked() == false) {
|
||||||
// it maybe that not ops have been transferred, block now
|
// it maybe that not ops have been transferred, block now
|
||||||
blockIfNeeded(RecoveryState.Stage.TRANSLOG);
|
blockIfNeeded(RecoveryState.Stage.TRANSLOG);
|
||||||
}
|
}
|
||||||
blockIfNeeded(RecoveryState.Stage.FINALIZE);
|
blockIfNeeded(RecoveryState.Stage.FINALIZE);
|
||||||
return super.finalizeRecovery();
|
super.finalizeRecovery();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,78 +0,0 @@
|
||||||
/*
|
|
||||||
* 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.seqno;
|
|
||||||
|
|
||||||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
|
||||||
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
|
||||||
import org.elasticsearch.common.xcontent.ToXContent;
|
|
||||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
|
||||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
|
||||||
import org.hamcrest.Matcher;
|
|
||||||
|
|
||||||
import java.util.ArrayList;
|
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.anyOf;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
|
||||||
|
|
||||||
@TestLogging("index.shard:TRACE,index.seqno:TRACE")
|
|
||||||
public class CheckpointsIT extends ESIntegTestCase {
|
|
||||||
|
|
||||||
@AwaitsFix(bugUrl = "boaz working om this.")
|
|
||||||
public void testCheckpointsAdvance() throws Exception {
|
|
||||||
prepareCreate("test").setSettings(
|
|
||||||
"index.seq_no.checkpoint_sync_interval", "100ms", // update global point frequently
|
|
||||||
"index.number_of_shards", "1" // simplify things so we know how many ops goes to the shards
|
|
||||||
).get();
|
|
||||||
final List<IndexRequestBuilder> builders = new ArrayList<>();
|
|
||||||
final long numDocs = scaledRandomIntBetween(0, 100);
|
|
||||||
logger.info("--> will index [{}] docs", numDocs);
|
|
||||||
for (int i = 0; i < numDocs; i++) {
|
|
||||||
builders.add(client().prepareIndex("test", "type", "id_" + i).setSource("{}"));
|
|
||||||
}
|
|
||||||
indexRandom(randomBoolean(), false, builders);
|
|
||||||
|
|
||||||
assertBusy(() -> {
|
|
||||||
IndicesStatsResponse stats = client().admin().indices().prepareStats("test").clear().get();
|
|
||||||
for (ShardStats shardStats : stats.getShards()) {
|
|
||||||
logger.debug("seq_no stats for {}: {}", shardStats.getShardRouting(),
|
|
||||||
XContentHelper.toString(shardStats.getSeqNoStats(),
|
|
||||||
new ToXContent.MapParams(Collections.singletonMap("pretty", "false"))));
|
|
||||||
assertThat(shardStats.getShardRouting() + " local checkpoint mismatch",
|
|
||||||
shardStats.getSeqNoStats().getLocalCheckpoint(), equalTo(numDocs - 1));
|
|
||||||
|
|
||||||
final Matcher<Long> globalCheckpointMatcher;
|
|
||||||
if (shardStats.getShardRouting().primary()) {
|
|
||||||
globalCheckpointMatcher = equalTo(numDocs - 1);
|
|
||||||
} else {
|
|
||||||
// nocommit: removed once fixed
|
|
||||||
globalCheckpointMatcher = anyOf(equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO), equalTo(numDocs - 1));
|
|
||||||
}
|
|
||||||
assertThat(shardStats.getShardRouting() + " global checkpoint mismatch",
|
|
||||||
shardStats.getSeqNoStats().getGlobalCheckpoint(), globalCheckpointMatcher);
|
|
||||||
assertThat(shardStats.getShardRouting() + " max seq no mismatch",
|
|
||||||
shardStats.getSeqNoStats().getMaxSeqNo(), equalTo(numDocs - 1));
|
|
||||||
}
|
|
||||||
});
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -19,18 +19,26 @@
|
||||||
package org.elasticsearch.index.seqno;
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.util.set.Sets;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
import org.elasticsearch.test.IndexSettingsModule;
|
import org.elasticsearch.test.IndexSettingsModule;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.greaterThan;
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
public class GlobalCheckpointTests extends ESTestCase {
|
public class GlobalCheckpointTests extends ESTestCase {
|
||||||
|
|
||||||
|
@ -41,89 +49,74 @@ public class GlobalCheckpointTests extends ESTestCase {
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
super.setUp();
|
super.setUp();
|
||||||
checkpointService = new GlobalCheckpointService(new ShardId("test", "_na_", 0),
|
checkpointService = new GlobalCheckpointService(new ShardId("test", "_na_", 0),
|
||||||
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), SequenceNumbersService.UNASSIGNED_SEQ_NO);
|
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testEmptyShards() {
|
public void testEmptyShards() {
|
||||||
assertFalse("checkpoint shouldn't be updated when the are no active shards", checkpointService.updateCheckpointOnPrimary());
|
assertFalse("checkpoint shouldn't be updated when the are no active shards", checkpointService.updateCheckpointOnPrimary());
|
||||||
assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
}
|
||||||
|
|
||||||
|
private final AtomicInteger aIdGenerator = new AtomicInteger();
|
||||||
|
|
||||||
|
private Map<String, Long> randomAllocationsWithLocalCheckpoints(int min, int max) {
|
||||||
|
Map<String, Long> allocations = new HashMap<>();
|
||||||
|
for (int i = randomIntBetween(min, max); i > 0; i--) {
|
||||||
|
allocations.put("id_" + aIdGenerator.incrementAndGet(), (long) randomInt(1000));
|
||||||
|
}
|
||||||
|
return allocations;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testGlobalCheckpointUpdate() {
|
public void testGlobalCheckpointUpdate() {
|
||||||
Map<String, Long> allocations = new HashMap<>();
|
Map<String, Long> allocations = new HashMap<>();
|
||||||
Set<String> active = new HashSet<>();
|
Map<String, Long> activeWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||||
Set<String> insync = new HashSet<>();
|
Set<String> active = new HashSet<>(activeWithCheckpoints.keySet());
|
||||||
Set<String> tracking = new HashSet<>();
|
allocations.putAll(activeWithCheckpoints);
|
||||||
long maxLocalCheckpoint = Long.MAX_VALUE;
|
Map<String, Long> initializingWithCheckpoints = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||||
for (int i = randomIntBetween(3, 10); i > 0; i--) {
|
Set<String> initializing = new HashSet<>(initializingWithCheckpoints.keySet());
|
||||||
String id = "id_" + i + "_" + randomAsciiOfLength(5);
|
allocations.putAll(initializingWithCheckpoints);
|
||||||
long localCheckpoint = randomInt(200);
|
assertThat(allocations.size(), equalTo(active.size() + initializing.size()));
|
||||||
switch (randomInt(2)) {
|
|
||||||
case 0:
|
|
||||||
active.add(id);
|
|
||||||
maxLocalCheckpoint = Math.min(maxLocalCheckpoint, localCheckpoint);
|
|
||||||
break;
|
|
||||||
case 1:
|
|
||||||
insync.add(id);
|
|
||||||
maxLocalCheckpoint = Math.min(maxLocalCheckpoint, localCheckpoint);
|
|
||||||
break;
|
|
||||||
case 2:
|
|
||||||
tracking.add(id);
|
|
||||||
break;
|
|
||||||
default:
|
|
||||||
throw new IllegalStateException("you messed up your numbers, didn't you?");
|
|
||||||
}
|
|
||||||
allocations.put(id, localCheckpoint);
|
|
||||||
}
|
|
||||||
|
|
||||||
if (maxLocalCheckpoint == Long.MAX_VALUE) {
|
// note: allocations can never be empty in practice as we always have at least one primary shard active/in sync
|
||||||
// note: this state can not happen in practice as we always have at least one primary shard active/in sync
|
// it is however nice not to assume this on this level and check we do the right thing.
|
||||||
// it is however nice not to assume this on this level and check we do the right thing.
|
final long maxLocalCheckpoint = allocations.values().stream().min(Long::compare).orElse(UNASSIGNED_SEQ_NO);
|
||||||
maxLocalCheckpoint = SequenceNumbersService.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
|
||||||
|
|
||||||
assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
logger.info("--> using allocations");
|
logger.info("--> using allocations");
|
||||||
allocations.keySet().stream().forEach(aId -> {
|
allocations.keySet().forEach(aId -> {
|
||||||
final String type;
|
final String type;
|
||||||
if (active.contains(aId)) {
|
if (active.contains(aId)) {
|
||||||
type = "active";
|
type = "active";
|
||||||
} else if (insync.contains(aId)) {
|
} else if (initializing.contains(aId)) {
|
||||||
type = "insync";
|
type = "init";
|
||||||
} else if (tracking.contains(aId)) {
|
|
||||||
type = "tracked";
|
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalStateException(aId + " not found in any map");
|
throw new IllegalStateException(aId + " not found in any map");
|
||||||
}
|
}
|
||||||
logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type);
|
logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type);
|
||||||
});
|
});
|
||||||
|
|
||||||
Set<String> initializing = new HashSet<>(insync);
|
|
||||||
initializing.addAll(tracking);
|
|
||||||
|
|
||||||
checkpointService.updateAllocationIdsFromMaster(active, initializing);
|
checkpointService.updateAllocationIdsFromMaster(active, initializing);
|
||||||
allocations.keySet().stream().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId)));
|
initializing.forEach(aId -> checkpointService.markAllocationIdAsInSync(aId));
|
||||||
|
allocations.keySet().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||||
|
|
||||||
// make sure insync allocation count
|
|
||||||
insync.stream().forEach(aId -> checkpointService.markAllocationIdAsInSync(aId, randomBoolean() ? 0 : allocations.get(aId)));
|
|
||||||
|
|
||||||
assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
assertThat(checkpointService.updateCheckpointOnPrimary(), equalTo(maxLocalCheckpoint != SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
assertThat(checkpointService.updateCheckpointOnPrimary(), equalTo(maxLocalCheckpoint != UNASSIGNED_SEQ_NO));
|
||||||
assertThat(checkpointService.getCheckpoint(), equalTo(maxLocalCheckpoint));
|
assertThat(checkpointService.getCheckpoint(), equalTo(maxLocalCheckpoint));
|
||||||
|
|
||||||
// increment checkpoints
|
// increment checkpoints
|
||||||
active.stream().forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4)));
|
||||||
insync.stream().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().stream().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId)));
|
allocations.keySet().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId)));
|
||||||
|
|
||||||
// now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested.
|
// now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested.
|
||||||
final String extraId = "extra_" + randomAsciiOfLength(5);
|
final String extraId = "extra_" + randomAsciiOfLength(5);
|
||||||
|
|
||||||
// first check that adding it without the master blessing doesn't change anything.
|
// first check that adding it without the master blessing doesn't change anything.
|
||||||
checkpointService.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4));
|
checkpointService.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4));
|
||||||
assertThat(checkpointService.getLocalCheckpointForAllocation(extraId), equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO));
|
assertThat(checkpointService.getLocalCheckpointForAllocation(extraId), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
Set<String> newActive = new HashSet<>(active);
|
Set<String> newActive = new HashSet<>(active);
|
||||||
newActive.add(extraId);
|
newActive.add(extraId);
|
||||||
|
@ -140,4 +133,112 @@ public class GlobalCheckpointTests extends ESTestCase {
|
||||||
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
assertThat(checkpointService.getCheckpoint(), greaterThan(maxLocalCheckpoint));
|
assertThat(checkpointService.getCheckpoint(), greaterThan(maxLocalCheckpoint));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMissingActiveIdsPreventAdvance() {
|
||||||
|
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||||
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||||
|
final Map<String, Long> assigned = new HashMap<>();
|
||||||
|
assigned.putAll(active);
|
||||||
|
assigned.putAll(initializing);
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(
|
||||||
|
new HashSet<>(randomSubsetOf(randomInt(active.size() - 1), active.keySet())),
|
||||||
|
initializing.keySet());
|
||||||
|
randomSubsetOf(initializing.keySet()).forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
assigned.forEach(checkpointService::updateLocalCheckpoint);
|
||||||
|
|
||||||
|
// now mark all active shards
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
|
|
||||||
|
// global checkpoint can't be advanced, but we need a sync
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
|
// update again
|
||||||
|
assigned.forEach(checkpointService::updateLocalCheckpoint);
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testMissingInSyncIdsPreventAdvance() {
|
||||||
|
final Map<String, Long> active = randomAllocationsWithLocalCheckpoints(0, 5);
|
||||||
|
final Map<String, Long> initializing = randomAllocationsWithLocalCheckpoints(1, 5);
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
|
initializing.keySet().forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
randomSubsetOf(randomInt(initializing.size() - 1),
|
||||||
|
initializing.keySet()).forEach(aId -> checkpointService.updateLocalCheckpoint(aId, initializing.get(aId)));
|
||||||
|
|
||||||
|
active.forEach(checkpointService::updateLocalCheckpoint);
|
||||||
|
|
||||||
|
// global checkpoint can't be advanced, but we need a sync
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
|
||||||
|
// update again
|
||||||
|
initializing.forEach(checkpointService::updateLocalCheckpoint);
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), 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);
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet());
|
||||||
|
initializing.keySet().forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
nonApproved.keySet().forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
|
||||||
|
List<Map<String, Long>> allocations = Arrays.asList(active, initializing, nonApproved);
|
||||||
|
Collections.shuffle(allocations, random());
|
||||||
|
allocations.forEach(a -> a.forEach(checkpointService::updateLocalCheckpoint));
|
||||||
|
|
||||||
|
// global checkpoint can be advanced, but we need a sync
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO)));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testInSyncIdsAreRemovedIfNotValidatedByMaster() {
|
||||||
|
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<>();
|
||||||
|
allocations.putAll(activeToStay);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
allocations.putAll(activeToBeRemoved);
|
||||||
|
}
|
||||||
|
allocations.putAll(initializingToStay);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
allocations.putAll(initializingToBeRemoved);
|
||||||
|
}
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(active, initializing);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
initializingToStay.keySet().forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
} else {
|
||||||
|
initializing.forEach(checkpointService::markAllocationIdAsInSync);
|
||||||
|
}
|
||||||
|
if (randomBoolean()) {
|
||||||
|
allocations.forEach(checkpointService::updateLocalCheckpoint);
|
||||||
|
}
|
||||||
|
|
||||||
|
// global checkpoint may be advanced, but we need a sync in any case
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
|
||||||
|
// now remove shards
|
||||||
|
if (randomBoolean()) {
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet());
|
||||||
|
allocations.forEach((aid, ckp) -> checkpointService.updateLocalCheckpoint(aid, ckp + 10L));
|
||||||
|
} else {
|
||||||
|
allocations.forEach((aid, ckp) -> checkpointService.updateLocalCheckpoint(aid, ckp + 10L));
|
||||||
|
checkpointService.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet());
|
||||||
|
}
|
||||||
|
|
||||||
|
final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream())
|
||||||
|
.min(Long::compare).get() + 10; // we added 10 to make sure it's advanced in the second time
|
||||||
|
|
||||||
|
// global checkpoint is advanced and we need a sync
|
||||||
|
assertTrue(checkpointService.updateCheckpointOnPrimary());
|
||||||
|
assertThat(checkpointService.getCheckpoint(), equalTo(checkpoint));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1043,7 +1043,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
};
|
};
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
||||||
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper);
|
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, () -> {});
|
||||||
|
|
||||||
recoveryShardFromStore(newShard);
|
recoveryShardFromStore(newShard);
|
||||||
|
|
||||||
|
@ -1183,7 +1183,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
|
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
IndexShard newShard = newShard(ShardRoutingHelper.reinitPrimary(shard.routingEntry()),
|
||||||
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper);
|
shard.shardPath(), shard.indexSettings().getIndexMetaData(), wrapper, () -> {});
|
||||||
|
|
||||||
recoveryShardFromStore(newShard);
|
recoveryShardFromStore(newShard);
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.indices.cluster;
|
||||||
|
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
|
@ -50,8 +51,10 @@ import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
import java.util.concurrent.ConcurrentMap;
|
import java.util.concurrent.ConcurrentMap;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static java.util.Collections.unmodifiableMap;
|
import static java.util.Collections.unmodifiableMap;
|
||||||
|
@ -82,8 +85,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
* @param state cluster state used for matching
|
* @param state cluster state used for matching
|
||||||
*/
|
*/
|
||||||
public void assertClusterStateMatchesNodeState(ClusterState state, IndicesClusterStateService indicesClusterStateService) {
|
public void assertClusterStateMatchesNodeState(ClusterState state, IndicesClusterStateService indicesClusterStateService) {
|
||||||
AllocatedIndices<? extends Shard, ? extends AllocatedIndex<? extends Shard>> indicesService =
|
MockIndicesService indicesService = (MockIndicesService) indicesClusterStateService.indicesService;
|
||||||
indicesClusterStateService.indicesService;
|
|
||||||
ConcurrentMap<ShardId, ShardRouting> failedShardsCache = indicesClusterStateService.failedShardsCache;
|
ConcurrentMap<ShardId, ShardRouting> failedShardsCache = indicesClusterStateService.failedShardsCache;
|
||||||
RoutingNode localRoutingNode = state.getRoutingNodes().node(state.getNodes().getLocalNodeId());
|
RoutingNode localRoutingNode = state.getRoutingNodes().node(state.getNodes().getLocalNodeId());
|
||||||
if (localRoutingNode != null) {
|
if (localRoutingNode != null) {
|
||||||
|
@ -95,7 +97,7 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
Index index = shardRouting.index();
|
Index index = shardRouting.index();
|
||||||
IndexMetaData indexMetaData = state.metaData().getIndexSafe(index);
|
IndexMetaData indexMetaData = state.metaData().getIndexSafe(index);
|
||||||
|
|
||||||
Shard shard = indicesService.getShardOrNull(shardRouting.shardId());
|
MockIndexShard shard = indicesService.getShardOrNull(shardRouting.shardId());
|
||||||
ShardRouting failedShard = failedShardsCache.get(shardRouting.shardId());
|
ShardRouting failedShard = failedShardsCache.get(shardRouting.shardId());
|
||||||
if (enableRandomFailures) {
|
if (enableRandomFailures) {
|
||||||
if (shard == null && failedShard == null) {
|
if (shard == null && failedShard == null) {
|
||||||
|
@ -122,6 +124,17 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
// shard has latest shard routing
|
// shard has latest shard routing
|
||||||
assertThat(shard.routingEntry(), equalTo(shardRouting));
|
assertThat(shard.routingEntry(), equalTo(shardRouting));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (shard.routingEntry().primary() && shard.routingEntry().active()) {
|
||||||
|
IndexShardRoutingTable shardRoutingTable = state.routingTable().shardRoutingTable(shard.shardId());
|
||||||
|
Set<String> activeIds = shardRoutingTable.activeShards().stream()
|
||||||
|
.map(r -> r.allocationId().getId()).collect(Collectors.toSet());
|
||||||
|
Set<String> initializingIds = shardRoutingTable.getAllInitializingShards().stream()
|
||||||
|
.map(r -> r.allocationId().getId()).collect(Collectors.toSet());
|
||||||
|
assertThat(shard.routingEntry() + " isn't updated with active aIDs", shard.activeAllocationIds, equalTo(activeIds));
|
||||||
|
assertThat(shard.routingEntry() + " isn't updated with init aIDs", shard.initializingAllocationIds,
|
||||||
|
equalTo(initializingIds));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -305,6 +318,8 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
protected class MockIndexShard implements IndicesClusterStateService.Shard {
|
protected class MockIndexShard implements IndicesClusterStateService.Shard {
|
||||||
private volatile ShardRouting shardRouting;
|
private volatile ShardRouting shardRouting;
|
||||||
private volatile RecoveryState recoveryState;
|
private volatile RecoveryState recoveryState;
|
||||||
|
private volatile Set<String> activeAllocationIds;
|
||||||
|
private volatile Set<String> initializingAllocationIds;
|
||||||
|
|
||||||
public MockIndexShard(ShardRouting shardRouting) {
|
public MockIndexShard(ShardRouting shardRouting) {
|
||||||
this.shardRouting = shardRouting;
|
this.shardRouting = shardRouting;
|
||||||
|
@ -337,5 +352,11 @@ public abstract class AbstractIndicesClusterStateServiceTestCase extends ESTestC
|
||||||
assert this.shardRouting.isSameAllocation(shardRouting);
|
assert this.shardRouting.isSameAllocation(shardRouting);
|
||||||
this.shardRouting = shardRouting;
|
this.shardRouting = shardRouting;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void updateAllocationIdsFromMaster(Set<String> activeAllocationIds, Set<String> initializingAllocationIds) {
|
||||||
|
this.activeAllocationIds = activeAllocationIds;
|
||||||
|
this.initializingAllocationIds = initializingAllocationIds;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -306,7 +306,6 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}).when(shard).relocated(any(String.class));
|
}).when(shard).relocated(any(String.class));
|
||||||
|
|
||||||
RecoveryTargetHandler targetHandler = mock(RecoveryTargetHandler.class);
|
RecoveryTargetHandler targetHandler = mock(RecoveryTargetHandler.class);
|
||||||
when(targetHandler.finalizeRecovery()).thenReturn(new RecoveryTargetHandler.FinalizeResponse("_mock_", 1));
|
|
||||||
|
|
||||||
final Supplier<Long> currentClusterStateVersionSupplier = () -> {
|
final Supplier<Long> currentClusterStateVersionSupplier = () -> {
|
||||||
assertFalse(ensureClusterStateVersionCalled.get());
|
assertFalse(ensureClusterStateVersionCalled.get());
|
||||||
|
|
|
@ -23,7 +23,12 @@ import com.carrotsearch.hppc.IntHashSet;
|
||||||
import com.carrotsearch.hppc.procedures.IntProcedure;
|
import com.carrotsearch.hppc.procedures.IntProcedure;
|
||||||
import org.apache.lucene.index.IndexFileNames;
|
import org.apache.lucene.index.IndexFileNames;
|
||||||
import org.apache.lucene.util.English;
|
import org.apache.lucene.util.English;
|
||||||
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.IndexShardStats;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.IndexStats;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||||
|
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||||
import org.elasticsearch.action.search.SearchResponse;
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
|
@ -40,12 +45,14 @@ import org.elasticsearch.common.Priority;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.env.NodeEnvironment;
|
import org.elasticsearch.env.NodeEnvironment;
|
||||||
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
|
import org.elasticsearch.index.seqno.SequenceNumbersService;
|
||||||
import org.elasticsearch.index.shard.IndexEventListener;
|
import org.elasticsearch.index.shard.IndexEventListener;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardState;
|
import org.elasticsearch.index.shard.IndexShardState;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
|
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||||
|
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.search.SearchHit;
|
import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.search.SearchHits;
|
import org.elasticsearch.search.SearchHits;
|
||||||
|
@ -73,11 +80,14 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Semaphore;
|
import java.util.concurrent.Semaphore;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL;
|
||||||
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||||
|
@ -91,6 +101,8 @@ import static org.hamcrest.Matchers.startsWith;
|
||||||
*/
|
*/
|
||||||
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
|
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
|
||||||
@TestLogging("_root:DEBUG,org.elasticsearch.indices.recovery:TRACE,org.elasticsearch.index.shard.service:TRACE")
|
@TestLogging("_root:DEBUG,org.elasticsearch.indices.recovery:TRACE,org.elasticsearch.index.shard.service:TRACE")
|
||||||
|
@LuceneTestCase.AwaitsFix(bugUrl = "primary relocation needs to transfer the global check point. otherwise the new primary sends a " +
|
||||||
|
"an unknown global checkpoint during sync, causing assertions to trigger")
|
||||||
public class RelocationIT extends ESIntegTestCase {
|
public class RelocationIT extends ESIntegTestCase {
|
||||||
private final TimeValue ACCEPTABLE_RELOCATION_TIME = new TimeValue(5, TimeUnit.MINUTES);
|
private final TimeValue ACCEPTABLE_RELOCATION_TIME = new TimeValue(5, TimeUnit.MINUTES);
|
||||||
|
|
||||||
|
@ -99,16 +111,53 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
return Arrays.asList(MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class);
|
return Arrays.asList(MockTransportService.TestPlugin.class, MockIndexEventListener.TestPlugin.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Settings indexSettings() {
|
||||||
|
return Settings.builder().put(super.indexSettings())
|
||||||
|
.put(INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL.getKey(), "200ms").build();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void beforeIndexDeletion() throws Exception {
|
||||||
|
super.beforeIndexDeletion();
|
||||||
|
assertBusy(() -> {
|
||||||
|
IndicesStatsResponse stats = client().admin().indices().prepareStats().clear().get();
|
||||||
|
for (IndexStats indexStats : stats.getIndices().values()) {
|
||||||
|
for (IndexShardStats indexShardStats : indexStats.getIndexShards().values()) {
|
||||||
|
Optional<ShardStats> maybePrimary = Stream.of(indexShardStats.getShards())
|
||||||
|
.filter(s -> s.getShardRouting().active() && s.getShardRouting().primary())
|
||||||
|
.findFirst();
|
||||||
|
if (maybePrimary.isPresent() == false) {
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
ShardStats primary = maybePrimary.get();
|
||||||
|
final SeqNoStats primarySeqNoStats = primary.getSeqNoStats();
|
||||||
|
assertThat(primary.getShardRouting() + " should have set the global checkpoint",
|
||||||
|
primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO)));
|
||||||
|
for (ShardStats shardStats : indexShardStats) {
|
||||||
|
final SeqNoStats seqNoStats = shardStats.getSeqNoStats();
|
||||||
|
assertThat(shardStats.getShardRouting() + " local checkpoint mismatch",
|
||||||
|
seqNoStats.getLocalCheckpoint(), equalTo(primarySeqNoStats.getLocalCheckpoint()));
|
||||||
|
|
||||||
|
assertThat(shardStats.getShardRouting() + " global checkpoint mismatch",
|
||||||
|
seqNoStats.getGlobalCheckpoint(), equalTo(primarySeqNoStats.getGlobalCheckpoint()));
|
||||||
|
assertThat(shardStats.getShardRouting() + " max seq no mismatch",
|
||||||
|
seqNoStats.getMaxSeqNo(), equalTo(primarySeqNoStats.getMaxSeqNo()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
|
||||||
public void testSimpleRelocationNoIndexing() {
|
public void testSimpleRelocationNoIndexing() {
|
||||||
logger.info("--> starting [node1] ...");
|
logger.info("--> starting [node1] ...");
|
||||||
final String node_1 = internalCluster().startNode();
|
final String node_1 = internalCluster().startNode();
|
||||||
|
|
||||||
logger.info("--> creating test index ...");
|
logger.info("--> creating test index ...");
|
||||||
client().admin().indices().prepareCreate("test")
|
prepareCreate("test", Settings.builder()
|
||||||
.setSettings(Settings.builder()
|
.put("index.number_of_shards", 1)
|
||||||
.put("index.number_of_shards", 1)
|
.put("index.number_of_replicas", 0)
|
||||||
.put("index.number_of_replicas", 0))
|
).get();
|
||||||
.execute().actionGet();
|
|
||||||
|
|
||||||
logger.info("--> index 10 docs");
|
logger.info("--> index 10 docs");
|
||||||
for (int i = 0; i < 10; i++) {
|
for (int i = 0; i < 10; i++) {
|
||||||
|
@ -158,10 +207,10 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
nodes[0] = internalCluster().startNode();
|
nodes[0] = internalCluster().startNode();
|
||||||
|
|
||||||
logger.info("--> creating test index ...");
|
logger.info("--> creating test index ...");
|
||||||
client().admin().indices().prepareCreate("test")
|
prepareCreate("test", Settings.builder()
|
||||||
.setSettings(Settings.builder()
|
.put("index.number_of_shards", 1)
|
||||||
.put("index.number_of_shards", 1)
|
.put("index.number_of_replicas", numberOfReplicas)
|
||||||
.put("index.number_of_replicas", numberOfReplicas)).execute().actionGet();
|
).get();
|
||||||
|
|
||||||
|
|
||||||
for (int i = 1; i < numberOfNodes; i++) {
|
for (int i = 1; i < numberOfNodes; i++) {
|
||||||
|
@ -260,12 +309,11 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
nodes[0] = internalCluster().startNode();
|
nodes[0] = internalCluster().startNode();
|
||||||
|
|
||||||
logger.info("--> creating test index ...");
|
logger.info("--> creating test index ...");
|
||||||
client().admin().indices().prepareCreate("test")
|
prepareCreate("test", Settings.builder()
|
||||||
.setSettings(Settings.builder()
|
.put("index.number_of_shards", 1)
|
||||||
.put("index.number_of_shards", 1)
|
.put("index.number_of_replicas", numberOfReplicas)
|
||||||
.put("index.number_of_replicas", numberOfReplicas)
|
.put("index.refresh_interval", -1) // we want to control refreshes c
|
||||||
.put("index.refresh_interval", -1) // we want to control refreshes c
|
).get();
|
||||||
).execute().actionGet();
|
|
||||||
|
|
||||||
for (int i = 1; i < numberOfNodes; i++) {
|
for (int i = 1; i < numberOfNodes; i++) {
|
||||||
logger.info("--> starting [node_{}] ...", i);
|
logger.info("--> starting [node_{}] ...", i);
|
||||||
|
@ -349,8 +397,9 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
|
|
||||||
final String p_node = internalCluster().startNode();
|
final String p_node = internalCluster().startNode();
|
||||||
|
|
||||||
client().admin().indices().prepareCreate(indexName)
|
prepareCreate(indexName, Settings.builder()
|
||||||
.setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)).get();
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1, IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||||
|
).get();
|
||||||
|
|
||||||
internalCluster().startNodesAsync(2).get();
|
internalCluster().startNodesAsync(2).get();
|
||||||
|
|
||||||
|
@ -383,17 +432,14 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
.setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")));
|
.setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")));
|
||||||
|
|
||||||
logger.info("--> wait for all replica shards to be removed, on all nodes");
|
logger.info("--> wait for all replica shards to be removed, on all nodes");
|
||||||
assertBusy(new Runnable() {
|
assertBusy(() -> {
|
||||||
@Override
|
for (String node : internalCluster().getNodeNames()) {
|
||||||
public void run() {
|
if (node.equals(p_node)) {
|
||||||
for (String node : internalCluster().getNodeNames()) {
|
continue;
|
||||||
if (node.equals(p_node)) {
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
ClusterState state = client(node).admin().cluster().prepareState().setLocal(true).get().getState();
|
|
||||||
assertThat(node + " indicates assigned replicas",
|
|
||||||
state.getRoutingTable().index(indexName).shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1));
|
|
||||||
}
|
}
|
||||||
|
ClusterState state = client(node).admin().cluster().prepareState().setLocal(true).get().getState();
|
||||||
|
assertThat(node + " indicates assigned replicas",
|
||||||
|
state.getRoutingTable().index(indexName).shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
@ -402,20 +448,17 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
NodeEnvironment nodeEnvironment = internalCluster().getInstance(NodeEnvironment.class, node);
|
NodeEnvironment nodeEnvironment = internalCluster().getInstance(NodeEnvironment.class, node);
|
||||||
for (final Path shardLoc : nodeEnvironment.availableShardPaths(new ShardId(indexName, "_na_", 0))) {
|
for (final Path shardLoc : nodeEnvironment.availableShardPaths(new ShardId(indexName, "_na_", 0))) {
|
||||||
if (Files.exists(shardLoc)) {
|
if (Files.exists(shardLoc)) {
|
||||||
assertBusy(new Runnable() {
|
assertBusy(() -> {
|
||||||
@Override
|
try {
|
||||||
public void run() {
|
Files.walkFileTree(shardLoc, new SimpleFileVisitor<Path>() {
|
||||||
try {
|
@Override
|
||||||
Files.walkFileTree(shardLoc, new SimpleFileVisitor<Path>() {
|
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
|
||||||
@Override
|
assertThat("found a temporary recovery file: " + file, file.getFileName().toString(), not(startsWith("recovery.")));
|
||||||
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
|
return FileVisitResult.CONTINUE;
|
||||||
assertThat("found a temporary recovery file: " + file, file.getFileName().toString(), not(startsWith("recovery.")));
|
}
|
||||||
return FileVisitResult.CONTINUE;
|
});
|
||||||
}
|
} catch (IOException e) {
|
||||||
});
|
throw new AssertionError("failed to walk file tree starting at [" + shardLoc + "]", e);
|
||||||
} catch (IOException e) {
|
|
||||||
throw new AssertionError("failed to walk file tree starting at [" + shardLoc + "]", e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -435,7 +478,7 @@ public class RelocationIT extends ESIntegTestCase {
|
||||||
logger.info("red nodes: {}", redFuture.get());
|
logger.info("red nodes: {}", redFuture.get());
|
||||||
ensureStableCluster(halfNodes * 2);
|
ensureStableCluster(halfNodes * 2);
|
||||||
|
|
||||||
assertAcked(prepareCreate("test").setSettings(Settings.builder()
|
assertAcked(prepareCreate("test", Settings.builder()
|
||||||
.put("index.routing.allocation.exclude.color", "blue")
|
.put("index.routing.allocation.exclude.color", "blue")
|
||||||
.put(indexSettings())
|
.put(indexSettings())
|
||||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(halfNodes - 1))
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(halfNodes - 1))
|
||||||
|
|
|
@ -194,9 +194,26 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
||||||
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
||||||
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
||||||
return newShard(shardRouting, indexMetaData, searcherWrapper);
|
return newShard(shardRouting, indexMetaData, searcherWrapper, () -> {});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* creates a new initializing shard. The shard will will be put in its proper path under the
|
||||||
|
* supplied node id.
|
||||||
|
*
|
||||||
|
* @param shardId the shard id to use
|
||||||
|
* @param primary indicates whether to a primary shard (ready to recover from an empty store) or a replica
|
||||||
|
* (ready to recover from another shard)
|
||||||
|
*/
|
||||||
|
protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, IndexMetaData indexMetaData,
|
||||||
|
Runnable globalCheckpointSyncer,
|
||||||
|
@Nullable IndexSearcherWrapper searcherWrapper) throws IOException {
|
||||||
|
ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
|
||||||
|
primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
|
||||||
|
return newShard(shardRouting, indexMetaData, searcherWrapper, globalCheckpointSyncer);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* creates a new initializing shard. The shard will will be put in its proper path under the
|
* creates a new initializing shard. The shard will will be put in its proper path under the
|
||||||
* current node id the shard is assigned to.
|
* current node id the shard is assigned to.
|
||||||
|
@ -207,26 +224,28 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners)
|
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, IndexingOperationListener... listeners)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
return newShard(routing, indexMetaData, null, listeners);
|
return newShard(routing, indexMetaData, null, () -> {}, listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* creates a new initializing shard. The shard will will be put in its proper path under the
|
* creates a new initializing shard. The shard will will be put in its proper path under the
|
||||||
* current node id the shard is assigned to.
|
* current node id the shard is assigned to.
|
||||||
*
|
*
|
||||||
* @param routing shard routing to use
|
* @param routing shard routing to use
|
||||||
* @param indexMetaData indexMetaData for the shard, including any mapping
|
* @param indexMetaData indexMetaData for the shard, including any mapping
|
||||||
* @param indexSearcherWrapper an optional wrapper to be used during searchers
|
* @param indexSearcherWrapper an optional wrapper to be used during searchers
|
||||||
* @param listeners an optional set of listeners to add to the shard
|
* @param globalCheckpointSyncer an runnable to run when the global check point needs syncing
|
||||||
|
* @param listeners an optional set of listeners to add to the shard
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData,
|
protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData,
|
||||||
@Nullable IndexSearcherWrapper indexSearcherWrapper, IndexingOperationListener... listeners)
|
@Nullable IndexSearcherWrapper indexSearcherWrapper, Runnable globalCheckpointSyncer,
|
||||||
|
IndexingOperationListener... listeners)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// add node id as name to settings for popper logging
|
// add node id as name to settings for popper logging
|
||||||
final ShardId shardId = routing.shardId();
|
final ShardId shardId = routing.shardId();
|
||||||
final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir());
|
final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir());
|
||||||
ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId);
|
ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId);
|
||||||
return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, listeners);
|
return newShard(routing, shardPath, indexMetaData, indexSearcherWrapper, globalCheckpointSyncer, listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -240,6 +259,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetaData indexMetaData,
|
protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMetaData indexMetaData,
|
||||||
@Nullable IndexSearcherWrapper indexSearcherWrapper,
|
@Nullable IndexSearcherWrapper indexSearcherWrapper,
|
||||||
|
Runnable globalCheckpointSyncer,
|
||||||
IndexingOperationListener... listeners) throws IOException {
|
IndexingOperationListener... listeners) throws IOException {
|
||||||
final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build();
|
final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build();
|
||||||
final IndexSettings indexSettings = new IndexSettings(indexMetaData, nodeSettings);
|
final IndexSettings indexSettings = new IndexSettings(indexMetaData, nodeSettings);
|
||||||
|
@ -263,7 +283,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
new NoneCircuitBreakerService(), mapperService);
|
new NoneCircuitBreakerService(), mapperService);
|
||||||
indexShard = new IndexShard(routing, indexSettings, shardPath, store, indexCache, mapperService, similarityService,
|
indexShard = new IndexShard(routing, indexSettings, shardPath, store, indexCache, mapperService, similarityService,
|
||||||
indexFieldDataService, null, indexEventListener, indexSearcherWrapper, threadPool, BigArrays.NON_RECYCLING_INSTANCE, warmer,
|
indexFieldDataService, null, indexEventListener, indexSearcherWrapper, threadPool, BigArrays.NON_RECYCLING_INSTANCE, warmer,
|
||||||
() -> {}, Collections.emptyList(), Arrays.asList(listeners));
|
globalCheckpointSyncer, Collections.emptyList(), Arrays.asList(listeners));
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
if (success == false) {
|
if (success == false) {
|
||||||
|
@ -293,7 +313,8 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
*/
|
*/
|
||||||
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
||||||
closeShards(current);
|
closeShards(current);
|
||||||
return newShard(routing, current.shardPath(), current.indexSettings().getIndexMetaData(), null, listeners);
|
return newShard(routing, current.shardPath(), current.indexSettings().getIndexMetaData(), null,
|
||||||
|
current.getGlobalCheckpointSyncer(), listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -28,14 +28,8 @@ import org.apache.lucene.util.LuceneTestCase;
|
||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.DocWriteResponse;
|
|
||||||
import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings;
|
|
||||||
import org.elasticsearch.discovery.DiscoveryModule;
|
|
||||||
import org.elasticsearch.client.RestClientBuilder;
|
|
||||||
import org.elasticsearch.env.NodeEnvironment;
|
|
||||||
import org.elasticsearch.script.ScriptService;
|
|
||||||
import org.elasticsearch.transport.MockTcpTransportPlugin;
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.DocWriteResponse;
|
||||||
import org.elasticsearch.action.ShardOperationFailedException;
|
import org.elasticsearch.action.ShardOperationFailedException;
|
||||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
|
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
|
||||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||||
|
@ -64,6 +58,7 @@ import org.elasticsearch.client.AdminClient;
|
||||||
import org.elasticsearch.client.Client;
|
import org.elasticsearch.client.Client;
|
||||||
import org.elasticsearch.client.Requests;
|
import org.elasticsearch.client.Requests;
|
||||||
import org.elasticsearch.client.RestClient;
|
import org.elasticsearch.client.RestClient;
|
||||||
|
import org.elasticsearch.client.RestClientBuilder;
|
||||||
import org.elasticsearch.cluster.ClusterState;
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
import org.elasticsearch.cluster.health.ClusterHealthStatus;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
@ -73,6 +68,7 @@ import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||||
|
import org.elasticsearch.cluster.routing.allocation.DiskThresholdSettings;
|
||||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
import org.elasticsearch.common.Nullable;
|
import org.elasticsearch.common.Nullable;
|
||||||
|
@ -99,9 +95,11 @@ import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
import org.elasticsearch.discovery.Discovery;
|
import org.elasticsearch.discovery.Discovery;
|
||||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
import org.elasticsearch.discovery.DiscoveryModule;
|
||||||
import org.elasticsearch.discovery.zen.ElectMasterService;
|
import org.elasticsearch.discovery.zen.ElectMasterService;
|
||||||
|
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||||
import org.elasticsearch.env.Environment;
|
import org.elasticsearch.env.Environment;
|
||||||
|
import org.elasticsearch.env.NodeEnvironment;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.index.IndexModule;
|
import org.elasticsearch.index.IndexModule;
|
||||||
import org.elasticsearch.index.IndexService;
|
import org.elasticsearch.index.IndexService;
|
||||||
|
@ -119,6 +117,7 @@ import org.elasticsearch.indices.store.IndicesStore;
|
||||||
import org.elasticsearch.node.NodeMocksPlugin;
|
import org.elasticsearch.node.NodeMocksPlugin;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
|
import org.elasticsearch.script.ScriptService;
|
||||||
import org.elasticsearch.search.MockSearchService;
|
import org.elasticsearch.search.MockSearchService;
|
||||||
import org.elasticsearch.search.SearchHit;
|
import org.elasticsearch.search.SearchHit;
|
||||||
import org.elasticsearch.test.client.RandomizingClient;
|
import org.elasticsearch.test.client.RandomizingClient;
|
||||||
|
@ -126,6 +125,7 @@ import org.elasticsearch.test.disruption.ServiceDisruptionScheme;
|
||||||
import org.elasticsearch.test.store.MockFSIndexStore;
|
import org.elasticsearch.test.store.MockFSIndexStore;
|
||||||
import org.elasticsearch.test.transport.AssertingLocalTransport;
|
import org.elasticsearch.test.transport.AssertingLocalTransport;
|
||||||
import org.elasticsearch.test.transport.MockTransportService;
|
import org.elasticsearch.test.transport.MockTransportService;
|
||||||
|
import org.elasticsearch.transport.MockTcpTransportPlugin;
|
||||||
import org.hamcrest.Matchers;
|
import org.hamcrest.Matchers;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.AfterClass;
|
import org.junit.AfterClass;
|
||||||
|
@ -575,7 +575,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
||||||
return Collections.emptySet();
|
return Collections.emptySet();
|
||||||
}
|
}
|
||||||
|
|
||||||
protected void beforeIndexDeletion() {
|
protected void beforeIndexDeletion() throws Exception {
|
||||||
cluster().beforeIndexDeletion();
|
cluster().beforeIndexDeletion();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -714,7 +714,7 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
||||||
* Creates a new {@link CreateIndexRequestBuilder} with the settings obtained from {@link #indexSettings()}.
|
* Creates a new {@link CreateIndexRequestBuilder} with the settings obtained from {@link #indexSettings()}.
|
||||||
*/
|
*/
|
||||||
public final CreateIndexRequestBuilder prepareCreate(String index) {
|
public final CreateIndexRequestBuilder prepareCreate(String index) {
|
||||||
return client().admin().indices().prepareCreate(index).setSettings(indexSettings());
|
return prepareCreate(index, -1);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -731,20 +731,26 @@ public abstract class ESIntegTestCase extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new {@link CreateIndexRequestBuilder} with the settings obtained from {@link #indexSettings()}.
|
* Creates a new {@link CreateIndexRequestBuilder} with the settings obtained from {@link #indexSettings()}, augmented
|
||||||
* The index that is created with this builder will only be allowed to allocate on the number of nodes passed to this
|
* by the given builder
|
||||||
* method.
|
|
||||||
* <p>
|
|
||||||
* This method uses allocation deciders to filter out certain nodes to allocate the created index on. It defines allocation
|
|
||||||
* rules based on <code>index.routing.allocation.exclude._name</code>.
|
|
||||||
* </p>
|
|
||||||
*/
|
*/
|
||||||
|
public CreateIndexRequestBuilder prepareCreate(String index, Settings.Builder settingsBuilder) {
|
||||||
|
return prepareCreate(index, -1, settingsBuilder);
|
||||||
|
}
|
||||||
|
/**
|
||||||
|
* Creates a new {@link CreateIndexRequestBuilder} with the settings obtained from {@link #indexSettings()}.
|
||||||
|
* The index that is created with this builder will only be allowed to allocate on the number of nodes passed to this
|
||||||
|
* method.
|
||||||
|
* <p>
|
||||||
|
* This method uses allocation deciders to filter out certain nodes to allocate the created index on. It defines allocation
|
||||||
|
* rules based on <code>index.routing.allocation.exclude._name</code>.
|
||||||
|
* </p>
|
||||||
|
*/
|
||||||
public CreateIndexRequestBuilder prepareCreate(String index, int numNodes, Settings.Builder settingsBuilder) {
|
public CreateIndexRequestBuilder prepareCreate(String index, int numNodes, Settings.Builder settingsBuilder) {
|
||||||
internalCluster().ensureAtLeastNumDataNodes(numNodes);
|
|
||||||
|
|
||||||
Settings.Builder builder = Settings.builder().put(indexSettings()).put(settingsBuilder.build());
|
Settings.Builder builder = Settings.builder().put(indexSettings()).put(settingsBuilder.build());
|
||||||
|
|
||||||
if (numNodes > 0) {
|
if (numNodes > 0) {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(numNodes);
|
||||||
getExcludeSettings(index, numNodes, builder);
|
getExcludeSettings(index, numNodes, builder);
|
||||||
}
|
}
|
||||||
return client().admin().indices().prepareCreate(index).setSettings(builder.build());
|
return client().admin().indices().prepareCreate(index).setSettings(builder.build());
|
||||||
|
|
|
@ -649,7 +649,7 @@ public abstract class ESTestCase extends LuceneTestCase {
|
||||||
* Returns a random subset of values (including a potential empty list)
|
* Returns a random subset of values (including a potential empty list)
|
||||||
*/
|
*/
|
||||||
public static <T> List<T> randomSubsetOf(Collection<T> collection) {
|
public static <T> List<T> randomSubsetOf(Collection<T> collection) {
|
||||||
return randomSubsetOf(randomInt(collection.size() - 1), collection);
|
return randomSubsetOf(randomInt(Math.max(collection.size() - 1, 0)), collection);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
Loading…
Reference in New Issue