Merge branch 'master' into feature/query-refactoring

Conflicts:
	core/src/main/java/org/elasticsearch/index/query/QueryParseContext.java
This commit is contained in:
javanna 2015-08-07 11:13:24 +02:00 committed by Luca Cavanna
commit 5edb287d3a
225 changed files with 6617 additions and 3510 deletions

2
.gitignore vendored
View File

@ -14,7 +14,7 @@ docs/html/
docs/build.log
/tmp/
backwards/
html_docs
## eclipse ignores (use 'mvn eclipse:eclipse' to build eclipse projects)
## All files (.project, .classpath, .settings/*) should be generated through Maven which
## will correctly set the classpath based on the declared dependencies and write settings

View File

@ -33,6 +33,7 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.InPlaceMergeSorter;
import org.apache.lucene.util.ToStringUtils;
import java.io.IOException;
import java.util.Arrays;
@ -62,13 +63,17 @@ import java.util.List;
public abstract class BlendedTermQuery extends Query {
private final Term[] terms;
private final float[] boosts;
public BlendedTermQuery(Term[] terms) {
public BlendedTermQuery(Term[] terms, float[] boosts) {
if (terms == null || terms.length == 0) {
throw new IllegalArgumentException("terms must not be null or empty");
}
if (boosts != null && boosts.length != terms.length) {
throw new IllegalArgumentException("boosts must have the same size as terms");
}
this.terms = terms;
this.boosts = boosts;
}
@Override
@ -231,8 +236,22 @@ public abstract class BlendedTermQuery extends Query {
@Override
public String toString(String field) {
return "blended(terms: " + Arrays.toString(terms) + ")";
StringBuilder builder = new StringBuilder("blended(terms:[");
for (int i = 0; i < terms.length; ++i) {
builder.append(terms[i]);
float boost = 1f;
if (boosts != null) {
boost = boosts[i];
}
builder.append(ToStringUtils.boost(boost));
builder.append(", ");
}
if (terms.length > 0) {
builder.setLength(builder.length() - 2);
}
builder.append("])");
builder.append(ToStringUtils.boost(getBoost()));
return builder.toString();
}
private volatile Term[] equalTerms = null;
@ -277,7 +296,7 @@ public abstract class BlendedTermQuery extends Query {
}
public static BlendedTermQuery booleanBlendedQuery(Term[] terms, final float[] boosts, final boolean disableCoord) {
return new BlendedTermQuery(terms) {
return new BlendedTermQuery(terms, boosts) {
@Override
protected Query topLevelQuery(Term[] terms, TermContext[] ctx, int[] docFreqs, int maxDoc) {
BooleanQuery query = new BooleanQuery(disableCoord);
@ -294,7 +313,7 @@ public abstract class BlendedTermQuery extends Query {
}
public static BlendedTermQuery commonTermsBlendedQuery(Term[] terms, final float[] boosts, final boolean disableCoord, final float maxTermFrequency) {
return new BlendedTermQuery(terms) {
return new BlendedTermQuery(terms, boosts) {
@Override
protected Query topLevelQuery(Term[] terms, TermContext[] ctx, int[] docFreqs, int maxDoc) {
BooleanQuery query = new BooleanQuery(true);
@ -334,7 +353,7 @@ public abstract class BlendedTermQuery extends Query {
}
public static BlendedTermQuery dismaxBlendedQuery(Term[] terms, final float[] boosts, final float tieBreakerMultiplier) {
return new BlendedTermQuery(terms) {
return new BlendedTermQuery(terms, boosts) {
@Override
protected Query topLevelQuery(Term[] terms, TermContext[] ctx, int[] docFreqs, int maxDoc) {
DisjunctionMaxQuery query = new DisjunctionMaxQuery(tieBreakerMultiplier);

View File

@ -272,13 +272,13 @@ public class TransportClusterHealthAction extends TransportMasterNodeReadAction<
} catch (IndexNotFoundException e) {
// one of the specified indices is not there - treat it as RED.
ClusterHealthResponse response = new ClusterHealthResponse(clusterName.value(), Strings.EMPTY_ARRAY, clusterState,
numberOfPendingTasks, numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState),
numberOfPendingTasks, numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(System.currentTimeMillis(), settings, clusterState),
pendingTaskTimeInQueue);
response.status = ClusterHealthStatus.RED;
return response;
}
return new ClusterHealthResponse(clusterName.value(), concreteIndices, clusterState, numberOfPendingTasks,
numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(settings, clusterState), pendingTaskTimeInQueue);
numberOfInFlightFetch, UnassignedInfo.getNumberOfDelayedUnassigned(System.currentTimeMillis(), settings, clusterState), pendingTaskTimeInQueue);
}
}

View File

@ -79,7 +79,7 @@ public class TransportIndicesShardStoresAction extends TransportMasterNodeReadAc
@Override
protected void masterOperation(IndicesShardStoresRequest request, ClusterState state, ActionListener<IndicesShardStoresResponse> listener) {
final RoutingTable routingTables = state.routingTable();
final RoutingNodes routingNodes = state.routingNodes();
final RoutingNodes routingNodes = state.getRoutingNodes();
final String[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request);
final Set<ShardId> shardIdsToFetch = new HashSet<>();

View File

@ -59,7 +59,7 @@ final class JVMCheck {
/** Returns an error message to the user for a broken version */
String getErrorMessage() {
StringBuilder sb = new StringBuilder();
sb.append("Java version: ").append(Constants.JAVA_VERSION);
sb.append("Java version: ").append(fullVersion());
sb.append(" suffers from critical bug ").append(bugUrl);
sb.append(" which can cause data corruption.");
sb.append(System.lineSeparator());
@ -111,7 +111,7 @@ final class JVMCheck {
*/
static void check() {
if (Boolean.parseBoolean(System.getProperty(JVM_BYPASS))) {
Loggers.getLogger(JVMCheck.class).warn("bypassing jvm version check for version [{}], this can result in data corruption!", Constants.JAVA_VERSION);
Loggers.getLogger(JVMCheck.class).warn("bypassing jvm version check for version [{}], this can result in data corruption!", fullVersion());
} else if ("Oracle Corporation".equals(Constants.JVM_VENDOR)) {
HotspotBug bug = JVM_BROKEN_HOTSPOT_VERSIONS.get(Constants.JVM_VERSION);
if (bug != null) {
@ -135,11 +135,28 @@ final class JVMCheck {
StringBuilder sb = new StringBuilder();
sb.append("IBM J9 runtimes < 2.8 suffer from several bugs which can cause data corruption.");
sb.append(System.lineSeparator());
sb.append("Your version: " + Constants.JVM_VERSION);
sb.append("Your version: " + fullVersion());
sb.append(System.lineSeparator());
sb.append("Please upgrade the JVM to a recent IBM JDK");
throw new RuntimeException(sb.toString());
}
}
}
/**
* Returns java + jvm version, looks like this:
* {@code Oracle Corporation 1.8.0_45 [Java HotSpot(TM) 64-Bit Server VM 25.45-b02]}
*/
static String fullVersion() {
StringBuilder sb = new StringBuilder();
sb.append(Constants.JAVA_VENDOR);
sb.append(" ");
sb.append(Constants.JAVA_VERSION);
sb.append(" [");
sb.append(Constants.JVM_NAME);
sb.append(" ");
sb.append(Constants.JVM_VERSION);
sb.append("]");
return sb.toString();
}
}

View File

@ -231,14 +231,6 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
return routingTable();
}
public RoutingNodes routingNodes() {
return routingTable.routingNodes(this);
}
public RoutingNodes getRoutingNodes() {
return readOnlyRoutingNodes();
}
public ClusterBlocks blocks() {
return this.blocks;
}
@ -269,14 +261,13 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
}
/**
* Returns a built (on demand) routing nodes view of the routing table. <b>NOTE, the routing nodes
* are mutable, use them just for read operations</b>
* Returns a built (on demand) routing nodes view of the routing table.
*/
public RoutingNodes readOnlyRoutingNodes() {
public RoutingNodes getRoutingNodes() {
if (routingNodes != null) {
return routingNodes;
}
routingNodes = routingTable.routingNodes(this);
routingNodes = new RoutingNodes(this);
return routingNodes;
}
@ -288,7 +279,7 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
sb.append("meta data version: ").append(metaData.version()).append("\n");
sb.append(nodes().prettyPrint());
sb.append(routingTable().prettyPrint());
sb.append(readOnlyRoutingNodes().prettyPrint());
sb.append(getRoutingNodes().prettyPrint());
return sb.toString();
}
@ -509,13 +500,13 @@ public class ClusterState implements ToXContent, Diffable<ClusterState> {
if (metrics.contains(Metric.ROUTING_NODES)) {
builder.startObject("routing_nodes");
builder.startArray("unassigned");
for (ShardRouting shardRouting : readOnlyRoutingNodes().unassigned()) {
for (ShardRouting shardRouting : getRoutingNodes().unassigned()) {
shardRouting.toXContent(builder, params);
}
builder.endArray();
builder.startObject("nodes");
for (RoutingNode routingNode : readOnlyRoutingNodes()) {
for (RoutingNode routingNode : getRoutingNodes()) {
builder.startArray(routingNode.nodeId() == null ? "null" : routingNode.nodeId(), XContentBuilder.FieldCaseConversion.NONE);
for (ShardRouting shardRouting : routingNode) {
shardRouting.toXContent(builder, params);

View File

@ -498,7 +498,6 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
for (ShardRouting shardRouting : indexShard) {
if (!shardRouting.primary()) {
builder.removeShard(shardRouting);
removed = true;
break;
}
}

View File

@ -58,6 +58,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
private final ImmutableOpenMap<String, ClusterState.Custom> customs;
private final boolean readOnly;
private int inactivePrimaryCount = 0;
private int inactiveShardCount = 0;
@ -67,6 +69,11 @@ public class RoutingNodes implements Iterable<RoutingNode> {
private final Map<String, ObjectIntHashMap<String>> nodesPerAttributeNames = new HashMap<>();
public RoutingNodes(ClusterState clusterState) {
this(clusterState, true);
}
public RoutingNodes(ClusterState clusterState, boolean readOnly) {
this.readOnly = readOnly;
this.metaData = clusterState.metaData();
this.blocks = clusterState.blocks();
this.routingTable = clusterState.routingTable();
@ -93,7 +100,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
entries = newArrayList();
nodesToShards.put(shard.currentNodeId(), entries);
}
ShardRouting sr = new ShardRouting(shard);
final ShardRouting sr = getRouting(shard, readOnly);
entries.add(sr);
assignedShardsAdd(sr);
if (shard.relocating()) {
@ -105,9 +112,12 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
// add the counterpart shard with relocatingNodeId reflecting the source from which
// it's relocating from.
sr = shard.buildTargetRelocatingShard();
entries.add(sr);
assignedShardsAdd(sr);
ShardRouting targetShardRouting = shard.buildTargetRelocatingShard();
if (readOnly) {
targetShardRouting.freeze();
}
entries.add(targetShardRouting);
assignedShardsAdd(targetShardRouting);
} else if (!shard.active()) { // shards that are initializing without being relocated
if (shard.primary()) {
inactivePrimaryCount++;
@ -115,7 +125,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
inactiveShardCount++;
}
} else {
ShardRouting sr = new ShardRouting(shard);
final ShardRouting sr = getRouting(shard, readOnly);
assignedShardsAdd(sr);
unassignedShards.add(sr);
}
@ -128,6 +138,15 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
}
private static ShardRouting getRouting(ShardRouting src, boolean readOnly) {
if (readOnly) {
src.freeze(); // we just freeze and reuse this instance if we are read only
} else {
src = new ShardRouting(src);
}
return src;
}
@Override
public Iterator<RoutingNode> iterator() {
return Iterators.unmodifiableIterator(nodesToShards.values().iterator());
@ -165,18 +184,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
return (T) customs.get(type);
}
public int requiredAverageNumberOfShardsPerNode() {
int totalNumberOfShards = 0;
// we need to recompute to take closed shards into account
for (ObjectCursor<IndexMetaData> cursor : metaData.indices().values()) {
IndexMetaData indexMetaData = cursor.value;
if (indexMetaData.state() == IndexMetaData.State.OPEN) {
totalNumberOfShards += indexMetaData.totalNumberOfShards();
}
}
return totalNumberOfShards / nodesToShards.size();
}
public boolean hasUnassigned() {
return !unassignedShards.isEmpty();
}
@ -339,6 +346,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* Moves a shard from unassigned to initialize state
*/
public void initialize(ShardRouting shard, String nodeId) {
ensureMutable();
assert shard.unassigned() : shard;
shard.initialize(nodeId);
node(nodeId).add(shard);
@ -355,6 +363,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* shard.
*/
public ShardRouting relocate(ShardRouting shard, String nodeId) {
ensureMutable();
relocatingShards++;
shard.relocate(nodeId);
ShardRouting target = shard.buildTargetRelocatingShard();
@ -367,6 +376,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* Mark a shard as started and adjusts internal statistics.
*/
public void started(ShardRouting shard) {
ensureMutable();
assert !shard.active() : "expected an intializing shard " + shard;
if (shard.relocatingNodeId() == null) {
// if this is not a target shard for relocation, we need to update statistics
@ -382,6 +392,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* Cancels a relocation of a shard that shard must relocating.
*/
public void cancelRelocation(ShardRouting shard) {
ensureMutable();
relocatingShards--;
shard.cancelRelocation();
}
@ -392,6 +403,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* @param shards the shard to have its primary status swapped.
*/
public void swapPrimaryFlag(ShardRouting... shards) {
ensureMutable();
for (ShardRouting shard : shards) {
if (shard.primary()) {
shard.moveFromPrimary();
@ -420,6 +432,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
* @param shard
*/
private void remove(ShardRouting shard) {
ensureMutable();
if (!shard.active() && shard.relocatingNodeId() == null) {
inactiveShardCount--;
assert inactiveShardCount >= 0;
@ -454,6 +467,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
private void assignedShardsRemove(ShardRouting shard) {
ensureMutable();
final List<ShardRouting> replicaSet = assignedShards.get(shard.shardId());
if (replicaSet != null) {
final Iterator<ShardRouting> iterator = replicaSet.iterator();
@ -473,6 +487,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
public void addNode(DiscoveryNode node) {
ensureMutable();
RoutingNode routingNode = new RoutingNode(node.id(), node);
nodesToShards.put(routingNode.nodeId(), routingNode);
}
@ -490,6 +505,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
public void reinitShadowPrimary(ShardRouting candidate) {
ensureMutable();
if (candidate.relocating()) {
cancelRelocation(candidate);
}
@ -623,6 +639,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
private void innerRemove() {
nodes.ensureMutable();
iterator.remove();
if (current.primary()) {
primaries--;
@ -810,6 +827,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
@Override
public void remove() {
ensureMutable();
delegate.remove();
RoutingNodes.this.remove(shard);
removed = true;
@ -827,6 +845,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
public void moveToUnassigned(UnassignedInfo unassignedInfo) {
ensureMutable();
if (isRemoved() == false) {
remove();
}
@ -839,4 +858,10 @@ public class RoutingNodes implements Iterable<RoutingNode> {
return shard;
}
}
private void ensureMutable() {
if (readOnly) {
throw new IllegalStateException("can't modify RoutingNodes - readonly");
}
}
}

View File

@ -57,6 +57,7 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
private AtomicBoolean rerouting = new AtomicBoolean();
private volatile long registeredNextDelaySetting = Long.MAX_VALUE;
private volatile ScheduledFuture registeredNextDelayFuture;
private volatile long unassignedShardsAllocatedTimestamp = 0;
@Inject
public RoutingService(Settings settings, ThreadPool threadPool, ClusterService clusterService, AllocationService allocationService) {
@ -87,6 +88,19 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
return this.allocationService;
}
/**
* Update the last time the allocator tried to assign unassigned shards
*
* This is used so that both the GatewayAllocator and RoutingService use a
* consistent timestamp for comparing which shards have been delayed to
* avoid a race condition where GatewayAllocator thinks the shard should
* be delayed and the RoutingService thinks it has already passed the delay
* and that the GatewayAllocator has/will handle it.
*/
public void setUnassignedShardsAllocatedTimestamp(long timeInMillis) {
this.unassignedShardsAllocatedTimestamp = timeInMillis;
}
/**
* Initiates a reroute.
*/
@ -108,20 +122,29 @@ public class RoutingService extends AbstractLifecycleComponent<RoutingService> i
if (nextDelaySetting > 0 && nextDelaySetting < registeredNextDelaySetting) {
FutureUtils.cancel(registeredNextDelayFuture);
registeredNextDelaySetting = nextDelaySetting;
TimeValue nextDelay = TimeValue.timeValueMillis(UnassignedInfo.findNextDelayedAllocationIn(settings, event.state()));
logger.info("delaying allocation for [{}] unassigned shards, next check in [{}]", UnassignedInfo.getNumberOfDelayedUnassigned(settings, event.state()), nextDelay);
registeredNextDelayFuture = threadPool.schedule(nextDelay, ThreadPool.Names.SAME, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
registeredNextDelaySetting = Long.MAX_VALUE;
reroute("assign delayed unassigned shards");
}
// We use System.currentTimeMillis here because we want the
// next delay from the "now" perspective, rather than the
// delay from the last time the GatewayAllocator tried to
// assign/delay the shard
TimeValue nextDelay = TimeValue.timeValueMillis(UnassignedInfo.findNextDelayedAllocationIn(System.currentTimeMillis(), settings, event.state()));
int unassignedDelayedShards = UnassignedInfo.getNumberOfDelayedUnassigned(unassignedShardsAllocatedTimestamp, settings, event.state());
if (unassignedDelayedShards > 0) {
logger.info("delaying allocation for [{}] unassigned shards, next check in [{}]",
unassignedDelayedShards, nextDelay);
registeredNextDelayFuture = threadPool.schedule(nextDelay, ThreadPool.Names.SAME, new AbstractRunnable() {
@Override
protected void doRun() throws Exception {
registeredNextDelaySetting = Long.MAX_VALUE;
reroute("assign delayed unassigned shards");
}
@Override
public void onFailure(Throwable t) {
logger.warn("failed to schedule/execute reroute post unassigned shard", t);
}
});
@Override
public void onFailure(Throwable t) {
logger.warn("failed to schedule/execute reroute post unassigned shard", t);
registeredNextDelaySetting = Long.MAX_VALUE;
}
});
}
} else {
logger.trace("no need to schedule reroute due to delayed unassigned, next_delay_setting [{}], registered [{}]", nextDelaySetting, registeredNextDelaySetting);
}

View File

@ -90,10 +90,6 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
return indicesRouting();
}
public RoutingNodes routingNodes(ClusterState state) {
return new RoutingNodes(state);
}
public RoutingTable validateRaiseException(MetaData metaData) throws RoutingValidationException {
RoutingTableValidation validation = validate(metaData);
if (!validation.valid()) {

View File

@ -199,12 +199,12 @@ public class UnassignedInfo implements ToXContent, Writeable<UnassignedInfo> {
/**
* The time in millisecond until this unassigned shard can be reassigned.
*/
public long getDelayAllocationExpirationIn(Settings settings, Settings indexSettings) {
public long getDelayAllocationExpirationIn(long unassignedShardsAllocatedTimestamp, Settings settings, Settings indexSettings) {
long delayTimeout = getAllocationDelayTimeoutSetting(settings, indexSettings);
if (delayTimeout == 0) {
return 0;
}
long delta = System.currentTimeMillis() - timestamp;
long delta = unassignedShardsAllocatedTimestamp - timestamp;
// account for time drift, treat it as no timeout
if (delta < 0) {
return 0;
@ -216,12 +216,12 @@ public class UnassignedInfo implements ToXContent, Writeable<UnassignedInfo> {
/**
* Returns the number of shards that are unassigned and currently being delayed.
*/
public static int getNumberOfDelayedUnassigned(Settings settings, ClusterState state) {
public static int getNumberOfDelayedUnassigned(long unassignedShardsAllocatedTimestamp, Settings settings, ClusterState state) {
int count = 0;
for (ShardRouting shard : state.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED)) {
if (shard.primary() == false) {
IndexMetaData indexMetaData = state.metaData().index(shard.getIndex());
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(settings, indexMetaData.getSettings());
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(unassignedShardsAllocatedTimestamp, settings, indexMetaData.getSettings());
if (delay > 0) {
count++;
}
@ -251,12 +251,12 @@ public class UnassignedInfo implements ToXContent, Writeable<UnassignedInfo> {
/**
* Finds the next (closest) delay expiration of an unassigned shard. Returns 0 if there are none.
*/
public static long findNextDelayedAllocationIn(Settings settings, ClusterState state) {
public static long findNextDelayedAllocationIn(long unassignedShardsAllocatedTimestamp, Settings settings, ClusterState state) {
long nextDelay = Long.MAX_VALUE;
for (ShardRouting shard : state.routingTable().shardsWithState(ShardRoutingState.UNASSIGNED)) {
if (shard.primary() == false) {
IndexMetaData indexMetaData = state.metaData().index(shard.getIndex());
long nextShardDelay = shard.unassignedInfo().getDelayAllocationExpirationIn(settings, indexMetaData.getSettings());
long nextShardDelay = shard.unassignedInfo().getDelayAllocationExpirationIn(unassignedShardsAllocatedTimestamp, settings, indexMetaData.getSettings());
if (nextShardDelay > 0 && nextShardDelay < nextDelay) {
nextDelay = nextShardDelay;
}

View File

@ -70,7 +70,7 @@ public class AllocationService extends AbstractComponent {
}
public RoutingAllocation.Result applyStartedShards(ClusterState clusterState, List<? extends ShardRouting> startedShards, boolean withReroute) {
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
// shuffle the unassigned nodes, just so we won't have things like poison failed shards
routingNodes.unassigned().shuffle();
StartedRerouteAllocation allocation = new StartedRerouteAllocation(allocationDeciders, routingNodes, clusterState.nodes(), startedShards, clusterInfoService.getClusterInfo());
@ -95,7 +95,7 @@ public class AllocationService extends AbstractComponent {
* <p>If the same instance of the routing table is returned, then no change has been made.</p>
*/
public RoutingAllocation.Result applyFailedShards(ClusterState clusterState, List<FailedRerouteAllocation.FailedShard> failedShards) {
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
// shuffle the unassigned nodes, just so we won't have things like poison failed shards
routingNodes.unassigned().shuffle();
FailedRerouteAllocation allocation = new FailedRerouteAllocation(allocationDeciders, routingNodes, clusterState.nodes(), failedShards, clusterInfoService.getClusterInfo());
@ -116,7 +116,7 @@ public class AllocationService extends AbstractComponent {
}
public RoutingAllocation.Result reroute(ClusterState clusterState, AllocationCommands commands, boolean explain) {
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
// we don't shuffle the unassigned shards here, to try and get as close as possible to
// a consistent result of the effect the commands have on the routing
// this allows systems to dry run the commands, see the resulting cluster state, and act on it
@ -149,7 +149,7 @@ public class AllocationService extends AbstractComponent {
* <p>If the same instance of the routing table is returned, then no change has been made.
*/
public RoutingAllocation.Result reroute(ClusterState clusterState, boolean debug) {
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = getMutableRoutingNodes(clusterState);
// shuffle the unassigned nodes, just so we won't have things like poison failed shards
routingNodes.unassigned().shuffle();
RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState.nodes(), clusterInfoService.getClusterInfo());
@ -461,4 +461,9 @@ public class AllocationService extends AbstractComponent {
assert matchedNode.isRemoved() : "failedShard " + failedShard + " was matched but wasn't removed";
return true;
}
private RoutingNodes getMutableRoutingNodes(ClusterState clusterState) {
RoutingNodes routingNodes = new RoutingNodes(clusterState, false); // this is a costly operation - only call this once!
return routingNodes;
}
}

View File

@ -389,7 +389,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
StringBuilder sb = new StringBuilder("failed to execute cluster state update in ").append(executionTime).append(", state:\nversion [").append(previousClusterState.version()).append("], source [").append(source).append("]\n");
sb.append(previousClusterState.nodes().prettyPrint());
sb.append(previousClusterState.routingTable().prettyPrint());
sb.append(previousClusterState.readOnlyRoutingNodes().prettyPrint());
sb.append(previousClusterState.getRoutingNodes().prettyPrint());
logger.trace(sb.toString(), e);
}
warnAboutSlowTaskIfNeeded(executionTime, source);
@ -533,7 +533,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
StringBuilder sb = new StringBuilder("failed to apply updated cluster state in ").append(executionTime).append(":\nversion [").append(newClusterState.version()).append("], uuid [").append(newClusterState.stateUUID()).append("], source [").append(source).append("]\n");
sb.append(newClusterState.nodes().prettyPrint());
sb.append(newClusterState.routingTable().prettyPrint());
sb.append(newClusterState.readOnlyRoutingNodes().prettyPrint());
sb.append(newClusterState.getRoutingNodes().prettyPrint());
logger.warn(sb.toString(), t);
// TODO: do we want to call updateTask.onFailure here?
}

View File

@ -57,6 +57,11 @@ public class BoostScoreFunction extends ScoreFunction {
};
}
@Override
public boolean needsScores() {
return false;
}
@Override
public String toString() {
return "boost[" + boost + "]";

View File

@ -91,6 +91,11 @@ public class FieldValueFactorFunction extends ScoreFunction {
};
}
@Override
public boolean needsScores() {
return false;
}
/**
* The Type class encapsulates the modification types that can be applied
* to the score/value product.

View File

@ -124,25 +124,31 @@ public class FiltersFunctionScoreQuery extends Query {
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
// TODO: needsScores
// if we dont need scores, just return the underlying Weight?
Weight subQueryWeight = subQuery.createWeight(searcher, needsScores);
if (needsScores == false) {
return subQuery.createWeight(searcher, needsScores);
}
boolean subQueryNeedsScores = combineFunction != CombineFunction.REPLACE;
Weight[] filterWeights = new Weight[filterFunctions.length];
for (int i = 0; i < filterFunctions.length; ++i) {
subQueryNeedsScores |= filterFunctions[i].function.needsScores();
filterWeights[i] = searcher.createNormalizedWeight(filterFunctions[i].filter, false);
}
return new CustomBoostFactorWeight(this, subQueryWeight, filterWeights);
Weight subQueryWeight = subQuery.createWeight(searcher, subQueryNeedsScores);
return new CustomBoostFactorWeight(this, subQueryWeight, filterWeights, subQueryNeedsScores);
}
class CustomBoostFactorWeight extends Weight {
final Weight subQueryWeight;
final Weight[] filterWeights;
final boolean needsScores;
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight, Weight[] filterWeights) throws IOException {
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight, Weight[] filterWeights, boolean needsScores) throws IOException {
super(parent);
this.subQueryWeight = subQueryWeight;
this.filterWeights = filterWeights;
this.needsScores = needsScores;
}
@Override
@ -179,7 +185,7 @@ public class FiltersFunctionScoreQuery extends Query {
Scorer filterScorer = filterWeights[i].scorer(context, null); // no need to apply accepted docs
docSets[i] = Lucene.asSequentialAccessBits(context.reader().maxDoc(), filterScorer);
}
return new FiltersFunctionFactorScorer(this, subQueryScorer, scoreMode, filterFunctions, maxBoost, functions, docSets, combineFunction, minScore);
return new FiltersFunctionFactorScorer(this, subQueryScorer, scoreMode, filterFunctions, maxBoost, functions, docSets, combineFunction, minScore, needsScores);
}
@Override
@ -269,21 +275,26 @@ public class FiltersFunctionScoreQuery extends Query {
private final ScoreMode scoreMode;
private final LeafScoreFunction[] functions;
private final Bits[] docSets;
private final boolean needsScores;
private FiltersFunctionFactorScorer(CustomBoostFactorWeight w, Scorer scorer, ScoreMode scoreMode, FilterFunction[] filterFunctions,
float maxBoost, LeafScoreFunction[] functions, Bits[] docSets, CombineFunction scoreCombiner, Float minScore) throws IOException {
float maxBoost, LeafScoreFunction[] functions, Bits[] docSets, CombineFunction scoreCombiner, Float minScore, boolean needsScores) throws IOException {
super(w, scorer, maxBoost, scoreCombiner, minScore);
this.scoreMode = scoreMode;
this.filterFunctions = filterFunctions;
this.functions = functions;
this.docSets = docSets;
this.needsScores = needsScores;
}
@Override
public float innerScore() throws IOException {
int docId = scorer.docID();
double factor = 1.0f;
float subQueryScore = scorer.score();
// Even if the weight is created with needsScores=false, it might
// be costly to call score(), so we explicitly check if scores
// are needed
float subQueryScore = needsScores ? scorer.score() : 0f;
if (scoreMode == ScoreMode.First) {
for (int i = 0; i < filterFunctions.length; i++) {
if (docSets[i].get(docId)) {

View File

@ -44,7 +44,7 @@ public class FunctionScoreQuery extends Query {
public FunctionScoreQuery(Query subQuery, ScoreFunction function, Float minScore) {
this.subQuery = subQuery;
this.function = function;
this.combineFunction = function == null? combineFunction.MULT : function.getDefaultScoreCombiner();
this.combineFunction = function == null? CombineFunction.MULT : function.getDefaultScoreCombiner();
this.minScore = minScore;
}
@ -87,19 +87,27 @@ public class FunctionScoreQuery extends Query {
@Override
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
// TODO: needsScores
// if we don't need scores, just return the underlying weight?
Weight subQueryWeight = subQuery.createWeight(searcher, needsScores);
return new CustomBoostFactorWeight(this, subQueryWeight);
if (needsScores == false) {
return subQuery.createWeight(searcher, needsScores);
}
boolean subQueryNeedsScores =
combineFunction != CombineFunction.REPLACE // if we don't replace we need the original score
|| function == null // when the function is null, we just multiply the score, so we need it
|| function.needsScores(); // some scripts can replace with a script that returns eg. 1/_score
Weight subQueryWeight = subQuery.createWeight(searcher, subQueryNeedsScores);
return new CustomBoostFactorWeight(this, subQueryWeight, subQueryNeedsScores);
}
class CustomBoostFactorWeight extends Weight {
final Weight subQueryWeight;
final boolean needsScores;
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight) throws IOException {
public CustomBoostFactorWeight(Query parent, Weight subQueryWeight, boolean needsScores) throws IOException {
super(parent);
this.subQueryWeight = subQueryWeight;
this.needsScores = needsScores;
}
@Override
@ -129,7 +137,7 @@ public class FunctionScoreQuery extends Query {
if (function != null) {
leafFunction = function.getLeafScoreFunction(context);
}
return new FunctionFactorScorer(this, subQueryScorer, leafFunction, maxBoost, combineFunction, minScore);
return new FunctionFactorScorer(this, subQueryScorer, leafFunction, maxBoost, combineFunction, minScore, needsScores);
}
@Override
@ -150,16 +158,21 @@ public class FunctionScoreQuery extends Query {
static class FunctionFactorScorer extends CustomBoostFactorScorer {
private final LeafScoreFunction function;
private final boolean needsScores;
private FunctionFactorScorer(CustomBoostFactorWeight w, Scorer scorer, LeafScoreFunction function, float maxBoost, CombineFunction scoreCombiner, Float minScore)
private FunctionFactorScorer(CustomBoostFactorWeight w, Scorer scorer, LeafScoreFunction function, float maxBoost, CombineFunction scoreCombiner, Float minScore, boolean needsScores)
throws IOException {
super(w, scorer, maxBoost, scoreCombiner, minScore);
this.function = function;
this.needsScores = needsScores;
}
@Override
public float innerScore() throws IOException {
float score = scorer.score();
// Even if the weight is created with needsScores=false, it might
// be costly to call score(), so we explicitly check if scores
// are needed
float score = needsScores ? scorer.score() : 0f;
if (function == null) {
return subQueryBoost * score;
} else {

View File

@ -81,4 +81,8 @@ public class RandomScoreFunction extends ScoreFunction {
};
}
@Override
public boolean needsScores() {
return false;
}
}

View File

@ -39,4 +39,11 @@ public abstract class ScoreFunction {
}
public abstract LeafScoreFunction getLeafScoreFunction(LeafReaderContext ctx) throws IOException;
/**
* Indicates if document scores are needed by this function.
*
* @return {@code true} if scores are needed.
*/
public abstract boolean needsScores();
}

View File

@ -126,6 +126,11 @@ public class ScriptScoreFunction extends ScoreFunction {
};
}
@Override
public boolean needsScores() {
return script.needsScores();
}
@Override
public String toString() {
return "script" + sScript.toString();

View File

@ -71,6 +71,11 @@ public class WeightFactorFunction extends ScoreFunction {
};
}
@Override
public boolean needsScores() {
return false;
}
public Explanation explainWeight() {
return Explanation.match(getWeight(), "weight");
}
@ -99,5 +104,10 @@ public class WeightFactorFunction extends ScoreFunction {
}
};
}
@Override
public boolean needsScores() {
return false;
}
}
}

View File

@ -113,6 +113,10 @@ public class GatewayAllocator extends AbstractComponent {
}
public boolean allocateUnassigned(final RoutingAllocation allocation) {
// Take a snapshot of the current time and tell the RoutingService
// about it, so it will use a consistent timestamp for delays
long lastAllocateUnassignedRun = System.currentTimeMillis();
this.routingService.setUnassignedShardsAllocatedTimestamp(lastAllocateUnassignedRun);
boolean changed = false;
RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned();
@ -127,7 +131,7 @@ public class GatewayAllocator extends AbstractComponent {
changed |= primaryShardAllocator.allocateUnassigned(allocation);
changed |= replicaShardAllocator.processExistingRecoveries(allocation);
changed |= replicaShardAllocator.allocateUnassigned(allocation);
changed |= replicaShardAllocator.allocateUnassigned(allocation, lastAllocateUnassignedRun);
return changed;
}

View File

@ -111,6 +111,10 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
}
public boolean allocateUnassigned(RoutingAllocation allocation) {
return allocateUnassigned(allocation, System.currentTimeMillis());
}
public boolean allocateUnassigned(RoutingAllocation allocation, long allocateUnassignedTimestapm) {
boolean changed = false;
final RoutingNodes routingNodes = allocation.routingNodes();
final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator();
@ -174,7 +178,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent {
// will anyhow wait to find an existing copy of the shard to be allocated
// note: the other side of the equation is scheduling a reroute in a timely manner, which happens in the RoutingService
IndexMetaData indexMetaData = allocation.metaData().index(shard.getIndex());
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(settings, indexMetaData.getSettings());
long delay = shard.unassignedInfo().getDelayAllocationExpirationIn(allocateUnassignedTimestapm, settings, indexMetaData.getSettings());
if (delay > 0) {
logger.debug("[{}][{}]: delaying allocation of [{}] for [{}]", shard.index(), shard.id(), shard, TimeValue.timeValueMillis(delay));
/**

View File

@ -228,8 +228,6 @@ public final class ShardGetService extends AbstractIndexShardComponent {
if (source.ttl > 0) {
value = docMapper.TTLFieldMapper().valueForSearch(source.timestamp + source.ttl);
}
} else if (field.equals(SizeFieldMapper.NAME) && docMapper.rootMapper(SizeFieldMapper.class).fieldType().stored()) {
value = source.source.length();
} else {
if (searchLookup == null) {
searchLookup = new SearchLookup(mapperService, null, new String[]{type});

View File

@ -48,7 +48,6 @@ import org.elasticsearch.index.mapper.internal.IdFieldMapper;
import org.elasticsearch.index.mapper.internal.IndexFieldMapper;
import org.elasticsearch.index.mapper.internal.ParentFieldMapper;
import org.elasticsearch.index.mapper.internal.RoutingFieldMapper;
import org.elasticsearch.index.mapper.internal.SizeFieldMapper;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.TTLFieldMapper;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
@ -106,7 +105,6 @@ public class DocumentMapper implements ToXContent {
this.rootMappers.put(IdFieldMapper.class, new IdFieldMapper(indexSettings, mapperService.fullName(IdFieldMapper.NAME)));
this.rootMappers.put(RoutingFieldMapper.class, new RoutingFieldMapper(indexSettings, mapperService.fullName(RoutingFieldMapper.NAME)));
// add default mappers, order is important (for example analyzer should come before the rest to set context.analyzer)
this.rootMappers.put(SizeFieldMapper.class, new SizeFieldMapper(indexSettings, mapperService.fullName(SizeFieldMapper.NAME)));
this.rootMappers.put(IndexFieldMapper.class, new IndexFieldMapper(indexSettings, mapperService.fullName(IndexFieldMapper.NAME)));
this.rootMappers.put(SourceFieldMapper.class, new SourceFieldMapper(indexSettings));
this.rootMappers.put(TypeFieldMapper.class, new TypeFieldMapper(indexSettings, mapperService.fullName(TypeFieldMapper.NAME)));
@ -283,10 +281,6 @@ public class DocumentMapper implements ToXContent {
return rootMapper(ParentFieldMapper.class);
}
public SizeFieldMapper sizeFieldMapper() {
return rootMapper(SizeFieldMapper.class);
}
public TimestampFieldMapper timestampFieldMapper() {
return rootMapper(TimestampFieldMapper.class);
}
@ -299,10 +293,6 @@ public class DocumentMapper implements ToXContent {
return rootMapper(IndexFieldMapper.class);
}
public SizeFieldMapper SizeFieldMapper() {
return rootMapper(SizeFieldMapper.class);
}
public Query typeFilter() {
return typeMapper().fieldType().termQuery(type, null);
}

View File

@ -20,7 +20,9 @@
package org.elasticsearch.index.mapper;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSortedMap;
import com.google.common.collect.Maps;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
@ -35,8 +37,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.mapper.core.*;
import org.elasticsearch.index.mapper.geo.GeoPointFieldMapper;
@ -50,6 +50,7 @@ import org.elasticsearch.index.similarity.SimilarityLookupService;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -73,6 +74,7 @@ public class DocumentMapperParser {
private volatile ImmutableMap<String, Mapper.TypeParser> typeParsers;
private volatile ImmutableMap<String, Mapper.TypeParser> rootTypeParsers;
private volatile ImmutableMap<String, Mapper.TypeParser> additionalRootMappers;
public DocumentMapperParser(@IndexSettings Settings indexSettings, MapperService mapperService, AnalysisService analysisService,
SimilarityLookupService similarityLookupService, ScriptService scriptService) {
@ -109,7 +111,6 @@ public class DocumentMapperParser {
typeParsers = typeParsersBuilder.immutableMap();
rootTypeParsers = new MapBuilder<String, Mapper.TypeParser>()
.put(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser())
.put(IndexFieldMapper.NAME, new IndexFieldMapper.TypeParser())
.put(SourceFieldMapper.NAME, new SourceFieldMapper.TypeParser())
.put(TypeFieldMapper.NAME, new TypeFieldMapper.TypeParser())
@ -123,6 +124,7 @@ public class DocumentMapperParser {
.put(IdFieldMapper.NAME, new IdFieldMapper.TypeParser())
.put(FieldNamesFieldMapper.NAME, new FieldNamesFieldMapper.TypeParser())
.immutableMap();
additionalRootMappers = ImmutableSortedMap.<String, Mapper.TypeParser>of();
indexVersionCreated = Version.indexCreated(indexSettings);
}
@ -139,6 +141,10 @@ public class DocumentMapperParser {
rootTypeParsers = new MapBuilder<>(rootTypeParsers)
.put(type, typeParser)
.immutableMap();
additionalRootMappers = ImmutableSortedMap.<String, Mapper.TypeParser>naturalOrder()
.putAll(additionalRootMappers)
.put(type, typeParser)
.build();
}
}
@ -204,6 +210,10 @@ public class DocumentMapperParser {
Mapper.TypeParser.ParserContext parserContext = parserContext();
// parse RootObjectMapper
DocumentMapper.Builder docBuilder = doc(indexSettings, (RootObjectMapper.Builder) rootObjectTypeParser.parse(type, mapping, parserContext), mapperService);
// Add default mapping for the plugged-in meta mappers
for (Map.Entry<String, Mapper.TypeParser> entry : additionalRootMappers.entrySet()) {
docBuilder.put((MetadataFieldMapper.Builder<?, ?>) entry.getValue().parse(entry.getKey(), Collections.<String, Object>emptyMap(), parserContext));
}
Iterator<Map.Entry<String, Object>> iterator = mapping.entrySet().iterator();
// parse DocumentMapper
while(iterator.hasNext()) {

View File

@ -185,7 +185,9 @@ public class QueryShardContext {
}
public void addNamedQuery(String name, Query query) {
namedQueries.put(name, query);
if (query != null) {
namedQueries.put(name, query);
}
}
public ImmutableMap<String, Query> copyNamedQueries() {

View File

@ -289,6 +289,11 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser {
this.fieldData = fieldData;
}
@Override
public boolean needsScores() {
return false;
}
@Override
protected NumericDoubleValues distance(LeafReaderContext context) {
final MultiGeoPointValues geoPointValues = fieldData.load(context).getGeoPointValues();
@ -352,6 +357,11 @@ public abstract class DecayFunctionParser implements ScoreFunctionParser {
this.origin = origin;
}
@Override
public boolean needsScores() {
return false;
}
@Override
protected NumericDoubleValues distance(LeafReaderContext context) {
final SortedNumericDoubleValues doubleValues = fieldData.load(context).getDoubleValues();

View File

@ -84,7 +84,7 @@ public class MultiMatchQuery extends MatchQuery {
throw new IllegalStateException("No such type: " + type);
}
final List<? extends Query> queries = queryBuilder.buildGroupedQueries(type, fieldNames, value, minimumShouldMatch);
return queryBuilder.conbineGrouped(queries);
return queryBuilder.combineGrouped(queries);
}
private QueryBuilder queryBuilder;
@ -118,7 +118,7 @@ public class MultiMatchQuery extends MatchQuery {
return parseAndApply(type, field, value, minimumShouldMatch, boostValue);
}
public Query conbineGrouped(List<? extends Query> groupQuery) {
public Query combineGrouped(List<? extends Query> groupQuery) {
if (groupQuery == null || groupQuery.isEmpty()) {
return null;
}
@ -195,7 +195,7 @@ public class MultiMatchQuery extends MatchQuery {
blendedFields = null;
}
final FieldAndFieldType fieldAndFieldType = group.get(0);
Query q = parseGroup(type.matchQueryType(), fieldAndFieldType.field, fieldAndFieldType.boost, value, minimumShouldMatch);
Query q = parseGroup(type.matchQueryType(), fieldAndFieldType.field, 1f, value, minimumShouldMatch);
if (q != null) {
queries.add(q);
}

View File

@ -244,7 +244,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
}
private void applyDeletedShards(final ClusterChangedEvent event) {
RoutingNodes.RoutingNodeIterator routingNode = event.state().readOnlyRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
if (routingNode == null) {
return;
}
@ -284,7 +284,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private void applyNewIndices(final ClusterChangedEvent event) {
// we only create indices for shards that are allocated
RoutingNodes.RoutingNodeIterator routingNode = event.state().readOnlyRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
if (routingNode == null) {
return;
}
@ -465,7 +465,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
}
RoutingTable routingTable = event.state().routingTable();
RoutingNodes.RoutingNodeIterator routingNode = event.state().readOnlyRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
if (routingNode == null) {
failedShards.clear();
@ -545,7 +545,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private void cleanFailedShards(final ClusterChangedEvent event) {
RoutingTable routingTable = event.state().routingTable();
RoutingNodes.RoutingNodeIterator routingNode = event.state().readOnlyRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
RoutingNodes.RoutingNodeIterator routingNode = event.state().getRoutingNodes().routingNodeIter(event.state().nodes().localNodeId());
if (routingNode == null) {
failedShards.clear();
return;

View File

@ -93,7 +93,7 @@ public class RecoverySource extends AbstractComponent {
// starting recovery from that our (the source) shard state is marking the shard to be in recovery mode as well, otherwise
// the index operations will not be routed to it properly
RoutingNode node = clusterService.state().readOnlyRoutingNodes().node(request.targetNode().id());
RoutingNode node = clusterService.state().getRoutingNodes().node(request.targetNode().id());
if (node == null) {
logger.debug("delaying recovery of {} as source node {} is unknown", request.shardId(), request.targetNode());
throw new DelayRecoveryException("source node does not have the node [" + request.targetNode() + "] in its state yet..");

View File

@ -86,6 +86,10 @@ public class NativeScriptEngineService extends AbstractComponent implements Scri
script.setLookup(lookup.getLeafSearchLookup(context));
return script;
}
@Override
public boolean needsScores() {
return scriptFactory.needsScores();
}
};
}

View File

@ -41,4 +41,11 @@ public interface NativeScriptFactory {
* @param params The parameters passed to the script. Can be <tt>null</tt>.
*/
ExecutableScript newScript(@Nullable Map<String, Object> params);
/**
* Indicates if document scores may be needed by the produced scripts.
*
* @return {@code true} if scores are needed.
*/
boolean needsScores();
}

View File

@ -29,4 +29,11 @@ public interface SearchScript {
LeafSearchScript getLeafSearchScript(LeafReaderContext context) throws IOException;
/**
* Indicates if document scores may be needed by this {@link SearchScript}.
*
* @return {@code true} if scores are needed.
*/
boolean needsScores();
}

View File

@ -112,7 +112,6 @@ public class ExpressionScriptEngineService extends AbstractComponent implements
for (String variable : expr.variables) {
if (variable.equals("_score")) {
bindings.add(new SortField("_score", SortField.Type.SCORE));
} else if (variable.equals("_value")) {
specialValue = new ReplaceableConstValueSource();
bindings.add("_value", specialValue);
@ -173,7 +172,8 @@ public class ExpressionScriptEngineService extends AbstractComponent implements
}
}
return new ExpressionSearchScript(compiledScript, bindings, specialValue);
final boolean needsScores = expr.getSortField(bindings, false).needsScores();
return new ExpressionSearchScript(compiledScript, bindings, specialValue, needsScores);
} catch (Exception exception) {
throw new ScriptException("Error during search with " + compiledScript, exception);
}

View File

@ -46,14 +46,21 @@ class ExpressionSearchScript implements SearchScript {
final SimpleBindings bindings;
final ValueSource source;
final ReplaceableConstValueSource specialValue; // _value
final boolean needsScores;
Scorer scorer;
int docid;
ExpressionSearchScript(CompiledScript c, SimpleBindings b, ReplaceableConstValueSource v) {
ExpressionSearchScript(CompiledScript c, SimpleBindings b, ReplaceableConstValueSource v, boolean needsScores) {
compiledScript = c;
bindings = b;
source = ((Expression)compiledScript.compiled()).getValueSource(bindings);
specialValue = v;
this.needsScores = needsScores;
}
@Override
public boolean needsScores() {
return needsScores;
}
@Override

View File

@ -168,6 +168,12 @@ public class GroovyScriptEngineService extends AbstractComponent implements Scri
}
return new GroovyScript(compiledScript, scriptObject, leafLookup, logger);
}
@Override
public boolean needsScores() {
// TODO: can we reliably know if a groovy script makes use of _score
return true;
}
};
}

View File

@ -216,8 +216,7 @@ public abstract class ValuesSource {
@Override
public boolean needsScores() {
// TODO: add a way to know whether scripts are using scores
return true;
return script.needsScores();
}
}
@ -295,8 +294,7 @@ public abstract class ValuesSource {
@Override
public boolean needsScores() {
// TODO: add a way to know whether scripts are using scores
return true;
return script.needsScores();
}
@Override
@ -431,8 +429,7 @@ public abstract class ValuesSource {
@Override
public boolean needsScores() {
// TODO: add a way to know whether scripts are using scores
return true;
return script.needsScores();
}
}
@ -451,8 +448,7 @@ public abstract class ValuesSource {
@Override
public boolean needsScores() {
// TODO: add a way to know whether scripts are using scores
return true;
return script.needsScores();
}
@Override

View File

@ -73,9 +73,9 @@ public class ClusterAllocationRerouteBenchmark {
logger.info("[{}] starting... ", i);
long runStart = System.currentTimeMillis();
ClusterState clusterState = initialClusterState;
while (clusterState.readOnlyRoutingNodes().hasUnassignedShards()) {
logger.info("[{}] remaining unassigned {}", i, clusterState.readOnlyRoutingNodes().unassigned().size());
RoutingAllocation.Result result = strategy.applyStartedShards(clusterState, clusterState.readOnlyRoutingNodes().shardsWithState(INITIALIZING));
while (clusterState.getRoutingNodes().hasUnassignedShards()) {
logger.info("[{}] remaining unassigned {}", i, clusterState.getRoutingNodes().unassigned().size());
RoutingAllocation.Result result = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(result).build();
result = strategy.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingResult(result).build();

View File

@ -34,6 +34,11 @@ public class NativeScript1 extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeScript1();
}
@Override
public boolean needsScores() {
return false;
}
}
public static final String NATIVE_SCRIPT_1 = "native_1";

View File

@ -34,6 +34,11 @@ public class NativeScript2 extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeScript2();
}
@Override
public boolean needsScores() {
return false;
}
}
public static final String NATIVE_SCRIPT_2 = "native_2";

View File

@ -34,6 +34,11 @@ public class NativeScript3 extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeScript3();
}
@Override
public boolean needsScores() {
return false;
}
}
public static final String NATIVE_SCRIPT_3 = "native_3";

View File

@ -34,6 +34,11 @@ public class NativeScript4 extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeScript4();
}
@Override
public boolean needsScores() {
return false;
}
}
public static final String NATIVE_SCRIPT_4 = "native_4";

View File

@ -36,6 +36,11 @@ public class NativeConstantForLoopScoreScript extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeConstantForLoopScoreScript(params);
}
@Override
public boolean needsScores() {
return false;
}
}
private NativeConstantForLoopScoreScript(Map<String, Object> params) {

View File

@ -36,6 +36,11 @@ public class NativeConstantScoreScript extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeConstantScoreScript();
}
@Override
public boolean needsScores() {
return false;
}
}
private NativeConstantScoreScript() {

View File

@ -42,6 +42,11 @@ public class NativeNaiveTFIDFScoreScript extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativeNaiveTFIDFScoreScript(params);
}
@Override
public boolean needsScores() {
return false;
}
}
private NativeNaiveTFIDFScoreScript(Map<String, Object> params) {

View File

@ -44,6 +44,11 @@ public class NativePayloadSumNoRecordScoreScript extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativePayloadSumNoRecordScoreScript(params);
}
@Override
public boolean needsScores() {
return false;
}
}
private NativePayloadSumNoRecordScoreScript(Map<String, Object> params) {

View File

@ -44,6 +44,11 @@ public class NativePayloadSumScoreScript extends AbstractSearchScript {
public ExecutableScript newScript(@Nullable Map<String, Object> params) {
return new NativePayloadSumScoreScript(params);
}
@Override
public boolean needsScores() {
return false;
}
}
private NativePayloadSumScoreScript(Map<String, Object> params) {

View File

@ -96,7 +96,7 @@ public class AckClusterUpdateSettingsIT extends ESIntegTestCase {
for (Client client : clients()) {
ClusterState clusterState = getLocalClusterState(client);
assertThat(clusterState.routingNodes().metaData().transientSettings().get("cluster.routing.allocation.exclude._id"), equalTo(excludedNodeId));
assertThat(clusterState.getRoutingNodes().metaData().transientSettings().get("cluster.routing.allocation.exclude._id"), equalTo(excludedNodeId));
for (IndexRoutingTable indexRoutingTable : clusterState.routingTable()) {
for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) {
for (ShardRouting shardRouting : indexShardRoutingTable) {

View File

@ -190,7 +190,7 @@ public class AckIT extends ESIntegTestCase {
for (Client client : clients()) {
ClusterState clusterState = getLocalClusterState(client);
for (ShardRouting shardRouting : clusterState.routingNodes().routingNodeIter(moveAllocationCommand.fromNode())) {
for (ShardRouting shardRouting : clusterState.getRoutingNodes().routingNodeIter(moveAllocationCommand.fromNode())) {
//if the shard that we wanted to move is still on the same node, it must be relocating
if (shardRouting.shardId().equals(moveAllocationCommand.shardId())) {
assertThat(shardRouting.relocating(), equalTo(true));
@ -199,7 +199,7 @@ public class AckIT extends ESIntegTestCase {
}
boolean found = false;
for (ShardRouting shardRouting : clusterState.routingNodes().routingNodeIter(moveAllocationCommand.toNode())) {
for (ShardRouting shardRouting : clusterState.getRoutingNodes().routingNodeIter(moveAllocationCommand.toNode())) {
if (shardRouting.shardId().equals(moveAllocationCommand.shardId())) {
assertThat(shardRouting.state(), anyOf(equalTo(ShardRoutingState.INITIALIZING), equalTo(ShardRoutingState.STARTED)));
found = true;
@ -240,7 +240,7 @@ public class AckIT extends ESIntegTestCase {
//all nodes hold the same cluster state version. We only know there was no need to change anything, thus no need for ack on this update.
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
boolean found = false;
for (ShardRouting shardRouting : clusterStateResponse.getState().routingNodes().routingNodeIter(moveAllocationCommand.fromNode())) {
for (ShardRouting shardRouting : clusterStateResponse.getState().getRoutingNodes().routingNodeIter(moveAllocationCommand.fromNode())) {
//the shard that we wanted to move is still on the same node, as we had dryRun flag
if (shardRouting.shardId().equals(moveAllocationCommand.shardId())) {
assertThat(shardRouting.started(), equalTo(true));
@ -250,7 +250,7 @@ public class AckIT extends ESIntegTestCase {
}
assertThat(found, equalTo(true));
for (ShardRouting shardRouting : clusterStateResponse.getState().routingNodes().routingNodeIter(moveAllocationCommand.toNode())) {
for (ShardRouting shardRouting : clusterStateResponse.getState().getRoutingNodes().routingNodeIter(moveAllocationCommand.toNode())) {
if (shardRouting.shardId().equals(moveAllocationCommand.shardId())) {
fail("shard [" + shardRouting + "] shouldn't be on node [" + moveAllocationCommand.toString() + "]");
}
@ -277,7 +277,7 @@ public class AckIT extends ESIntegTestCase {
String toNodeId = null;
ShardRouting shardToBeMoved = null;
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
for (RoutingNode routingNode : clusterStateResponse.getState().routingNodes()) {
for (RoutingNode routingNode : clusterStateResponse.getState().getRoutingNodes()) {
if (routingNode.node().isDataNode()) {
if (fromNodeId == null && routingNode.numberOfOwningShards() > 0) {
fromNodeId = routingNode.nodeId();

View File

@ -34,15 +34,18 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationComman
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Allocation;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.InternalTestCluster;
import org.junit.Test;
import java.nio.file.Path;
@ -93,7 +96,7 @@ public class ClusterRerouteIT extends ESIntegTestCase {
.execute().actionGet();
ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(2));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(2));
logger.info("--> explicitly allocate shard 1, *under dry_run*");
state = client().admin().cluster().prepareReroute()
@ -101,28 +104,28 @@ public class ClusterRerouteIT extends ESIntegTestCase {
.add(new AllocateAllocationCommand(new ShardId("test", 0), node_1, true))
.setDryRun(true)
.execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
logger.info("--> get the state, verify nothing changed because of the dry run");
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(2));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(2));
logger.info("--> explicitly allocate shard 1, actually allocating, no dry run");
state = client().admin().cluster().prepareReroute()
.setExplain(randomBoolean())
.add(new AllocateAllocationCommand(new ShardId("test", 0), node_1, true))
.execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
assertThat(healthResponse.isTimedOut(), equalTo(false));
logger.info("--> get the state, verify shard 1 primary allocated");
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
logger.info("--> move shard 1 primary from node1 to node2");
state = client().admin().cluster().prepareReroute()
@ -130,8 +133,8 @@ public class ClusterRerouteIT extends ESIntegTestCase {
.add(new MoveAllocationCommand(new ShardId("test", 0), node_1, node_2))
.execute().actionGet().getState();
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.RELOCATING));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_2).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.RELOCATING));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_2).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().setWaitForRelocatingShards(0).execute().actionGet();
@ -139,8 +142,8 @@ public class ClusterRerouteIT extends ESIntegTestCase {
logger.info("--> get the state, verify shard 1 primary moved from node1 to node2");
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_2).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_2).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
}
@Test
@ -160,6 +163,40 @@ public class ClusterRerouteIT extends ESIntegTestCase {
rerouteWithAllocateLocalGateway(commonSettings);
}
@Test
public void testDelayWithALargeAmountOfShards() throws Exception {
Settings commonSettings = settingsBuilder()
.put("gateway.type", "local")
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 1)
.build();
logger.info("--> starting 4 nodes");
String node_1 = internalCluster().startNode(commonSettings);
internalCluster().startNode(commonSettings);
internalCluster().startNode(commonSettings);
internalCluster().startNode(commonSettings);
assertThat(cluster().size(), equalTo(4));
ClusterHealthResponse healthResponse = client().admin().cluster().prepareHealth().setWaitForNodes("4").execute().actionGet();
assertThat(healthResponse.isTimedOut(), equalTo(false));
logger.info("--> create indices");
for (int i = 0; i < 25; i++) {
client().admin().indices().prepareCreate("test" + i)
.setSettings(settingsBuilder()
.put("index.number_of_shards", 5).put("index.number_of_replicas", 1)
.put("index.unassigned.node_left.delayed_timeout", randomIntBetween(250, 1000) + "ms"))
.execute().actionGet();
}
ensureGreen(TimeValue.timeValueMinutes(1));
logger.info("--> stopping node1");
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node_1));
// This might run slowly on older hardware
ensureGreen(TimeValue.timeValueMinutes(2));
}
private void rerouteWithAllocateLocalGateway(Settings commonSettings) throws Exception {
logger.info("--> starting 2 nodes");
String node_1 = internalCluster().startNode(commonSettings);
@ -174,23 +211,23 @@ public class ClusterRerouteIT extends ESIntegTestCase {
.execute().actionGet();
ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(2));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(2));
logger.info("--> explicitly allocate shard 1, actually allocating, no dry run");
state = client().admin().cluster().prepareReroute()
.setExplain(randomBoolean())
.add(new AllocateAllocationCommand(new ShardId("test", 0), node_1, true))
.execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
assertThat(healthResponse.isTimedOut(), equalTo(false));
logger.info("--> get the state, verify shard 1 primary allocated");
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet();
@ -215,16 +252,16 @@ public class ClusterRerouteIT extends ESIntegTestCase {
.setExplain(randomBoolean())
.add(new AllocateAllocationCommand(new ShardId("test", 0), node_1, true))
.execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
healthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForYellowStatus().execute().actionGet();
assertThat(healthResponse.isTimedOut(), equalTo(false));
logger.info("--> get the state, verify shard 1 primary allocated");
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(1));
assertThat(state.routingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).id()).get(0).state(), equalTo(ShardRoutingState.STARTED));
}

View File

@ -50,8 +50,8 @@ public class SimpleAllocationIT extends ESIntegTestCase {
ensureGreen();
ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.routingNodes()) {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
assertThat(node.size(), equalTo(2));
}
@ -60,8 +60,8 @@ public class SimpleAllocationIT extends ESIntegTestCase {
ensureGreen();
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.routingNodes()) {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
assertThat(node.size(), equalTo(1));
}
@ -75,8 +75,8 @@ public class SimpleAllocationIT extends ESIntegTestCase {
ensureGreen();
state = client().admin().cluster().prepareState().execute().actionGet().getState();
assertThat(state.routingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.routingNodes()) {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(0));
for (RoutingNode node : state.getRoutingNodes()) {
if (!node.isEmpty()) {
assertThat(node.size(), equalTo(4));
}

View File

@ -76,7 +76,7 @@ public class DelayedAllocationIT extends ESIntegTestCase {
assertBusy(new Runnable() {
@Override
public void run() {
assertThat(client().admin().cluster().prepareState().all().get().getState().routingNodes().hasUnassigned(), equalTo(true));
assertThat(client().admin().cluster().prepareState().all().get().getState().getRoutingNodes().hasUnassigned(), equalTo(true));
}
});
assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1));
@ -124,7 +124,7 @@ public class DelayedAllocationIT extends ESIntegTestCase {
assertBusy(new Runnable() {
@Override
public void run() {
assertThat(client().admin().cluster().prepareState().all().get().getState().routingNodes().hasUnassigned(), equalTo(true));
assertThat(client().admin().cluster().prepareState().all().get().getState().getRoutingNodes().hasUnassigned(), equalTo(true));
}
});
assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1));
@ -151,7 +151,7 @@ public class DelayedAllocationIT extends ESIntegTestCase {
assertBusy(new Runnable() {
@Override
public void run() {
assertThat(client().admin().cluster().prepareState().all().get().getState().routingNodes().hasUnassigned(), equalTo(true));
assertThat(client().admin().cluster().prepareState().all().get().getState().getRoutingNodes().hasUnassigned(), equalTo(true));
}
});
assertThat(client().admin().cluster().prepareHealth().get().getDelayedUnassignedShards(), equalTo(1));

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
@ -75,10 +76,10 @@ public class RoutingServiceTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).localNodeId("node1").masterNodeId("node1")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
ClusterState prevState = clusterState;
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
@ -104,14 +105,18 @@ public class RoutingServiceTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).localNodeId("node1").masterNodeId("node1")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
ClusterState prevState = clusterState;
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// We need to update the routing service's last attempted run to
// signal that the GatewayAllocator tried to allocated it but
// it was delayed
routingService.setUnassignedShardsAllocatedTimestamp(System.currentTimeMillis());
ClusterState newState = clusterState;
routingService.clusterChanged(new ClusterChangedEvent("test", newState, prevState));
@ -125,6 +130,44 @@ public class RoutingServiceTests extends ESAllocationTestCase {
assertThat(routingService.getRegisteredNextDelaySetting(), equalTo(Long.MAX_VALUE));
}
@Test
public void testDelayedUnassignedDoesNotRerouteForNegativeDelays() throws Exception {
AllocationService allocation = createAllocationService();
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT).put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "100ms"))
.numberOfShards(1).numberOfReplicas(1))
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsNew(metaData.index("test"))).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).localNodeId("node1").masterNodeId("node1")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
ClusterState prevState = clusterState;
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// Set it in the future so the delay will be negative
routingService.setUnassignedShardsAllocatedTimestamp(System.currentTimeMillis() + TimeValue.timeValueMinutes(1).millis());
ClusterState newState = clusterState;
routingService.clusterChanged(new ClusterChangedEvent("test", newState, prevState));
assertBusy(new Runnable() {
@Override
public void run() {
assertThat(routingService.hasReroutedAndClear(), equalTo(false));
// verify the registration has been updated
assertThat(routingService.getRegisteredNextDelaySetting(), equalTo(100L));
}
});
}
private class TestRoutingService extends RoutingService {
private AtomicBoolean rerouted = new AtomicBoolean();

View File

@ -95,7 +95,7 @@ public class RoutingTableTest extends ESAllocationTestCase {
private void startInitializingShards(String index) {
this.clusterState = ClusterState.builder(clusterState).routingTable(this.testRoutingTable).build();
logger.info("start primary shards for index " + index);
RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.applyStartedShards(this.clusterState, this.clusterState.routingNodes().shardsWithState(index, INITIALIZING));
RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.applyStartedShards(this.clusterState, this.clusterState.getRoutingNodes().shardsWithState(index, INITIALIZING));
this.clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
this.testRoutingTable = rerouteResult.routingTable();
}

View File

@ -90,7 +90,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsNew(metaData.index("test"))).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.INDEX_CREATED));
}
}
@ -103,7 +103,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsRecovery(metaData.index("test"))).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.CLUSTER_RECOVERED));
}
}
@ -116,7 +116,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsFromCloseToOpen(metaData.index("test"))).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.INDEX_REOPENED));
}
}
@ -129,7 +129,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsNewRestore(metaData.index("test"), new RestoreSource(new SnapshotId("rep1", "snp1"), Version.CURRENT, "test"), new IntHashSet())).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NEW_INDEX_RESTORED));
}
}
@ -142,7 +142,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsRestore(metaData.index("test"), new RestoreSource(new SnapshotId("rep1", "snp1"), Version.CURRENT, "test"))).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED));
}
}
@ -155,7 +155,7 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT)
.metaData(metaData)
.routingTable(RoutingTable.builder().addAsFromDangling(metaData.index("test"))).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
assertThat(shard.unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.DANGLING_INDEX_IMPORTED));
}
}
@ -172,16 +172,16 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1"))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
IndexRoutingTable.Builder builder = IndexRoutingTable.builder("test");
for (IndexShardRoutingTable indexShardRoutingTable : clusterState.routingTable().index("test")) {
builder.addIndexShard(indexShardRoutingTable);
}
builder.addReplica();
clusterState = ClusterState.builder(clusterState).routingTable(RoutingTable.builder(clusterState.routingTable()).add(builder)).build();
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.REPLICA_ADDED));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.REPLICA_ADDED));
}
/**
@ -215,19 +215,19 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// verify that NODE_LEAVE is the reason for meta
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(true));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NODE_LEFT));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getTimestampInMillis(), greaterThan(0l));
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(true));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NODE_LEFT));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getTimestampInMillis(), greaterThan(0l));
}
/**
@ -245,21 +245,21 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// fail shard
ShardRouting shardToFail = clusterState.routingNodes().shardsWithState(STARTED).get(0);
ShardRouting shardToFail = clusterState.getRoutingNodes().shardsWithState(STARTED).get(0);
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyFailedShards(clusterState, ImmutableList.of(new FailedRerouteAllocation.FailedShard(shardToFail, "test fail", null)))).build();
// verify the reason and details
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(true));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getMessage(), equalTo("test fail"));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getDetails(), equalTo("test fail"));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getTimestampInMillis(), greaterThan(0l));
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(true));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo(), notNullValue());
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getMessage(), equalTo("test fail"));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getDetails(), equalTo("test fail"));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).get(0).unassignedInfo().getTimestampInMillis(), greaterThan(0l));
}
/**
@ -273,7 +273,8 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
assertBusy(new Runnable() {
@Override
public void run() {
long delay = unassignedInfo.getDelayAllocationExpirationIn(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), Settings.EMPTY);
long delay = unassignedInfo.getDelayAllocationExpirationIn(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), Settings.EMPTY);
assertThat(delay, greaterThan(0l));
assertThat(delay, lessThan(TimeValue.timeValueHours(10).millis()));
}
@ -290,7 +291,8 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
UnassignedInfo unassignedInfo = new UnassignedInfo(RandomPicks.randomFrom(getRandom(), reasons), null);
long delay = unassignedInfo.getAllocationDelayTimeoutSetting(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), Settings.EMPTY);
assertThat(delay, equalTo(0l));
delay = unassignedInfo.getDelayAllocationExpirationIn(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), Settings.EMPTY);
delay = unassignedInfo.getDelayAllocationExpirationIn(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), Settings.EMPTY);
assertThat(delay, equalTo(0l));
}
@ -306,16 +308,18 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.routingTable(RoutingTable.builder().addAsNew(metaData.index("test1")).addAsNew(metaData.index("test2"))).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(0));
assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(0));
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
assertThat(clusterState.prettyPrint(), UnassignedInfo.getNumberOfDelayedUnassigned(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(2));
assertThat(clusterState.prettyPrint(), UnassignedInfo.getNumberOfDelayedUnassigned(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(2));
}
@Test
@ -330,12 +334,13 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
.routingTable(RoutingTable.builder().addAsNew(metaData.index("test1")).addAsNew(metaData.index("test2"))).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2"))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(0));
assertThat(UnassignedInfo.getNumberOfDelayedUnassigned(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState), equalTo(0));
// starting primaries
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
// starting replicas
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.routingNodes().hasUnassigned(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().hasUnassigned(), equalTo(false));
// remove node2 and reroute
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node2")).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState)).build();
@ -343,7 +348,8 @@ public class UnassignedInfoTests extends ESAllocationTestCase {
long nextDelaySetting = UnassignedInfo.findSmallestDelayedAllocationSetting(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState);
assertThat(nextDelaySetting, equalTo(TimeValue.timeValueHours(10).millis()));
long nextDelay = UnassignedInfo.findNextDelayedAllocationIn(Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState);
long nextDelay = UnassignedInfo.findNextDelayedAllocationIn(System.currentTimeMillis(),
Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, "10h").build(), clusterState);
assertThat(nextDelay, greaterThan(TimeValue.timeValueHours(9).millis()));
assertThat(nextDelay, lessThanOrEqualTo(TimeValue.timeValueHours(10).millis()));
}

View File

@ -56,13 +56,13 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
AllocationService service = createAllocationService(settings.build());
ClusterState clusterState = initCluster(service, 1, 3, 3, 1);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(9));
int nodeOffset = 1;
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(0));
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(3));
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
@ -73,23 +73,23 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
clusterState = addIndex(clusterState, service, 3, 2, 3);
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(2));
assertNumIndexShardsPerNode(clusterState, "test3", Matchers.equalTo(2));
assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2));
clusterState = addIndex(clusterState, service, 4, 2, 3);
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(4));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(4));
assertNumIndexShardsPerNode(clusterState, "test4", Matchers.equalTo(2));
assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2));
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(0));
clusterState = removeNodes(clusterState, service, 1);
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(4));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(4));
assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2));
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertNumIndexShardsPerNode(clusterState, Matchers.lessThanOrEqualTo(2));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(0));
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
}
@ -101,13 +101,13 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
AllocationService service = createAllocationService(settings.build());
ClusterState clusterState = initCluster(service, 1, 3, 3, 1);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(9));
int nodeOffset = 1;
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(0));
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(3));
logger.info("now, start one more node, check that rebalancing will happen because we set it to always");
@ -117,47 +117,46 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
RoutingTable routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
RoutingTable prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(4));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(4));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(6));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(6));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
assertThat(prev, Matchers.sameInstance(routingTable));
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
@ -172,13 +171,13 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
AllocationService service = createAllocationService(settings.build());
ClusterState clusterState = initCluster(service, 1, 3, 3, 1);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(9));
int nodeOffset = 1;
clusterState = addNodes(clusterState, service, 1, nodeOffset++);
assertThat(clusterState.routingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.routingNodes().unassigned().size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), Matchers.equalTo(9));
assertThat(clusterState.getRoutingNodes().unassigned().size(), Matchers.equalTo(0));
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(3));
logger.info("now, start one more node, check that rebalancing will happen because we set it to always");
@ -188,47 +187,47 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
RoutingTable routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
RoutingTable prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(4));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(4));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(6));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
routingNodes = clusterState.getRoutingNodes();
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), Matchers.equalTo(6));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node0").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), Matchers.equalTo(0));
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.sameInstance(routingTable));
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
@ -242,7 +241,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
}
private void assertNumIndexShardsPerNode(ClusterState state, String index, Matcher<Integer> matcher) {
for (RoutingNode node : state.routingNodes()) {
for (RoutingNode node : state.getRoutingNodes()) {
assertThat(node.shardsWithState(index, STARTED).size(), matcher);
}
}
@ -251,7 +250,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
private void assertAtLeastOneIndexShardPerNode(ClusterState state) {
for (String index : state.routingTable().indicesRouting().keySet()) {
for (RoutingNode node : state.routingNodes()) {
for (RoutingNode node : state.getRoutingNodes()) {
assertThat(node.shardsWithState(index, STARTED).size(), Matchers.greaterThanOrEqualTo(1));
}
}
@ -269,7 +268,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
RoutingTable routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
// move initializing to started
@ -278,7 +277,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -314,19 +313,19 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).nodes(nodes).metaData(metaData).routingTable(routingTable).build();
routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
@ -334,7 +333,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -359,19 +358,19 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
@ -379,7 +378,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -402,22 +401,22 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
}
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("start all the primary shards, replicas will start initializing");
RoutingTable routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("rebalancing");
routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
@ -425,7 +424,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;

View File

@ -72,7 +72,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("start primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("move the shard");
@ -86,15 +86,15 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand(new ShardId("test", 0), existingNodeId, toNodeId)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(existingNodeId).get(0).state(), equalTo(ShardRoutingState.RELOCATING));
assertThat(clusterState.routingNodes().node(toNodeId).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(existingNodeId).get(0).state(), equalTo(ShardRoutingState.RELOCATING));
assertThat(clusterState.getRoutingNodes().node(toNodeId).get(0).state(), equalTo(ShardRoutingState.INITIALIZING));
logger.info("finish moving the shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(existingNodeId).isEmpty(), equalTo(true));
assertThat(clusterState.routingNodes().node(toNodeId).get(0).state(), equalTo(ShardRoutingState.STARTED));
assertThat(clusterState.getRoutingNodes().node(existingNodeId).isEmpty(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node(toNodeId).get(0).state(), equalTo(ShardRoutingState.STARTED));
}
@Test
@ -122,7 +122,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> allocating with primary flag set to false, should fail");
try {
@ -142,16 +142,16 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node1", true)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
logger.info("--> start the primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
logger.info("--> allocate the replica shard on the primary shard node, should fail");
try {
@ -164,19 +164,19 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node2", false)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> verify that we fail when there are no unassigned shards");
try {
@ -210,15 +210,15 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> allocating with primary flag set to true");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node1", true)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
logger.info("--> cancel primary allocation, make sure it fails...");
try {
@ -228,11 +228,11 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
}
logger.info("--> start the primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
logger.info("--> cancel primary allocation, make sure it fails...");
try {
@ -245,28 +245,28 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node2", false)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> cancel the relocation allocation");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand(new ShardId("test", 0), "node2", false)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node2", false)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> cancel the primary being replicated, make sure it fails");
try {
@ -276,64 +276,64 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
}
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> cancel allocation of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand(new ShardId("test", 0), "node2", false)));
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateAllocationCommand(new ShardId("test", 0), "node2", false)));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> move the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand(new ShardId("test", 0), "node2", "node3")));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> cancel the move of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand(new ShardId("test", 0), "node3", false)));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).size(), equalTo(1));
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand(new ShardId("test", 0), "node1", true)));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.routingNodes().node("node2").shardsWithState(STARTED).get(0).primary(), equalTo(true));
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).get(0).primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
}
@Test

View File

@ -71,17 +71,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -90,15 +90,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -110,7 +110,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@Test
@ -141,17 +141,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -160,15 +160,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node4"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node4"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -180,7 +180,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@Test
@ -216,30 +216,30 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(INITIALIZING)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(INITIALIZING)) {
logger.info(shard.toString());
}
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(STARTED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(STARTED)) {
logger.info(shard.toString());
}
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(RELOCATING)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(RELOCATING)) {
logger.info(shard.toString());
}
for (ShardRouting shard : clusterState.routingNodes().shardsWithState(UNASSIGNED)) {
for (ShardRouting shard : clusterState.getRoutingNodes().shardsWithState(UNASSIGNED)) {
logger.info(shard.toString());
}
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -248,20 +248,20 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(5));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(5));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(5));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
logger.info("--> complete initializing");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> run it again, since we still might have relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -272,13 +272,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -315,17 +315,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -334,21 +334,21 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
logger.info("--> complete initializing");
for (int i = 0; i < 2; i++) {
logger.info("--> complete initializing round: [{}]", i);
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(5));
@ -362,15 +362,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
logger.info("--> complete relocation");
for (int i = 0; i < 2; i++) {
logger.info("--> complete initializing round: [{}]", i);
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node4").size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(5));
@ -407,17 +407,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -426,15 +426,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -445,15 +445,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node4"));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node4"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
logger.info("--> make sure another reroute does not move things");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -488,17 +488,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -507,15 +507,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node5"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node5"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -526,15 +526,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node6"));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node6"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
logger.info("--> make sure another reroute does not move things");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -568,15 +568,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> replica will not start because we have only one rack value");
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -585,15 +585,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -605,7 +605,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@Test
@ -637,15 +637,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> replica will not start because we have only one rack value");
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -654,15 +654,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node4"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node4"));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -674,7 +674,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@Test
@ -712,13 +712,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
logger.info("--> add a new node with a new rack and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -727,19 +727,19 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
logger.info("--> complete initializing");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> run it again, since we still might have relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -750,13 +750,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
logger.info("--> do another reroute, make sure nothing moves");
assertThat(strategy.reroute(clusterState).routingTable(), sameInstance(clusterState.routingTable()));
@ -792,20 +792,20 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> all replicas are allocated and started since we have on node in each zone");
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> add a new node in zone 'a' and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -813,15 +813,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(8));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("A-1"));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo("A-1"));
logger.info("--> starting initializing shards on the new node");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().node("A-1").size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("A-0").size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("B-0").size(), equalTo(5));

View File

@ -21,7 +21,6 @@ package org.elasticsearch.cluster.routing.allocation;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterInfoService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.EmptyClusterInfoService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -133,26 +132,26 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).nodes(nodes).metaData(metaData).routingTable(routingTable).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
while (true) {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -169,7 +168,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
RoutingTable routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
// move initializing to started
@ -177,7 +176,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
while (true) {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -195,29 +194,29 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
}
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("start all the primary shards, replicas will start initializing");
RoutingTable routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("rebalancing");
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
while (true) {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
@ -428,7 +427,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).nodes(nodes).metaData(metaData).routingTable(routingTable).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {
for (ShardRouting shardRouting : routingNode) {
@ -438,10 +437,10 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
strategy = createAllocationService(settings.build());
logger.info("use the new allocator and check if it moves shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {
for (ShardRouting shardRouting : routingNode) {
assertThat(shardRouting.state(), Matchers.equalTo(ShardRoutingState.STARTED));
@ -451,7 +450,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
logger.info("start the replica shards");
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {
for (ShardRouting shardRouting : routingNode) {
@ -462,7 +461,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
logger.info("rebalancing");
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {
for (ShardRouting shardRouting : routingNode) {

View File

@ -77,11 +77,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -96,11 +96,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -121,7 +121,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
assertThat(routingNodes.node("node3").get(0).shardId().index().name(), equalTo("test1"));
@ -164,11 +164,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
@ -183,11 +183,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
@ -202,11 +202,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
@ -227,7 +227,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
assertThat(routingNodes.node("node3").get(0).shardId().index().name(), equalTo("test1"));
@ -269,11 +269,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -288,11 +288,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -313,7 +313,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
}
@ -354,11 +354,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -373,11 +373,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -392,11 +392,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -411,11 +411,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test2 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -436,7 +436,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
assertThat(routingNodes.node("node3").get(0).shardId().index().name(), anyOf(equalTo("test1"), equalTo("test2")));
@ -478,11 +478,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -497,11 +497,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -522,7 +522,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
}
@ -563,11 +563,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -582,11 +582,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -601,11 +601,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
@ -626,7 +626,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
}

View File

@ -81,11 +81,11 @@ public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
@ -100,7 +100,7 @@ public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
@ -109,44 +109,44 @@ public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
}
logger.info("start the replica shards, rebalancing should start, but, only 3 should be rebalancing");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(7));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(3));
logger.info("finalize this session relocation, 3 more should relocate now");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(7));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(3));
logger.info("finalize this session relocation, 2 more should relocate now");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(8));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(2));
logger.info("finalize this session relocation, no more relocation");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));

View File

@ -69,17 +69,17 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
logger.info("--> fail node with primary");
String nodeIdToFail = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
@ -91,8 +91,8 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(nodeIdRemaining).get(0).primary(), equalTo(true));
assertThat(clusterState.routingNodes().node(nodeIdRemaining).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).get(0).primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).get(0).state(), equalTo(STARTED));
}
@Test
@ -121,17 +121,17 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
logger.info("--> adding additional node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -140,11 +140,11 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
String origPrimaryNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
String origReplicaNodeId = clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId();
@ -155,8 +155,8 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.routingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
logger.info("--> fail primary shard recovering instance on node3 being initialized by killing node3");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -166,8 +166,8 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
}
@Test
@ -196,17 +196,17 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
logger.info("--> adding additional node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -215,11 +215,11 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
String origPrimaryNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
String origReplicaNodeId = clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId();
@ -230,8 +230,8 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.routingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
logger.info("--> fail primary shard recovering instance on 'origPrimaryNodeId' being relocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -241,7 +241,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
}
}

View File

@ -68,7 +68,7 @@ public class DisableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
}
@ -97,13 +97,13 @@ public class DisableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
}
@Test
@ -130,17 +130,17 @@ public class DisableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> verify only enabled index has been routed");
assertThat(clusterState.readOnlyRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
}
}

View File

@ -67,17 +67,17 @@ public class ElectReplicaAsPrimaryDuringRelocationTests extends ESAllocationTest
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));

View File

@ -66,18 +66,18 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(1));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(1));
@ -95,7 +95,7 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {
assertThat(routingNode.numberOfShardsWithState(STARTED), equalTo(1));

View File

@ -73,17 +73,17 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING));
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
logger.info("--> adding additional node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -92,11 +92,11 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.routingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
String origPrimaryNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
String origReplicaNodeId = clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId();
@ -107,15 +107,15 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.routingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
logger.info("--> fail primary shard recovering instance on node3 being initialized");
rerouteResult = allocation.applyFailedShard(clusterState, new ShardRouting(clusterState.routingNodes().node("node3").get(0)));
rerouteResult = allocation.applyFailedShard(clusterState, new ShardRouting(clusterState.getRoutingNodes().node("node3").get(0)));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingNodes().node("node3").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
logger.info("--> moving primary shard to node3");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(
@ -123,15 +123,15 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
assertThat(clusterState.routingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.routingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0).state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").get(0).state(), equalTo(INITIALIZING));
logger.info("--> fail primary shard recovering instance on node1 being relocated");
rerouteResult = allocation.applyFailedShard(clusterState, new ShardRouting(clusterState.routingNodes().node(origPrimaryNodeId).get(0)));
rerouteResult = allocation.applyFailedShard(clusterState, new ShardRouting(clusterState.getRoutingNodes().node(origPrimaryNodeId).get(0)));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
// check promotion of replica to primary
assertThat(clusterState.routingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(origReplicaNodeId));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), anyOf(equalTo(origPrimaryNodeId), equalTo("node3")));
}
@ -162,7 +162,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the shards (primaries)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
@ -181,7 +181,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
}
logger.info("Start the shards (backups)");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -203,7 +203,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.applyFailedShard(clusterState, shardToFail).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
@ -256,7 +256,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
logger.info("fail the first shard, will have no place to be rerouted to (single node), so stays unassigned");
prevRoutingTable = routingTable;
ShardRouting firstShard = clusterState.routingNodes().node("node1").get(0);
ShardRouting firstShard = clusterState.getRoutingNodes().node("node1").get(0);
routingTable = strategy.applyFailedShard(clusterState, firstShard).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -313,7 +313,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
int shardsToFail = randomIntBetween(1, numberOfReplicas);
ArrayList<FailedRerouteAllocation.FailedShard> failedShards = new ArrayList<>();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < shardsToFail; i++) {
String n = "node" + Integer.toString(randomInt(numberOfReplicas));
logger.info("failing shard on node [{}]", n);
@ -324,7 +324,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
routingTable = strategy.applyFailedShards(clusterState, failedShards).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (FailedRerouteAllocation.FailedShard failedShard : failedShards) {
if (!routingNodes.node(failedShard.shard.currentNodeId()).isEmpty()) {
fail("shard " + failedShard + " was re-assigned to it's node");
@ -371,7 +371,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
logger.info("fail the first shard, will start INITIALIZING on the second node");
prevRoutingTable = routingTable;
final ShardRouting firstShard = clusterState.routingNodes().node(nodeHoldingPrimary).get(0);
final ShardRouting firstShard = clusterState.getRoutingNodes().node(nodeHoldingPrimary).get(0);
routingTable = strategy.applyFailedShard(clusterState, firstShard).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
@ -419,7 +419,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the shards (primaries)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -437,7 +437,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
}
logger.info("Start the shards (backups)");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -459,7 +459,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
@ -472,11 +472,11 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
logger.info("Fail the shards on node 3");
ShardRouting shardToFail = routingNodes.node("node3").get(0);
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyFailedShard(clusterState, new ShardRouting(shardToFail)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
@ -507,12 +507,12 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
// add 4 nodes
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3")).put(newNode("node4"))).build();
clusterState = ClusterState.builder(clusterState).routingTable(allocation.reroute(clusterState).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
// start primary shards
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
// fail the primary shard, check replicas get removed as well...
ShardRouting primaryShardToFail = clusterState.routingTable().index("test").shard(0).primaryShard();
@ -520,16 +520,16 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
assertThat(routingResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(routingResult.routingTable()).build();
// the primary gets allocated on another node, replicas are unassigned
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
ShardRouting newPrimaryShard = clusterState.routingTable().index("test").shard(0).primaryShard();
assertThat(newPrimaryShard, not(equalTo(primaryShardToFail)));
// start the primary shard
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
// simulate another failure coming in, with the "old" shard routing, verify that nothing changes, and we ignore it
routingResult = allocation.applyFailedShard(clusterState, primaryShardToFail);
@ -554,25 +554,25 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
// add 4 nodes
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3")).put(newNode("node4"))).build();
clusterState = ClusterState.builder(clusterState).routingTable(allocation.reroute(clusterState).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
// start primary shards
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
// start another replica shard, while keep one initializing
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, ImmutableList.of(clusterState.routingNodes().shardsWithState(INITIALIZING).get(0))).routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, ImmutableList.of(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0))).routingTable()).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
// fail the primary shard, check one replica gets elected to primary, others become INITIALIZING (from it)
ShardRouting primaryShardToFail = clusterState.routingTable().index("test").shard(0).primaryShard();
RoutingAllocation.Result routingResult = allocation.applyFailedShard(clusterState, primaryShardToFail);
assertThat(routingResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(routingResult.routingTable()).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
ShardRouting newPrimaryShard = clusterState.routingTable().index("test").shard(0).primaryShard();
assertThat(newPrimaryShard, not(equalTo(primaryShardToFail)));

View File

@ -74,18 +74,18 @@ public class FilterRoutingTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> make sure shards are only allocated on tag1 with value1 and value2");
List<ShardRouting> startedShards = clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED);
List<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
assertThat(startedShards.size(), equalTo(4));
for (ShardRouting startedShard : startedShards) {
assertThat(startedShard.currentNodeId(), Matchers.anyOf(equalTo("node1"), equalTo("node2")));
@ -123,18 +123,18 @@ public class FilterRoutingTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> make sure shards are only allocated on tag1 with value1 and value2");
List<ShardRouting> startedShards = clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED);
List<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
assertThat(startedShards.size(), equalTo(4));
for (ShardRouting startedShard : startedShards) {
assertThat(startedShard.currentNodeId(), Matchers.anyOf(equalTo("node1"), equalTo("node2")));
@ -153,14 +153,14 @@ public class FilterRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).metaData(metaData).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(2));
logger.info("--> finish relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
startedShards = clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED);
startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
assertThat(startedShards.size(), equalTo(4));
for (ShardRouting startedShard : startedShards) {
assertThat(startedShard.currentNodeId(), Matchers.anyOf(equalTo("node1"), equalTo("node4")));

View File

@ -108,7 +108,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -132,11 +132,11 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the more shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
@ -150,7 +150,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
@ -240,7 +240,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the primary shard");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -264,11 +264,11 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
@ -282,7 +282,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
@ -309,18 +309,18 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
@ -391,7 +391,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -415,11 +415,11 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the more shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
@ -431,7 +431,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
@ -483,7 +483,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -507,11 +507,11 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the more shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
@ -523,7 +523,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));

View File

@ -97,11 +97,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
@ -110,11 +110,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
}
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
@ -129,7 +129,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
@ -145,7 +145,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
@ -154,11 +154,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
}
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
@ -287,11 +287,11 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
}
private ClusterState stabilize(ClusterState clusterState, AllocationService service) {
logger.trace("RoutingNodes: {}", clusterState.routingNodes().prettyPrint());
logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint());
RoutingTable routingTable = service.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertRecoveryNodeVersions(routingNodes);
logger.info("complete rebalancing");
@ -301,7 +301,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
if (stable = (routingTable == prev)) {
break;
}

View File

@ -71,8 +71,8 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
while (!clusterState.routingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
@ -95,8 +95,8 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("[{}] primaries should be still started but [{}] other primaries should be unassigned", numberOfShards, numberOfShards);
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(numberOfShards));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(numberOfShards));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(numberOfShards));
logger.info("start node back up");
@ -105,17 +105,17 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
while (clusterState.routingNodes().shardsWithState(STARTED).size() < totalNumberOfShards) {
while (clusterState.getRoutingNodes().shardsWithState(STARTED).size() < totalNumberOfShards) {
int localInitializations = 0;
int relocatingInitializations = 0;
for (ShardRouting routing : clusterState.routingNodes().shardsWithState(INITIALIZING)) {
for (ShardRouting routing : clusterState.getRoutingNodes().shardsWithState(INITIALIZING)) {
if (routing.relocatingNodeId() == null) {
localInitializations++;
} else {
relocatingInitializations++;
}
}
int needToInitialize = totalNumberOfShards - clusterState.routingNodes().shardsWithState(STARTED).size() - clusterState.routingNodes().shardsWithState(RELOCATING).size();
int needToInitialize = totalNumberOfShards - clusterState.getRoutingNodes().shardsWithState(STARTED).size() - clusterState.getRoutingNodes().shardsWithState(RELOCATING).size();
logger.info("local initializations: [{}], relocating: [{}], need to initialize: {}", localInitializations, relocatingInitializations, needToInitialize);
assertThat(localInitializations, equalTo(Math.min(primaryRecoveries, needToInitialize)));
clusterState = startRandomInitializingShard(clusterState, strategy);

View File

@ -67,8 +67,8 @@ public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
while (!clusterState.routingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
@ -81,7 +81,7 @@ public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("2 replicas should be initializing now for the existing indices (we throttle to 1)");
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("create a new index");
metaData = MetaData.builder(clusterState.metaData())

View File

@ -67,13 +67,13 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the backup shard (on node2)");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -83,7 +83,7 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
@ -117,10 +117,10 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
rerouteResult = allocation.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(2));
@ -134,7 +134,7 @@ public class PrimaryElectionRoutingTests extends ESAllocationTestCase {
).build();
rerouteResult = allocation.reroute(clusterState);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingNodes.shardsWithState(INITIALIZING).size(), equalTo(1));

View File

@ -69,7 +69,7 @@ public class PrimaryNotRelocatedWhileBeingRecoveredTests extends ESAllocationTes
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

View File

@ -20,7 +20,6 @@
package org.elasticsearch.cluster.routing.allocation;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterInfoService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.EmptyClusterInfoService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -111,8 +110,8 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
clusterState = stateBuilder.build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
if (clusterState.routingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))
if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
@ -137,20 +136,20 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
iterations++;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
if (clusterState.routingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING))
if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
} while (clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 ||
clusterState.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size() != 0 && iterations < 200);
} while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 ||
clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size() != 0 && iterations < 200);
logger.info("Done Balancing after [{}] iterations", iterations);
// we stop after 200 iterations if it didn't stabelize by then something is likely to be wrong
assertThat("max num iteration exceeded", iterations, Matchers.lessThan(200));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0));
int shards = clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0));
int shards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size();
assertThat(shards, equalTo(totalNumShards));
final int numNodes = clusterState.nodes().size();
final int upperBound = (int) Math.round(((shards / numNodes) * 1.10));

View File

@ -87,11 +87,11 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
@ -106,7 +106,7 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
@ -115,33 +115,33 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
}
logger.info("start the replica shards, rebalancing should start");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(5));
logger.info("complete relocation, other half of relocation should happen");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
// we now only relocate 3, since 2 remain where they are!
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(7));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(3));
logger.info("complete relocation, thats it!");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));
// make sure we have an even relocation

View File

@ -86,7 +86,7 @@ public class ReplicaAllocatedAfterPrimaryTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), nullValue());
logger.info("Start all the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node(nodeHoldingPrimary).shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

View File

@ -61,12 +61,12 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
logger.info("Adding three node and performing rerouting");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3"))).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
// all shards are unassigned. so no inactive shards or primaries.
@ -77,7 +77,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -90,22 +90,22 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
logger.info("Start the more shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));
@ -114,7 +114,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
}
@ -150,7 +150,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the primary shard");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -160,15 +160,15 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState)
@ -182,18 +182,18 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
@ -231,7 +231,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3"))).build();
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));
assertThat(routingNodes.hasInactivePrimaries(), equalTo(false));
@ -240,7 +240,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -255,7 +255,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node1").numberOfShardsWithState(INITIALIZING), equalTo(1));
assertThat(routingNodes.node("node2").numberOfShardsWithState(INITIALIZING), equalTo(1));
assertThat(routingNodes.node("node3").numberOfShardsWithState(INITIALIZING), equalTo(1));
@ -263,7 +263,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -279,11 +279,11 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the more shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));
@ -311,7 +311,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
.addAsNew(metaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));
@ -329,7 +329,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -339,11 +339,11 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Reroute, start the primaries");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -351,11 +351,11 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false));
logger.info("Reroute, start the replicas");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));
@ -376,7 +376,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove(indexShardRoutingTable.primaryShard().currentNodeId())).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -385,11 +385,11 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false));
logger.info("Start Recovering shards round 1");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(true));
@ -397,11 +397,11 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false));
logger.info("Start Recovering shards round 2");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
assertThat(routingNodes.hasInactiveShards(), equalTo(false));

View File

@ -67,14 +67,14 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2));
logger.info("--> start all primary shards, no replica will be started since its on the same host");
routingTable = strategy.applyStartedShards(clusterState, clusterState.readOnlyRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(0));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(0));
logger.info("--> add another node, with a different host, replicas will be allocating");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -82,9 +82,9 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2));
for (ShardRouting shardRouting : clusterState.readOnlyRoutingNodes().shardsWithState(INITIALIZING)) {
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2));
for (ShardRouting shardRouting : clusterState.getRoutingNodes().shardsWithState(INITIALIZING)) {
assertThat(shardRouting.currentNodeId(), equalTo("node3"));
}
}

View File

@ -78,11 +78,11 @@ public class ShardVersioningTests extends ESAllocationTestCase {
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));

View File

@ -68,22 +68,22 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(2));
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0));
assertThat(clusterState.readOnlyRoutingNodes().unassigned().size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0));
assertThat(clusterState.getRoutingNodes().unassigned().size(), equalTo(4));
logger.info("Do another reroute, make sure its still not allocated");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
}
@ -119,11 +119,11 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), STARTED), equalTo(5));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(5));
logger.info("add another index with 5 shards");
metaData = MetaData.builder(metaData)
@ -143,16 +143,16 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(numberOfShardsOfType(clusterState.readOnlyRoutingNodes(), STARTED), equalTo(10));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(10));
for (ShardRouting shardRouting : clusterState.readOnlyRoutingNodes().node("node1")) {
for (ShardRouting shardRouting : clusterState.getRoutingNodes().node("node1")) {
assertThat(shardRouting.index(), equalTo("test"));
}
for (ShardRouting shardRouting : clusterState.readOnlyRoutingNodes().node("node2")) {
for (ShardRouting shardRouting : clusterState.getRoutingNodes().node("node2")) {
assertThat(shardRouting.index(), equalTo("test1"));
}
@ -172,17 +172,17 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(3));
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(RELOCATING), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(RELOCATING), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(RELOCATING), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(RELOCATING), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(3));
// the first move will destroy the balance and the balancer will move 2 shards from node2 to node one right after
// moving the nodes to node2 since we consider INITIALIZING nodes during rebalance
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// now we are done compared to EvenShardCountAllocator since the Balancer is not soely based on the average
assertThat(clusterState.readOnlyRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(5));
assertThat(clusterState.readOnlyRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(5));
}
}

View File

@ -94,7 +94,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Marking the shard as started");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -141,7 +141,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(routingTable == prevRoutingTable, equalTo(true));
logger.info("Start the shard on node 2");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -191,7 +191,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
logger.info("Marking the shard as failed");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyFailedShard(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING).get(0)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -261,7 +261,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
int nodeIndex = Integer.parseInt(nodeId.substring("node".length()));
assertThat(nodeIndex, lessThan(25));
}
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
Set<String> encounteredIndices = newHashSet();
for (RoutingNode routingNode : routingNodes) {
assertThat(routingNode.numberOfShardsWithState(STARTED), equalTo(0));
@ -359,7 +359,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), equalTo(INITIALIZING));
}
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(numberOfShardsOfType(routingNodes, INITIALIZING), equalTo(numberOfIndices));
assertThat(routingNodes.node("node1").numberOfShardsWithState(INITIALIZING), anyOf(equalTo(3), equalTo(4)));
assertThat(routingNodes.node("node2").numberOfShardsWithState(INITIALIZING), anyOf(equalTo(3), equalTo(4)));
@ -376,7 +376,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -388,7 +388,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(RELOCATING), equalTo(STARTED)));
}
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat("4 source shard routing are relocating", numberOfShardsOfType(routingNodes, RELOCATING), equalTo(4));
assertThat("4 target shard routing are initializing", numberOfShardsOfType(routingNodes, INITIALIZING), equalTo(4));
@ -405,7 +405,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(RELOCATING), equalTo(STARTED)));
}
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(numberOfShardsOfType(routingNodes, STARTED), equalTo(numberOfIndices));
for (RoutingNode routingNode : routingNodes) {
assertThat(routingNode.numberOfShardsWithState(STARTED), equalTo(2));

View File

@ -93,7 +93,7 @@ public class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -116,7 +116,7 @@ public class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

View File

@ -105,7 +105,7 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -129,11 +129,11 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
assertThat(prevRoutingTable == routingTable, equalTo(true));
logger.info("Start the backup shard");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
@ -154,7 +154,7 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
@ -165,11 +165,11 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
assertThat(routingNodes.node("node3").numberOfShardsWithState(INITIALIZING), equalTo(6));
logger.info("Start the shards on node 3");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node3").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));

View File

@ -75,13 +75,13 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start all the primary shards");
RoutingNodes routingNodes = clusterState.routingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("Start all the replica shards");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -100,7 +100,7 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
logger.info("add another replica");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = RoutingTable.builder(routingTable).updateNumberOfReplicas(2).build();
metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(2).build();
@ -135,7 +135,7 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
@ -151,7 +151,7 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).get(1).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
logger.info("now remove a replica");
routingNodes = clusterState.routingNodes();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = RoutingTable.builder(routingTable).updateNumberOfReplicas(1).build();
metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(1).build();

View File

@ -119,25 +119,25 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Primary shard should be initializing, replica should not
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that we're able to start the primary
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
// Assert that node1 didn't get any shards because its disk usage is too high
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that the replica couldn't be started since node1 doesn't have enough space
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
logger.info("--> adding node3");
@ -149,16 +149,16 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Assert that the replica is initialized now that node3 is available with enough space
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that the replica couldn't be started since node1 doesn't have enough space
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
@ -189,7 +189,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
@ -220,7 +220,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
// Shard hasn't been moved off of node2 yet because there's nowhere for it to go
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
@ -236,11 +236,11 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
@ -315,7 +315,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Primary should initialize, even though both nodes are over the limit initialize
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
String nodeWithPrimary, nodeWithoutPrimary;
if (clusterState.getRoutingNodes().node("node1").size() == 1) {
@ -354,15 +354,15 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Now the replica should be able to initialize
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that we're able to start the primary and replica, since they were both initializing
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
// Assert that node1 got a single shard (the primary), even though its disk usage is too high
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
// Assert that node2 got a single shard (a replica)
@ -381,16 +381,16 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Assert that the replica is initialized now that node3 is available with enough space
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that all replicas could be started
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
@ -421,7 +421,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(1));
@ -452,7 +452,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
// Shard hasn't been moved off of node2 yet because there's nowhere for it to go
assertThat(clusterState.getRoutingNodes().node("node2").size(), equalTo(1));
@ -468,13 +468,13 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
// One shard is relocating off of node1
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
@ -495,14 +495,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// Shards remain started on node3 and node4
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
// One shard is relocating off of node2 now
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(1));
// Initializing on node5
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> final cluster state:");
@ -572,14 +572,14 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Shard can't be allocated to node1 (or node2) because it would cause too much usage
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
// No shards are started, no nodes have enough disk for allocation
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(0));
}
@Test
@ -641,17 +641,17 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Shard can be allocated to node1, even though it only has 25% free,
// because it's a primary that's never been allocated before
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// A single shard is started on node1, even though it normally would not
// be allowed, because it's a primary that hasn't been allocated, and node1
// is still below the high watermark (unlike node3)
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
}
@ -748,15 +748,15 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logShardStates(clusterState);
// shards should be initializing
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
logger.info("--> start the shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logShardStates(clusterState);
// Assert that we're able to start the primary and replicas
assertThat(clusterState.routingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
logger.info("--> adding node3");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
@ -898,7 +898,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
}
public void logShardStates(ClusterState state) {
RoutingNodes rn = state.routingNodes();
RoutingNodes rn = state.getRoutingNodes();
logger.info("--> counts: total: {}, unassigned: {}, initializing: {}, relocating: {}, started: {}",
rn.shards(new Predicate<ShardRouting>() {
@Override

View File

@ -81,7 +81,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
}
@ -110,13 +110,13 @@ public class EnableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
}
@Test
@ -145,17 +145,17 @@ public class EnableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
logger.info("--> verify only enabled index has been routed");
assertThat(clusterState.readOnlyRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
assertThat(clusterState.readOnlyRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState("disabled", STARTED).size(), equalTo(0));
}
@ -193,17 +193,17 @@ public class EnableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(4));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> adding one nodes and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -214,8 +214,8 @@ public class EnableAllocationTests extends ESAllocationTestCase {
ClusterState prevState = clusterState;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
if (useClusterSetting) {
prevState = clusterState;
@ -234,9 +234,9 @@ public class EnableAllocationTests extends ESAllocationTestCase {
nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat("expected 6 shards to be started 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat("expected 2 shards to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
List<ShardRouting> mutableShardRoutings = clusterState.routingNodes().shardsWithState(RELOCATING);
assertThat("expected 6 shards to be started 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat("expected 2 shards to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
List<ShardRouting> mutableShardRoutings = clusterState.getRoutingNodes().shardsWithState(RELOCATING);
switch (allowedOnes) {
case PRIMARIES:
for (ShardRouting routing : mutableShardRoutings) {
@ -258,10 +258,10 @@ public class EnableAllocationTests extends ESAllocationTestCase {
default:
fail("only replicas, primaries or all are allowed");
}
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
}
@ -294,12 +294,12 @@ public class EnableAllocationTests extends ESAllocationTestCase {
).build();
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(6));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(6));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.routingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> adding one nodes and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -310,8 +310,8 @@ public class EnableAllocationTests extends ESAllocationTestCase {
ClusterState prevState = clusterState;
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
if (useClusterSetting) {
prevState = clusterState;
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(settingsBuilder()
@ -326,8 +326,8 @@ public class EnableAllocationTests extends ESAllocationTestCase {
nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat("expected 4 primaries to be started and 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(STARTED).size(), equalTo(4));
assertThat("expected 2 primaries to relocate useClusterSettings: " + useClusterSetting, clusterState.routingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
assertThat("expected 4 primaries to be started and 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4));
assertThat("expected 2 primaries to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(2));
}

View File

@ -238,10 +238,10 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// after all are started, check routing iteration
@ -288,7 +288,7 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
ShardsIterator shardsIterator = clusterState.routingTable().index("test").shard(0).onlyNodeSelectorActiveInitializingShardsIt("disk:ebs",clusterState.nodes());
@ -344,10 +344,10 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
OperationRouting operationRouting = new OperationRouting(Settings.Builder.EMPTY_SETTINGS, new AwarenessAllocationDecider());
@ -409,7 +409,7 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
routingTable = strategy.reroute(clusterState).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
// When replicas haven't initialized, it comes back with the primary first, then initializing replicas
@ -431,10 +431,10 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
assertFalse(routing.primary());
assertTrue(routing.initializing());
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.routingNodes().shardsWithState(INITIALIZING)).routingTable();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();

View File

@ -73,5 +73,10 @@ public class ScriptScoreFunctionTests extends ESTestCase {
}
};
}
@Override
public boolean needsScores() {
return false;
}
}
}

View File

@ -187,7 +187,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.getRoutingNodes(), state.nodes(), null);
testAllocator.addData(node1, -1).addData(node2, -1);
boolean changed = testAllocator.allocateUnassigned(allocation);
@ -212,7 +212,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -220,7 +220,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -228,7 +228,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0));
@ -254,7 +254,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
boolean changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -262,7 +262,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node1, 1);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(false));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1));
@ -270,7 +270,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas
testAllocator.addData(node2, 2);
allocation = new RoutingAllocation(yesAllocationDeciders(), state.routingNodes(), state.nodes(), null);
allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state.nodes(), null);
changed = testAllocator.allocateUnassigned(allocation);
assertThat(changed, equalTo(true));
assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0));
@ -344,7 +344,7 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase {
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), null);
}
class TestAllocator extends PrimaryShardAllocator {

View File

@ -302,7 +302,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), null);
}
private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders) {
@ -321,7 +321,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
.metaData(metaData)
.routingTable(routingTable)
.nodes(DiscoveryNodes.builder().put(node1).put(node2).put(node3)).build();
return new RoutingAllocation(deciders, state.routingNodes(), state.nodes(), null);
return new RoutingAllocation(deciders, new RoutingNodes(state, false), state.nodes(), null);
}
class TestAllocator extends ReplicaShardAllocator {

View File

@ -1031,9 +1031,6 @@ public class GetActionIT extends ESIntegTestCase {
" \"doc\": {\n" +
" \"_timestamp\": {\n" +
" \"enabled\": true\n" +
" },\n" +
" \"_size\": {\n" +
" \"enabled\": true\n" +
" }\n" +
" }\n" +
" }\n" +
@ -1045,7 +1042,7 @@ public class GetActionIT extends ESIntegTestCase {
" \"text\": \"some text.\"\n" +
"}\n";
client().prepareIndex("test", "doc").setId("1").setSource(doc).setRouting("1").get();
String[] fieldsList = {"_timestamp", "_size", "_routing"};
String[] fieldsList = {"_timestamp", "_routing"};
// before refresh - document is only in translog
assertGetFieldsAlwaysWorks(indexOrAlias(), "doc", "1", fieldsList, "1");
refresh();

View File

@ -133,7 +133,7 @@ public class TransportIndexFailuresIT extends ESIntegTestCase {
client().prepareGet(INDEX, "doc", resp.getId()).get().isExists(), equalTo(true));
state = getNodeClusterState(randomFrom(nodes.toArray(Strings.EMPTY_ARRAY)));
RoutingNodes rn = state.routingNodes();
RoutingNodes rn = state.getRoutingNodes();
logger.info("--> counts: total: {}, unassigned: {}, initializing: {}, relocating: {}, started: {}",
rn.shards(new Predicate<ShardRouting>() {
@Override

View File

@ -44,8 +44,8 @@ import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import org.elasticsearch.index.mapper.internal.SizeFieldMapper;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.index.mapper.internal.TimestampFieldMapper;
import org.hamcrest.Matchers;
import org.junit.Test;
@ -387,7 +387,7 @@ public class SimpleAllMapperTests extends ESSingleNodeTestCase {
String mapping = "{";
Map<String, String> rootTypes = new HashMap<>();
//just pick some example from DocumentMapperParser.rootTypeParsers
rootTypes.put(SizeFieldMapper.NAME, "{\"enabled\" : true}");
rootTypes.put(TimestampFieldMapper.NAME, "{\"enabled\" : true}");
rootTypes.put("include_in_all", "true");
rootTypes.put("dynamic_date_formats", "[\"yyyy-MM-dd\", \"dd-MM-yyyy\"]");
rootTypes.put("numeric_detection", "true");

View File

@ -33,16 +33,16 @@ public class ParseMappingTypeLevelTests extends ESSingleNodeTestCase {
@Test
public void testTypeLevel() throws Exception {
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("_size").field("enabled", true).endObject()
.startObject("_timestamp").field("enabled", true).endObject()
.endObject().endObject().string();
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper mapper = parser.parse("type", mapping);
assertThat(mapper.type(), equalTo("type"));
assertThat(mapper.sizeFieldMapper().enabled(), equalTo(true));
assertThat(mapper.timestampFieldMapper().enabled(), equalTo(true));
mapper = parser.parse(mapping);
assertThat(mapper.type(), equalTo("type"));
assertThat(mapper.sizeFieldMapper().enabled(), equalTo(true));
assertThat(mapper.timestampFieldMapper().enabled(), equalTo(true));
}
}

View File

@ -158,25 +158,6 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
assertTrue(documentMapper.timestampFieldMapper().fieldType().stored());
}
@Test
public void testSizeParsing() throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().build());
XContentBuilder indexMapping = XContentFactory.jsonBuilder();
boolean enabled = randomBoolean();
indexMapping.startObject()
.startObject("type")
.startObject("_size")
.field("enabled", enabled)
.endObject()
.endObject()
.endObject();
DocumentMapper documentMapper = indexService.mapperService().parse("type", new CompressedXContent(indexMapping.string()), true);
assertThat(documentMapper.sizeFieldMapper().enabled(), equalTo(enabled));
assertTrue(documentMapper.sizeFieldMapper().fieldType().stored());
documentMapper = indexService.mapperService().parse("type", new CompressedXContent(documentMapper.mappingSource().string()), true);
assertThat(documentMapper.sizeFieldMapper().enabled(), equalTo(enabled));
}
@Test
public void testSizeTimestampIndexParsing() throws IOException {
IndexService indexService = createIndex("test", Settings.settingsBuilder().build());
@ -192,7 +173,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
createIndex("test1", Settings.settingsBuilder().build());
createIndex("test2", Settings.settingsBuilder().build());
XContentBuilder defaultMapping = XContentFactory.jsonBuilder().startObject()
.startObject(MapperService.DEFAULT_MAPPING).startObject("_size").field("enabled", true).endObject().endObject()
.startObject(MapperService.DEFAULT_MAPPING).startObject("_timestamp").field("enabled", true).endObject().endObject()
.endObject();
client().admin().indices().preparePutMapping().setType(MapperService.DEFAULT_MAPPING).setSource(defaultMapping).get();
XContentBuilder typeMapping = XContentFactory.jsonBuilder().startObject()
@ -204,7 +185,7 @@ public class UpdateMappingTests extends ESSingleNodeTestCase {
GetMappingsResponse response = client().admin().indices().prepareGetMappings("test2").get();
assertNotNull(response.getMappings().get("test2").get("type").getSourceAsMap().get("_all"));
assertFalse((Boolean) ((LinkedHashMap) response.getMappings().get("test2").get("type").getSourceAsMap().get("_all")).get("enabled"));
assertNotNull(response.getMappings().get("test2").get("type").getSourceAsMap().get("_size"));
assertTrue((Boolean)((LinkedHashMap)response.getMappings().get("test2").get("type").getSourceAsMap().get("_size")).get("enabled"));
assertNotNull(response.getMappings().get("test2").get("type").getSourceAsMap().get("_timestamp"));
assertTrue((Boolean)((LinkedHashMap)response.getMappings().get("test2").get("type").getSourceAsMap().get("_timestamp")).get("enabled"));
}
}

View File

@ -1 +1 @@
{"type":{"_size":{"enabled":false},"_timestamp":{"enabled":false}}}
{"type":{"_timestamp":{"enabled":false}}}

View File

@ -0,0 +1,52 @@
/*
* 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.query;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.junit.Test;
import java.io.IOException;
public class CommonTermsQueryParserTest extends ESSingleNodeTestCase {
@Test
public void testWhenParsedQueryIsNullNoNullPointerExceptionIsThrown() throws IOException {
final String index = "test-index";
final String type = "test-type";
client()
.admin()
.indices()
.prepareCreate(index)
.addMapping(type, "name", "type=string,analyzer=stop")
.execute()
.actionGet();
ensureGreen();
CommonTermsQueryBuilder commonTermsQueryBuilder =
new CommonTermsQueryBuilder("name", "the").queryName("query-name");
// the named query parses to null; we are testing this does not cause a NullPointerException
SearchResponse response =
client().prepareSearch(index).setTypes(type).setQuery(commonTermsQueryBuilder).execute().actionGet();
assertNotNull(response);
assertEquals(response.getHits().hits().length, 0);
}
}

View File

@ -54,6 +54,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
@ -83,6 +84,7 @@ import static org.hamcrest.Matchers.*;
public class SimpleIndexQueryParserTests extends ESSingleNodeTestCase {
private IndexQueryParserService queryParser;
private IndexService indexService;
@Before
public void setup() throws IOException {
@ -99,6 +101,7 @@ public class SimpleIndexQueryParserTests extends ESSingleNodeTestCase {
assertNotNull(doc.dynamicMappingsUpdate());
client().admin().indices().preparePutMapping("test").setType("person").setSource(doc.dynamicMappingsUpdate().toString()).get();
this.indexService = indexService;
queryParser = indexService.queryParserService();
}
@ -2264,6 +2267,23 @@ public class SimpleIndexQueryParserTests extends ESSingleNodeTestCase {
assertThat(parsedQuery, instanceOf(BooleanQuery.class));
}
public void testCrossFieldMultiMatchQuery() throws IOException {
IndexQueryParserService queryParser = queryParser();
Query parsedQuery = queryParser.parse(multiMatchQuery("banon", "name.first^2", "name.last^3", "foobar").type(MultiMatchQueryBuilder.Type.CROSS_FIELDS)).query();
try (Engine.Searcher searcher = indexService.shardSafe(0).acquireSearcher("test")) {
Query rewrittenQuery = searcher.searcher().rewrite(parsedQuery);
BooleanQuery expected = new BooleanQuery();
expected.add(new TermQuery(new Term("foobar", "banon")), Occur.SHOULD);
TermQuery tq1 = new TermQuery(new Term("name.first", "banon"));
tq1.setBoost(2);
TermQuery tq2 = new TermQuery(new Term("name.last", "banon"));
tq2.setBoost(3);
expected.add(new DisjunctionMaxQuery(Arrays.<Query>asList(tq1, tq2), 0f), Occur.SHOULD);
assertEquals(expected, rewrittenQuery);
}
}
@Test
public void testSimpleQueryString() throws Exception {
IndexQueryParserService queryParser = queryParser();

View File

@ -431,7 +431,7 @@ public class CorruptedFileIT extends ESIntegTestCase {
// we are green so primaries got not corrupted.
// ensure that no shard is actually allocated on the unlucky node
ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().get();
for (IndexShardRoutingTable table : clusterStateResponse.getState().routingNodes().getRoutingTable().index("test")) {
for (IndexShardRoutingTable table : clusterStateResponse.getState().getRoutingNodes().getRoutingTable().index("test")) {
for (ShardRouting routing : table) {
if (unluckyNode.getNode().getId().equals(routing.currentNodeId())) {
assertThat(routing.state(), not(equalTo(ShardRoutingState.STARTED)));

Some files were not shown because too many files have changed in this diff Show More