Merge branch 'master' into feature/rank-eval

This commit is contained in:
Isabel Drost-Fromm 2016-08-24 12:44:37 +02:00
commit e5fd0cb3a2
135 changed files with 3088 additions and 2854 deletions

4
Vagrantfile vendored
View File

@ -78,8 +78,8 @@ Vagrant.configure(2) do |config|
config.vm.synced_folder ".", "/vagrant", disabled: true
config.vm.synced_folder ".", "/elasticsearch"
config.vm.provider "virtualbox" do |v|
# Give the boxes 2GB so they can run our tests if they have to.
v.memory = 2048
# Give the boxes 3GB because Elasticsearch defaults to using 2GB
v.memory = 3072
end
if Vagrant.has_plugin?("vagrant-cachier")
config.cache.scope = :box

View File

@ -57,6 +57,9 @@ dependencies {
}
compileJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-try,-unchecked"
// enable the JMH's BenchmarkProcessor to generate the final benchmark classes
// needs to be added separately otherwise Gradle will quote it and javac will fail
compileJava.options.compilerArgs.addAll(["-processor", "org.openjdk.jmh.generators.BenchmarkProcessor"])
compileTestJava.options.compilerArgs << "-Xlint:-cast,-deprecation,-rawtypes,-try,-unchecked"
forbiddenApis {

View File

@ -394,7 +394,13 @@ class BuildPlugin implements Plugin<Project> {
* -serial because we don't use java serialization.
*/
// don't even think about passing args with -J-xxx, oracle will ask you to submit a bug report :)
options.compilerArgs << '-Werror' << '-proc:none' << '-Xlint:all,-path,-serial,-options,-deprecation' << '-Xdoclint:all' << '-Xdoclint:-missing'
options.compilerArgs << '-Werror' << '-Xlint:all,-path,-serial,-options,-deprecation' << '-Xdoclint:all' << '-Xdoclint:-missing'
// either disable annotation processor completely (default) or allow to enable them if an annotation processor is explicitly defined
if (options.compilerArgs.contains("-processor") == false) {
options.compilerArgs << '-proc:none'
}
options.encoding = 'UTF-8'
//options.incremental = true

View File

@ -54,7 +54,9 @@ class ClusterConfiguration {
boolean debug = false
@Input
String jvmArgs = System.getProperty('tests.jvm.argline', '')
String jvmArgs = "-Xms" + System.getProperty('tests.heap.size', '512m') +
" " + "-Xmx" + System.getProperty('tests.heap.size', '512m') +
" " + System.getProperty('tests.jvm.argline', '')
/**
* The seed nodes port file. In the case the cluster has more than one node we use a seed node

View File

@ -38,6 +38,8 @@ public class SnapshotsStatusRequest extends MasterNodeRequest<SnapshotsStatusReq
private String[] snapshots = Strings.EMPTY_ARRAY;
private boolean ignoreUnavailable;
public SnapshotsStatusRequest() {
}
@ -112,11 +114,33 @@ public class SnapshotsStatusRequest extends MasterNodeRequest<SnapshotsStatusReq
return this;
}
/**
* Set to <code>true</code> to ignore unavailable snapshots, instead of throwing an exception.
* Defaults to <code>false</code>, which means unavailable snapshots cause an exception to be thrown.
*
* @param ignoreUnavailable whether to ignore unavailable snapshots
* @return this request
*/
public SnapshotsStatusRequest ignoreUnavailable(boolean ignoreUnavailable) {
this.ignoreUnavailable = ignoreUnavailable;
return this;
}
/**
* Returns whether the request permits unavailable snapshots to be ignored.
*
* @return true if the request will ignore unavailable snapshots, false if it will throw an exception on unavailable snapshots
*/
public boolean ignoreUnavailable() {
return ignoreUnavailable;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
repository = in.readString();
snapshots = in.readStringArray();
ignoreUnavailable = in.readBoolean();
}
@Override
@ -124,5 +148,6 @@ public class SnapshotsStatusRequest extends MasterNodeRequest<SnapshotsStatusReq
super.writeTo(out);
out.writeString(repository);
out.writeStringArray(snapshots);
out.writeBoolean(ignoreUnavailable);
}
}

View File

@ -74,4 +74,16 @@ public class SnapshotsStatusRequestBuilder extends MasterNodeOperationRequestBui
request.snapshots(ArrayUtils.concat(request.snapshots(), snapshots));
return this;
}
/**
* Set to <code>true</code> to ignore unavailable snapshots, instead of throwing an exception.
* Defaults to <code>false</code>, which means unavailable snapshots cause an exception to be thrown.
*
* @param ignoreUnavailable whether to ignore unavailable snapshots.
* @return this builder
*/
public SnapshotsStatusRequestBuilder setIgnoreUnavailable(boolean ignoreUnavailable) {
request.ignoreUnavailable(ignoreUnavailable);
return this;
}
}

View File

@ -214,7 +214,14 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
SnapshotId snapshotId = matchedSnapshotIds.get(snapshotName);
if (snapshotId == null) {
// neither in the current snapshot entries nor found in the repository
throw new SnapshotMissingException(repositoryName, snapshotName);
if (request.ignoreUnavailable()) {
// ignoring unavailable snapshots, so skip over
logger.debug("snapshot status request ignoring snapshot [{}], not found in repository [{}]",
snapshotName, repositoryName);
continue;
} else {
throw new SnapshotMissingException(repositoryName, snapshotName);
}
}
SnapshotInfo snapshotInfo = snapshotsService.snapshot(repositoryName, snapshotId);
List<SnapshotIndexShardStatus> shardStatusBuilder = new ArrayList<>();

View File

@ -273,7 +273,7 @@ final class Bootstrap {
// guice: log the shortened exc to the log file
ByteArrayOutputStream os = new ByteArrayOutputStream();
PrintStream ps = new PrintStream(os, false, "UTF-8");
new StartupError(e).printStackTrace(ps);
new StartupException(e).printStackTrace(ps);
ps.flush();
logger.error("Guice Exception: {}", os.toString("UTF-8"));
} else {

View File

@ -166,7 +166,6 @@ final class BootstrapCheck {
if (Constants.LINUX || Constants.MAC_OS_X) {
checks.add(new MaxSizeVirtualMemoryCheck());
}
checks.add(new MinMasterNodesCheck(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(settings)));
if (Constants.LINUX) {
checks.add(new MaxMapCountCheck());
}
@ -330,32 +329,6 @@ final class BootstrapCheck {
}
static class MinMasterNodesCheck implements Check {
final boolean minMasterNodesIsSet;
MinMasterNodesCheck(boolean minMasterNodesIsSet) {
this.minMasterNodesIsSet = minMasterNodesIsSet;
}
@Override
public boolean check() {
return minMasterNodesIsSet == false;
}
@Override
public String errorMessage() {
return "please set [" + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() +
"] to a majority of the number of master eligible nodes in your cluster";
}
@Override
public final boolean isSystemCheck() {
return false;
}
}
static class MaxNumberOfThreadsCheck implements Check {
private final long maxNumberOfThreadsThreshold = 1 << 11;

View File

@ -23,7 +23,6 @@ import joptsimple.OptionSet;
import joptsimple.OptionSpec;
import joptsimple.OptionSpecBuilder;
import joptsimple.util.PathConverter;
import joptsimple.util.PathProperties;
import org.elasticsearch.Build;
import org.elasticsearch.cli.ExitCodes;
import org.elasticsearch.cli.SettingCommand;
@ -102,7 +101,7 @@ class Elasticsearch extends SettingCommand {
} catch (final Throwable t) {
// format exceptions to the console in a special way
// to avoid 2MB stacktraces from guice, etc.
throw new StartupError(t);
throw new StartupException(t);
}
}

View File

@ -23,6 +23,8 @@ import org.elasticsearch.common.inject.CreationException;
import org.elasticsearch.common.inject.spi.Message;
import java.io.PrintStream;
import java.io.PrintWriter;
import java.util.function.Consumer;
/**
* Wraps an exception in a special way that it gets formatted
@ -32,18 +34,18 @@ import java.io.PrintStream;
*/
//TODO: remove this when guice is removed, and exceptions are cleaned up
//this is horrible, but its what we must do
final class StartupError extends RuntimeException {
final class StartupException extends RuntimeException {
/** maximum length of a stacktrace, before we truncate it */
static final int STACKTRACE_LIMIT = 30;
/** all lines from this package are RLE-compressed */
static final String GUICE_PACKAGE = "org.elasticsearch.common.inject";
/**
* Create a new StartupError that will format {@code cause}
/**
* Create a new StartupException that will format {@code cause}
* to the console on failure.
*/
StartupError(Throwable cause) {
StartupException(Throwable cause) {
super(cause);
}
@ -53,15 +55,24 @@ final class StartupError extends RuntimeException {
*/
@Override
public void printStackTrace(PrintStream s) {
printStackTrace(s::println);
}
@Override
public void printStackTrace(PrintWriter s) {
printStackTrace(s::println);
}
private void printStackTrace(Consumer<String> consumer) {
Throwable originalCause = getCause();
Throwable cause = originalCause;
if (cause instanceof CreationException) {
cause = getFirstGuiceCause((CreationException)cause);
}
String message = cause.toString();
s.println(message);
consumer.accept(message);
if (cause != null) {
// walk to the root cause
while (cause.getCause() != null) {
@ -70,7 +81,7 @@ final class StartupError extends RuntimeException {
// print the root cause message, only if it differs!
if (cause != originalCause && (message.equals(cause.toString()) == false)) {
s.println("Likely root cause: " + cause);
consumer.accept("Likely root cause: " + cause);
}
// print stacktrace of cause
@ -78,33 +89,33 @@ final class StartupError extends RuntimeException {
int linesWritten = 0;
for (int i = 0; i < stack.length; i++) {
if (linesWritten == STACKTRACE_LIMIT) {
s.println("\t<<<truncated>>>");
consumer.accept("\t<<<truncated>>>");
break;
}
String line = stack[i].toString();
// skip past contiguous runs of this garbage:
if (line.startsWith(GUICE_PACKAGE)) {
while (i + 1 < stack.length && stack[i + 1].toString().startsWith(GUICE_PACKAGE)) {
i++;
}
s.println("\tat <<<guice>>>");
consumer.accept("\tat <<<guice>>>");
linesWritten++;
continue;
}
s.println("\tat " + line.toString());
consumer.accept("\tat " + line.toString());
linesWritten++;
}
}
// if its a guice exception, the whole thing really will not be in the log, its megabytes.
// refer to the hack in bootstrap, where we don't log it
if (originalCause instanceof CreationException == false) {
s.println("Refer to the log for complete error details.");
consumer.accept("Refer to the log for complete error details.");
}
}
/**
/**
* Returns first cause from a guice error (it can have multiple).
*/
static Throwable getFirstGuiceCause(CreationException guice) {
@ -116,4 +127,5 @@ final class StartupError extends RuntimeException {
}
return guice; // we tried
}
}

View File

@ -69,25 +69,25 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.Function;
import java.util.function.Supplier;
/**
* Configures classes and services that affect the entire cluster.
*/
public class ClusterModule extends AbstractModule {
public static final String EVEN_SHARD_COUNT_ALLOCATOR = "even_shard";
public static final String BALANCED_ALLOCATOR = "balanced"; // default
public static final Setting<String> SHARDS_ALLOCATOR_TYPE_SETTING =
new Setting<>("cluster.routing.allocation.type", BALANCED_ALLOCATOR, Function.identity(), Property.NodeScope);
private final Settings settings;
private final ExtensionPoint.SelectedType<ShardsAllocator> shardsAllocators = new ExtensionPoint.SelectedType<>("shards_allocator", ShardsAllocator.class);
private final ExtensionPoint.ClassSet<IndexTemplateFilter> indexTemplateFilters = new ExtensionPoint.ClassSet<>("index_template_filter", IndexTemplateFilter.class);
private final ClusterService clusterService;
private final IndexNameExpressionResolver indexNameExpressionResolver;
// pkg private for tests
final Collection<AllocationDecider> allocationDeciders;
final ShardsAllocator shardsAllocator;
// pkg private so tests can mock
Class<? extends ClusterInfoService> clusterInfoServiceImpl = InternalClusterInfoService.class;
@ -95,20 +95,11 @@ public class ClusterModule extends AbstractModule {
public ClusterModule(Settings settings, ClusterService clusterService, List<ClusterPlugin> clusterPlugins) {
this.settings = settings;
this.allocationDeciders = createAllocationDeciders(settings, clusterService.getClusterSettings(), clusterPlugins);
registerShardsAllocator(ClusterModule.BALANCED_ALLOCATOR, BalancedShardsAllocator.class);
registerShardsAllocator(ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR, BalancedShardsAllocator.class);
this.shardsAllocator = createShardsAllocator(settings, clusterService.getClusterSettings(), clusterPlugins);
this.clusterService = clusterService;
indexNameExpressionResolver = new IndexNameExpressionResolver(settings);
}
public void registerShardsAllocator(String name, Class<? extends ShardsAllocator> clazz) {
shardsAllocators.registerExtension(name, clazz);
}
public void registerIndexTemplateFilter(Class<? extends IndexTemplateFilter> indexTemplateFilter) {
indexTemplateFilters.registerExtension(indexTemplateFilter);
}
public IndexNameExpressionResolver getIndexNameExpressionResolver() {
return indexNameExpressionResolver;
}
@ -148,16 +139,29 @@ public class ClusterModule extends AbstractModule {
}
}
private static ShardsAllocator createShardsAllocator(Settings settings, ClusterSettings clusterSettings,
List<ClusterPlugin> clusterPlugins) {
Map<String, Supplier<ShardsAllocator>> allocators = new HashMap<>();
allocators.put(BALANCED_ALLOCATOR, () -> new BalancedShardsAllocator(settings, clusterSettings));
for (ClusterPlugin plugin : clusterPlugins) {
plugin.getShardsAllocators(settings, clusterSettings).forEach((k, v) -> {
if (allocators.put(k, v) != null) {
throw new IllegalArgumentException("ShardsAllocator [" + k + "] already defined");
}
});
}
String allocatorName = SHARDS_ALLOCATOR_TYPE_SETTING.get(settings);
Supplier<ShardsAllocator> allocatorSupplier = allocators.get(allocatorName);
if (allocatorSupplier == null) {
throw new IllegalArgumentException("Unknown ShardsAllocator [" + allocatorName + "]");
}
return Objects.requireNonNull(allocatorSupplier.get(),
"ShardsAllocator factory for [" + allocatorName + "] returned null");
}
@Override
protected void configure() {
// bind ShardsAllocator
String shardsAllocatorType = shardsAllocators.bindType(binder(), settings, ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), ClusterModule.BALANCED_ALLOCATOR);
if (shardsAllocatorType.equals(ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR)) {
final ESLogger logger = Loggers.getLogger(getClass(), settings);
logger.warn("{} allocator has been removed in 2.0 using {} instead", ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR, ClusterModule.BALANCED_ALLOCATOR);
}
indexTemplateFilters.bind(binder());
bind(ClusterInfoService.class).to(clusterInfoServiceImpl).asEagerSingleton();
bind(GatewayAllocator.class).asEagerSingleton();
bind(AllocationService.class).asEagerSingleton();
@ -178,5 +182,6 @@ public class ClusterModule extends AbstractModule {
bind(MappingUpdatedAction.class).asEagerSingleton();
bind(TaskResultsService.class).asEagerSingleton();
bind(AllocationDeciders.class).toInstance(new AllocationDeciders(settings, allocationDeciders));
bind(ShardsAllocator.class).toInstance(shardsAllocator);
}
}

View File

@ -859,6 +859,10 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
return this;
}
public Set<String> getActiveAllocationIds(int shardId) {
return activeAllocationIds.get(shardId);
}
public long version() {
return this.version;
}

View File

@ -31,23 +31,4 @@ public interface IndexTemplateFilter {
* {@code false} otherwise.
*/
boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template);
class Compound implements IndexTemplateFilter {
private IndexTemplateFilter[] filters;
Compound(IndexTemplateFilter... filters) {
this.filters = filters;
}
@Override
public boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template) {
for (IndexTemplateFilter filter : filters) {
if (!filter.apply(request, template)) {
return false;
}
}
return true;
}
}
}

View File

@ -102,13 +102,11 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_C
public class MetaDataCreateIndexService extends AbstractComponent {
public static final int MAX_INDEX_NAME_BYTES = 255;
private static final DefaultIndexTemplateFilter DEFAULT_INDEX_TEMPLATE_FILTER = new DefaultIndexTemplateFilter();
private final ClusterService clusterService;
private final IndicesService indicesService;
private final AllocationService allocationService;
private final AliasValidator aliasValidator;
private final IndexTemplateFilter indexTemplateFilter;
private final Environment env;
private final NodeServicesProvider nodeServicesProvider;
private final IndexScopedSettings indexScopedSettings;
@ -117,8 +115,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
@Inject
public MetaDataCreateIndexService(Settings settings, ClusterService clusterService,
IndicesService indicesService, AllocationService allocationService,
AliasValidator aliasValidator,
Set<IndexTemplateFilter> indexTemplateFilters, Environment env,
AliasValidator aliasValidator, Environment env,
NodeServicesProvider nodeServicesProvider, IndexScopedSettings indexScopedSettings,
ThreadPool threadPool) {
super(settings);
@ -129,18 +126,6 @@ public class MetaDataCreateIndexService extends AbstractComponent {
this.env = env;
this.nodeServicesProvider = nodeServicesProvider;
this.indexScopedSettings = indexScopedSettings;
if (indexTemplateFilters.isEmpty()) {
this.indexTemplateFilter = DEFAULT_INDEX_TEMPLATE_FILTER;
} else {
IndexTemplateFilter[] templateFilters = new IndexTemplateFilter[indexTemplateFilters.size() + 1];
templateFilters[0] = DEFAULT_INDEX_TEMPLATE_FILTER;
int i = 1;
for (IndexTemplateFilter indexTemplateFilter : indexTemplateFilters) {
templateFilters[i++] = indexTemplateFilter;
}
this.indexTemplateFilter = new IndexTemplateFilter.Compound(templateFilters);
}
this.activeShardsObserver = new ActiveShardsObserver(settings, clusterService, threadPool);
}
@ -242,7 +227,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
// we only find a template when its an API call (a new index)
// find templates, highest order are better matching
List<IndexTemplateMetaData> templates = findTemplates(request, currentState, indexTemplateFilter);
List<IndexTemplateMetaData> templates = findTemplates(request, currentState);
Map<String, Custom> customs = new HashMap<>();
@ -470,11 +455,11 @@ public class MetaDataCreateIndexService extends AbstractComponent {
});
}
private List<IndexTemplateMetaData> findTemplates(CreateIndexClusterStateUpdateRequest request, ClusterState state, IndexTemplateFilter indexTemplateFilter) throws IOException {
private List<IndexTemplateMetaData> findTemplates(CreateIndexClusterStateUpdateRequest request, ClusterState state) throws IOException {
List<IndexTemplateMetaData> templates = new ArrayList<>();
for (ObjectCursor<IndexTemplateMetaData> cursor : state.metaData().templates().values()) {
IndexTemplateMetaData template = cursor.value;
if (indexTemplateFilter.apply(request, template)) {
if (Regex.simpleMatch(template.template(), request.index())) {
templates.add(template);
}
}
@ -525,13 +510,6 @@ public class MetaDataCreateIndexService extends AbstractComponent {
return validationErrors;
}
private static class DefaultIndexTemplateFilter implements IndexTemplateFilter {
@Override
public boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template) {
return Regex.simpleMatch(template.template(), request.index());
}
}
/**
* Validates the settings and mappings for shrinking an index.
* @return the list of nodes at least one instance of the source index shards are allocated

View File

@ -128,6 +128,11 @@ public class IndexRoutingTable extends AbstractDiffable<IndexRoutingTable> imple
throw new IllegalStateException("shard routing has an index [" + shardRouting.index() + "] that is different " +
"from the routing table");
}
if (shardRouting.active() &&
indexMetaData.activeAllocationIds(shardRouting.id()).contains(shardRouting.allocationId().getId()) == false) {
throw new IllegalStateException("active shard routing " + shardRouting + " has no corresponding entry in the " +
"in-sync allocation set " + indexMetaData.activeAllocationIds(shardRouting.id()));
}
}
}
return true;

View File

@ -23,13 +23,14 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
/**
* An allocation decider that prevents multiple instances of the same shard to
* be allocated on the same <tt>node</tt>.
*
* The {@value #SAME_HOST_SETTING} setting allows to perform a check to prevent
* The {@link #CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING} setting allows to perform a check to prevent
* allocation of multiple instances of the same shard on a single <tt>host</tt>,
* based on host name and host address. Defaults to `false`, meaning that no
* check is performed by default.
@ -44,14 +45,15 @@ public class SameShardAllocationDecider extends AllocationDecider {
public static final String NAME = "same_shard";
public static final String SAME_HOST_SETTING = "cluster.routing.allocation.same_shard.host";
public static final Setting<Boolean> CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING =
Setting.boolSetting("cluster.routing.allocation.same_shard.host", false, Setting.Property.NodeScope);
private final boolean sameHost;
public SameShardAllocationDecider(Settings settings) {
super(settings);
this.sameHost = settings.getAsBoolean(SAME_HOST_SETTING, false);
this.sameHost = CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING.get(settings);
}
@Override

View File

@ -39,6 +39,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllo
import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
@ -198,6 +199,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING,
DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING,
SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING,
SnapshotInProgressAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING,

View File

@ -30,7 +30,6 @@ public class GatewayModule extends AbstractModule {
@Override
protected void configure() {
bind(MetaStateService.class).asEagerSingleton();
bind(DanglingIndicesState.class).asEagerSingleton();
bind(GatewayService.class).asEagerSingleton();
bind(TransportNodesListGatewayMetaState.class).asEagerSingleton();

View File

@ -40,7 +40,6 @@ public class MetaStateService extends AbstractComponent {
private final NodeEnvironment nodeEnv;
@Inject
public MetaStateService(Settings settings, NodeEnvironment nodeEnv) {
super(settings);
this.nodeEnv = nodeEnv;

View File

@ -19,11 +19,15 @@
package org.elasticsearch.index.engine;
import org.apache.lucene.util.RamUsageEstimator;
import org.elasticsearch.index.translog.Translog;
/** Holds a deleted version, which just adds a timestamp to {@link VersionValue} so we know when we can expire the deletion. */
class DeleteVersionValue extends VersionValue {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(DeleteVersionValue.class);
private final long time;
public DeleteVersionValue(long version, long time, Translog.Location translogLocation) {
@ -43,6 +47,7 @@ class DeleteVersionValue extends VersionValue {
@Override
public long ramBytesUsed() {
return super.ramBytesUsed() + Long.BYTES;
Translog.Location translogLocation = translogLocation();
return BASE_RAM_BYTES_USED + (translogLocation != null ? translogLocation.ramBytesUsed() : 0);
}
}

View File

@ -591,7 +591,7 @@ public abstract class Engine implements Closeable {
the store is closed so we need to make sure we increment it here
*/
try {
return !getSearcherManager().isSearcherCurrent();
return getSearcherManager().isSearcherCurrent() == false;
} catch (IOException e) {
logger.error("failed to access searcher manager", e);
failEngine("failed to access searcher manager", e);

View File

@ -59,9 +59,8 @@ public class EngineSearcher extends Engine.Searcher {
} catch (IOException e) {
throw new IllegalStateException("Cannot close", e);
} catch (AlreadyClosedException e) {
/* this one can happen if we already closed the
* underlying store / directory and we call into the
* IndexWriter to free up pending files. */
// This means there's a bug somewhere: don't suppress it
throw new AssertionError(e);
} finally {
store.decRef();
}

View File

@ -562,8 +562,8 @@ public class InternalEngine extends Engine {
ensureOpen();
searcherManager.maybeRefreshBlocking();
} catch (AlreadyClosedException e) {
ensureOpen();
maybeFailEngine("refresh", e);
failOnTragicEvent(e);
throw e;
} catch (EngineClosedException e) {
throw e;
} catch (Exception e) {
@ -610,8 +610,8 @@ public class InternalEngine extends Engine {
indexWriter.flush();
}
} catch (AlreadyClosedException e) {
ensureOpen();
maybeFailEngine("writeIndexingBuffer", e);
failOnTragicEvent(e);
throw e;
} catch (EngineClosedException e) {
throw e;
} catch (Exception e) {
@ -835,6 +835,14 @@ public class InternalEngine extends Engine {
} finally {
store.decRef();
}
} catch (AlreadyClosedException ex) {
/* in this case we first check if the engine is still open. If so this exception is just fine
* and expected. We don't hold any locks while we block on forceMerge otherwise it would block
* closing the engine as well. If we are not closed we pass it on to failOnTragicEvent which ensures
* we are handling a tragic even exception here */
ensureOpen();
failOnTragicEvent(ex);
throw ex;
} catch (Exception e) {
try {
maybeFailEngine("force merge", e);
@ -869,26 +877,35 @@ public class InternalEngine extends Engine {
}
}
private void failOnTragicEvent(AlreadyClosedException ex) {
// if we are already closed due to some tragic exception
// we need to fail the engine. it might have already been failed before
// but we are double-checking it's failed and closed
if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) {
final Exception tragedy = indexWriter.getTragicException() instanceof Exception ?
(Exception) indexWriter.getTragicException() :
new Exception(indexWriter.getTragicException());
failEngine("already closed by tragic event on the index writer", tragedy);
} else if (translog.isOpen() == false && translog.getTragicException() != null) {
failEngine("already closed by tragic event on the translog", translog.getTragicException());
} else {
// this smells like a bug - we only expect ACE if we are in a fatal case ie. either translog or IW is closed by
// a tragic event or has closed itself. if that is not the case we are in a buggy state and raise an assertion error
throw new AssertionError("Unexpected AlreadyClosedException", ex);
}
}
@Override
protected boolean maybeFailEngine(String source, Exception e) {
boolean shouldFail = super.maybeFailEngine(source, e);
if (shouldFail) {
return true;
}
// Check for AlreadyClosedException
// Check for AlreadyClosedException -- ACE is a very special
// exception that should only be thrown in a tragic event. we pass on the checks to failOnTragicEvent which will
// throw and AssertionError if the tragic event condition is not met.
if (e instanceof AlreadyClosedException) {
// if we are already closed due to some tragic exception
// we need to fail the engine. it might have already been failed before
// but we are double-checking it's failed and closed
if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) {
final Exception tragedy = indexWriter.getTragicException() instanceof Exception ?
(Exception) indexWriter.getTragicException() :
new Exception(indexWriter.getTragicException());
failEngine("already closed by tragic event on the index writer", tragedy);
} else if (translog.isOpen() == false && translog.getTragicException() != null) {
failEngine("already closed by tragic event on the translog", translog.getTragicException());
}
failOnTragicEvent((AlreadyClosedException)e);
return true;
} else if (e != null &&
((indexWriter.isOpen() == false && indexWriter.getTragicException() == e)
@ -914,6 +931,7 @@ public class InternalEngine extends Engine {
@Override
public long getIndexBufferRAMBytesUsed() {
// We don't guard w/ readLock here, so we could throw AlreadyClosedException
return indexWriter.ramBytesUsed() + versionMap.ramBytesUsedForRefresh();
}
@ -963,8 +981,9 @@ public class InternalEngine extends Engine {
logger.trace("rollback indexWriter");
try {
indexWriter.rollback();
} catch (AlreadyClosedException e) {
// ignore
} catch (AlreadyClosedException ex) {
failOnTragicEvent(ex);
throw ex;
}
logger.trace("rollback indexWriter done");
} catch (Exception e) {

View File

@ -59,26 +59,37 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
private volatile Maps maps = new Maps();
private ReferenceManager mgr;
private ReferenceManager<?> mgr;
/** Bytes consumed for each BytesRef UID:
*
* NUM_BYTES_OBJECT_HEADER + 2*NUM_BYTES_INT + NUM_BYTES_OBJECT_REF + NUM_BYTES_ARRAY_HEADER [ + bytes.length] */
private static final int BASE_BYTES_PER_BYTESREF = RamUsageEstimator.NUM_BYTES_OBJECT_HEADER +
2*Integer.BYTES +
RamUsageEstimator.NUM_BYTES_OBJECT_REF +
RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
* In this base value, we account for the {@link BytesRef} object itself as
* well as the header of the byte[] array it holds, and some lost bytes due
* to object alignment. So consumers of this constant just have to add the
* length of the byte[] (assuming it is not shared between multiple
* instances). */
private static final long BASE_BYTES_PER_BYTESREF =
// shallow memory usage of the BytesRef object
RamUsageEstimator.shallowSizeOfInstance(BytesRef.class) +
// header of the byte[] array
RamUsageEstimator.NUM_BYTES_ARRAY_HEADER +
// with an alignment size (-XX:ObjectAlignmentInBytes) of 8 (default),
// there could be between 0 and 7 lost bytes, so we account for 3
// lost bytes on average
3;
/** Bytes used by having CHM point to a key/value:
*
* CHM.Entry:
* + NUM_BYTES_OBJECT_HEADER + 3*NUM_BYTES_OBJECT_REF + NUM_BYTES_INT
*
* CHM's pointer to CHM.Entry, double for approx load factor:
* + 2*NUM_BYTES_OBJECT_REF */
private static final int BASE_BYTES_PER_CHM_ENTRY = RamUsageEstimator.NUM_BYTES_OBJECT_HEADER +
Integer.BYTES +
5*RamUsageEstimator.NUM_BYTES_OBJECT_REF;
/** Bytes used by having CHM point to a key/value. */
private static final long BASE_BYTES_PER_CHM_ENTRY;
static {
// use the same impl as the Maps does
Map<Integer, Integer> map = ConcurrentCollections.newConcurrentMapWithAggressiveConcurrency();
map.put(0, 0);
long chmEntryShallowSize = RamUsageEstimator.shallowSizeOf(map.entrySet().iterator().next());
// assume a load factor of 50%
// for each entry, we need two object refs, one for the entry itself
// and one for the free space that is due to the fact hash tables can
// not be fully loaded
BASE_BYTES_PER_CHM_ENTRY = chmEntryShallowSize + 2 * RamUsageEstimator.NUM_BYTES_OBJECT_REF;
}
/** Tracks bytes used by current map, i.e. what is freed on refresh. For deletes, which are also added to tombstones, we only account
* for the CHM entry here, and account for BytesRef/VersionValue against the tombstones, since refresh would not clear this RAM. */
@ -88,7 +99,7 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
final AtomicLong ramBytesUsedTombstones = new AtomicLong();
/** Sync'd because we replace old mgr. */
synchronized void setManager(ReferenceManager newMgr) {
synchronized void setManager(ReferenceManager<?> newMgr) {
if (mgr != null) {
mgr.removeListener(this);
}
@ -146,7 +157,7 @@ class LiveVersionMap implements ReferenceManager.RefreshListener, Accountable {
/** Adds this uid/version to the pending adds map. */
void putUnderLock(BytesRef uid, VersionValue version) {
assert uid.bytes.length == uid.length : "Oversized _uid! UID length: " + uid.length + ", bytes length: " + uid.bytes.length;
long uidRAMBytesUsed = BASE_BYTES_PER_BYTESREF + uid.bytes.length;
final VersionValue prev = maps.current.put(uid, version);

View File

@ -191,7 +191,8 @@ public class ShadowEngine extends Engine {
ensureOpen();
searcherManager.maybeRefreshBlocking();
} catch (AlreadyClosedException e) {
ensureOpen();
// This means there's a bug somewhere: don't suppress it
throw new AssertionError(e);
} catch (EngineClosedException e) {
throw e;
} catch (Exception e) {

View File

@ -28,6 +28,8 @@ import java.util.Collections;
class VersionValue implements Accountable {
private static final long BASE_RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(VersionValue.class);
private final long version;
private final Translog.Location translogLocation;
@ -54,8 +56,7 @@ class VersionValue implements Accountable {
@Override
public long ramBytesUsed() {
return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + Long.BYTES + RamUsageEstimator.NUM_BYTES_OBJECT_REF +
(translogLocation != null ? translogLocation.size : 0);
return BASE_RAM_BYTES_USED + (translogLocation != null ? translogLocation.ramBytesUsed() : 0);
}
@Override

View File

@ -205,15 +205,21 @@ public abstract class BaseGeoPointFieldMapper extends FieldMapper implements Arr
builder.precisionStep(XContentMapValues.nodeIntegerValue(propNode));
iterator.remove();
} else if (propName.equals("geohash")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash parameter is deprecated and will be removed "
+ "in the next major release");
builder.enableGeoHash(XContentMapValues.lenientNodeBooleanValue(propNode));
iterator.remove();
} else if (propName.equals("geohash_prefix")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash_prefix parameter is deprecated and will be removed "
+ "in the next major release");
builder.geoHashPrefix(XContentMapValues.lenientNodeBooleanValue(propNode));
if (XContentMapValues.lenientNodeBooleanValue(propNode)) {
builder.enableGeoHash(true);
}
iterator.remove();
} else if (propName.equals("geohash_precision")) {
deprecationLogger.deprecated(CONTENT_TYPE + " geohash_precision parameter is deprecated and will be removed "
+ "in the next major release");
if (propNode instanceof Integer) {
builder.geoHashPrecision(XContentMapValues.nodeIntegerValue(propNode));
} else {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.index.mapper.ParseContext.Document;
@ -32,7 +33,8 @@ public class ParsedDocument {
private final Field version;
private final String uid, id, type;
private final String id, type;
private final BytesRef uid;
private final String routing;
@ -52,7 +54,7 @@ public class ParsedDocument {
this.version = version;
this.id = id;
this.type = type;
this.uid = Uid.createUid(type, id);
this.uid = Uid.createUidAsBytes(type, id);
this.routing = routing;
this.timestamp = timestamp;
this.ttl = ttl;
@ -64,7 +66,7 @@ public class ParsedDocument {
return version;
}
public String uid() {
public BytesRef uid() {
return uid;
}

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index.mapper;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.index.DocValuesType;
import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -47,6 +48,7 @@ public class VersionFieldMapper extends MetadataFieldMapper {
static {
FIELD_TYPE.setName(NAME);
FIELD_TYPE.setDocValuesType(DocValuesType.NUMERIC);
FIELD_TYPE.setIndexOptions(IndexOptions.NONE);
FIELD_TYPE.setHasDocValues(true);
FIELD_TYPE.freeze();
}

View File

@ -63,6 +63,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
/** Default for geo distance computation. */
public static final GeoDistance DEFAULT_GEO_DISTANCE = GeoDistance.DEFAULT;
/** Default for optimising query through pre computed bounding box query. */
@Deprecated
public static final String DEFAULT_OPTIMIZE_BBOX = "memory";
/**
@ -75,7 +76,9 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
.withAllDeprecated("use validation_method instead");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize")
.withAllDeprecated("use validation_method instead");
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox");
@Deprecated
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox")
.withAllDeprecated("no replacement: `optimize_bbox` is no longer supported due to recent improvements");
private static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
private static final ParseField UNIT_FIELD = new ParseField("unit");
private static final ParseField DISTANCE_FIELD = new ParseField("distance");
@ -89,7 +92,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
/** Algorithm to use for distance computation. */
private GeoDistance geoDistance = DEFAULT_GEO_DISTANCE;
/** Whether or not to use a bbox for pre-filtering. TODO change to enum? */
private String optimizeBbox = DEFAULT_OPTIMIZE_BBOX;
private String optimizeBbox = null;
/** How strict should geo coordinate validation be? */
private GeoValidationMethod validationMethod = GeoValidationMethod.DEFAULT;
@ -115,7 +118,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
distance = in.readDouble();
validationMethod = GeoValidationMethod.readFromStream(in);
center = in.readGeoPoint();
optimizeBbox = in.readString();
optimizeBbox = in.readOptionalString();
geoDistance = GeoDistance.readFromStream(in);
ignoreUnmapped = in.readBoolean();
}
@ -126,7 +129,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
out.writeDouble(distance);
validationMethod.writeTo(out);
out.writeGeoPoint(center);
out.writeString(optimizeBbox);
out.writeOptionalString(optimizeBbox);
geoDistance.writeTo(out);
out.writeBoolean(ignoreUnmapped);
}
@ -220,26 +223,20 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
* Set this to memory or indexed if before running the distance
* calculation you want to limit the candidates to hits in the
* enclosing bounding box.
* @deprecated
**/
@Deprecated
public GeoDistanceQueryBuilder optimizeBbox(String optimizeBbox) {
if (optimizeBbox == null) {
throw new IllegalArgumentException("optimizeBbox must not be null");
}
switch (optimizeBbox) {
case "none":
case "memory":
case "indexed":
break;
default:
throw new IllegalArgumentException("optimizeBbox must be one of [none, memory, indexed]");
}
this.optimizeBbox = optimizeBbox;
return this;
}
/**
* Returns whether or not to run a BoundingBox query prior to
* distance query for optimization purposes.*/
* distance query for optimization purposes.
* @deprecated
**/
@Deprecated
public String optimizeBbox() {
return this.optimizeBbox;
}
@ -303,8 +300,9 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
if (indexVersionCreated.before(Version.V_2_2_0)) {
GeoPointFieldMapperLegacy.GeoPointFieldType geoFieldType = ((GeoPointFieldMapperLegacy.GeoPointFieldType) fieldType);
IndexGeoPointFieldData indexFieldData = shardContext.getForField(fieldType);
String bboxOptimization = Strings.isEmpty(optimizeBbox) ? DEFAULT_OPTIMIZE_BBOX : optimizeBbox;
return new GeoDistanceRangeQuery(center, null, normDistance, true, false, geoDistance,
geoFieldType, indexFieldData, optimizeBbox);
geoFieldType, indexFieldData, bboxOptimization);
}
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
@ -324,7 +322,9 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
builder.startArray(fieldName).value(center.lon()).value(center.lat()).endArray();
builder.field(DISTANCE_FIELD.getPreferredName(), distance);
builder.field(DISTANCE_TYPE_FIELD.getPreferredName(), geoDistance.name().toLowerCase(Locale.ROOT));
builder.field(OPTIMIZE_BBOX_FIELD.getPreferredName(), optimizeBbox);
if (Strings.isEmpty(optimizeBbox) == false) {
builder.field(OPTIMIZE_BBOX_FIELD.getPreferredName(), optimizeBbox);
}
builder.field(VALIDATION_METHOD_FIELD.getPreferredName(), validationMethod);
builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
printBoostAndQueryName(builder);
@ -344,7 +344,7 @@ public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQue
Object vDistance = null;
DistanceUnit unit = GeoDistanceQueryBuilder.DEFAULT_DISTANCE_UNIT;
GeoDistance geoDistance = GeoDistanceQueryBuilder.DEFAULT_GEO_DISTANCE;
String optimizeBbox = GeoDistanceQueryBuilder.DEFAULT_OPTIMIZE_BBOX;
String optimizeBbox = null;
boolean coerce = GeoValidationMethod.DEFAULT_LENIENT_PARSING;
boolean ignoreMalformed = GeoValidationMethod.DEFAULT_LENIENT_PARSING;
GeoValidationMethod validationMethod = null;

View File

@ -54,6 +54,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
public static final boolean DEFAULT_INCLUDE_UPPER = true;
public static final GeoDistance DEFAULT_GEO_DISTANCE = GeoDistance.DEFAULT;
public static final DistanceUnit DEFAULT_UNIT = DistanceUnit.DEFAULT;
@Deprecated
public static final String DEFAULT_OPTIMIZE_BBOX = "memory";
/**
@ -73,7 +74,9 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
private static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
private static final ParseField NAME_FIELD = new ParseField("_name");
private static final ParseField BOOST_FIELD = new ParseField("boost");
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox");
@Deprecated
private static final ParseField OPTIMIZE_BBOX_FIELD = new ParseField("optimize_bbox")
.withAllDeprecated("no replacement: `optimize_bbox` is no longer supported due to recent improvements");
private static final ParseField COERCE_FIELD = new ParseField("coerce", "normalize")
.withAllDeprecated("use validation_method instead");
private static final ParseField IGNORE_MALFORMED_FIELD = new ParseField("ignore_malformed")
@ -96,7 +99,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
private DistanceUnit unit = DEFAULT_UNIT;
private String optimizeBbox = DEFAULT_OPTIMIZE_BBOX;
private String optimizeBbox = null;
private GeoValidationMethod validationMethod = GeoValidationMethod.DEFAULT;
@ -132,7 +135,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
includeUpper = in.readBoolean();
unit = DistanceUnit.valueOf(in.readString());
geoDistance = GeoDistance.readFromStream(in);
optimizeBbox = in.readString();
optimizeBbox = in.readOptionalString();
validationMethod = GeoValidationMethod.readFromStream(in);
ignoreUnmapped = in.readBoolean();
}
@ -147,7 +150,7 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
out.writeBoolean(includeUpper);
out.writeString(unit.name());
geoDistance.writeTo(out);;
out.writeString(optimizeBbox);
out.writeOptionalString(optimizeBbox);
validationMethod.writeTo(out);
out.writeBoolean(ignoreUnmapped);
}
@ -242,22 +245,13 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
return unit;
}
@Deprecated
public GeoDistanceRangeQueryBuilder optimizeBbox(String optimizeBbox) {
if (optimizeBbox == null) {
throw new IllegalArgumentException("optimizeBbox must not be null");
}
switch (optimizeBbox) {
case "none":
case "memory":
case "indexed":
break;
default:
throw new IllegalArgumentException("optimizeBbox must be one of [none, memory, indexed]");
}
this.optimizeBbox = optimizeBbox;
return this;
}
@Deprecated
public String optimizeBbox() {
return optimizeBbox;
}
@ -356,8 +350,9 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
if (indexVersionCreated.before(Version.V_2_2_0)) {
GeoPointFieldMapperLegacy.GeoPointFieldType geoFieldType = ((GeoPointFieldMapperLegacy.GeoPointFieldType) fieldType);
IndexGeoPointFieldData indexFieldData = context.getForField(fieldType);
String bboxOptimization = Strings.isEmpty(optimizeBbox) ? DEFAULT_OPTIMIZE_BBOX : optimizeBbox;
return new GeoDistanceRangeQuery(point, fromValue, toValue, includeLower, includeUpper, geoDistance, geoFieldType,
indexFieldData, optimizeBbox);
indexFieldData, bboxOptimization);
}
// if index created V_2_2 use (soon to be legacy) numeric encoding postings format
@ -380,7 +375,9 @@ public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistan
builder.field(INCLUDE_UPPER_FIELD.getPreferredName(), includeUpper);
builder.field(UNIT_FIELD.getPreferredName(), unit);
builder.field(DISTANCE_TYPE_FIELD.getPreferredName(), geoDistance.name().toLowerCase(Locale.ROOT));
builder.field(OPTIMIZE_BBOX_FIELD.getPreferredName(), optimizeBbox);
if (Strings.isEmpty(optimizeBbox) == false) {
builder.field(OPTIMIZE_BBOX_FIELD.getPreferredName(), optimizeBbox);
}
builder.field(VALIDATION_METHOD.getPreferredName(), validationMethod);
builder.field(IGNORE_UNMAPPED_FIELD.getPreferredName(), ignoreUnmapped);
printBoostAndQueryName(builder);

View File

@ -40,8 +40,10 @@ import org.elasticsearch.index.mapper.GeoPointFieldMapperLegacy;
import java.io.IOException;
/**
*
* Query geo_point fields by distance ranges. Used for indexes created prior to 2.2
* @deprecated
*/
@Deprecated
public class GeoDistanceRangeQuery extends Query {
private final double lat;
@ -91,7 +93,8 @@ public class GeoDistanceRangeQuery extends Query {
if ("memory".equals(optimizeBbox)) {
boundingBoxFilter = null;
} else if ("indexed".equals(optimizeBbox)) {
boundingBoxFilter = IndexedGeoBoundingBoxQuery.create(distanceBoundingCheck.topLeft(), distanceBoundingCheck.bottomRight(), fieldType);
boundingBoxFilter = IndexedGeoBoundingBoxQuery.create(distanceBoundingCheck.topLeft(),
distanceBoundingCheck.bottomRight(), fieldType);
distanceBoundingCheck = GeoDistance.ALWAYS_INSTANCE; // fine, we do the bounding box check using the filter
} else {
throw new IllegalArgumentException("type [" + optimizeBbox + "] for bounding box optimization not supported");
@ -207,7 +210,8 @@ public class GeoDistanceRangeQuery extends Query {
@Override
public String toString(String field) {
return "GeoDistanceRangeQuery(" + indexFieldData.getFieldName() + ", " + geoDistance + ", [" + inclusiveLowerPoint + " - " + inclusiveUpperPoint + "], " + lat + ", " + lon + ")";
return "GeoDistanceRangeQuery(" + indexFieldData.getFieldName() + ", " + geoDistance + ", ["
+ inclusiveLowerPoint + " - " + inclusiveUpperPoint + "], " + lat + ", " + lon + ")";
}
@Override

View File

@ -657,6 +657,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
public static class Location implements Accountable, Comparable<Location> {
private static final long RAM_BYTES_USED = RamUsageEstimator.shallowSizeOfInstance(Location.class);
public final long generation;
public final long translogLocation;
public final int size;
@ -669,7 +671,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
@Override
public long ramBytesUsed() {
return RamUsageEstimator.NUM_BYTES_OBJECT_HEADER + 2 * Long.BYTES + Integer.BYTES;
return RAM_BYTES_USED;
}
@Override

View File

@ -173,7 +173,6 @@ public class IndicesModule extends AbstractModule {
protected void configure() {
bindMapperExtension();
bind(IndicesService.class).asEagerSingleton();
bind(RecoverySettings.class).asEagerSingleton();
bind(RecoveryTargetService.class).asEagerSingleton();
bind(RecoverySource.class).asEagerSingleton();

View File

@ -168,7 +168,6 @@ public class IndicesService extends AbstractLifecycleComponent
threadPool.schedule(this.cleanInterval, ThreadPool.Names.SAME, this.cacheCleaner);
}
@Inject
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv,
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,

View File

@ -74,6 +74,7 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.gateway.GatewayModule;
import org.elasticsearch.gateway.GatewayService;
import org.elasticsearch.gateway.MetaStateService;
import org.elasticsearch.http.HttpServer;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.index.analysis.AnalysisRegistry;
@ -349,6 +350,11 @@ public class Node implements Closeable {
.flatMap(p -> p.getNamedWriteables().stream()))
.flatMap(Function.identity()).collect(Collectors.toList());
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
final MetaStateService metaStateService = new MetaStateService(settings, nodeEnvironment);
final IndicesService indicesService = new IndicesService(settings, pluginsService, nodeEnvironment,
settingsModule.getClusterSettings(), analysisModule.getAnalysisRegistry(), searchModule.getQueryParserRegistry(),
clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry,
threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, metaStateService);
client = new NodeClient(settings, threadPool);
Collection<Object> pluginComponents = pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService,
@ -374,6 +380,9 @@ public class Node implements Closeable {
b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry());
b.bind(IngestService.class).toInstance(ingestService);
b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry);
b.bind(MetaDataUpgrader.class).toInstance(metaDataUpgrader);
b.bind(MetaStateService.class).toInstance(metaStateService);
b.bind(IndicesService.class).toInstance(indicesService);
Class<? extends SearchService> searchServiceImpl = pickSearchServiceImplementation();
if (searchServiceImpl == SearchService.class) {
b.bind(SearchService.class).asEagerSingleton();
@ -381,7 +390,7 @@ public class Node implements Closeable {
b.bind(SearchService.class).to(searchServiceImpl).asEagerSingleton();
}
pluginComponents.stream().forEach(p -> b.bind((Class) p.getClass()).toInstance(p));
b.bind(MetaDataUpgrader.class).toInstance(metaDataUpgrader);
}
);
injector = modules.createInjector();

View File

@ -21,7 +21,10 @@ package org.elasticsearch.plugins;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.function.Supplier;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
@ -41,4 +44,18 @@ public interface ClusterPlugin {
default Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
return Collections.emptyList();
}
/**
* Return {@link ShardsAllocator} implementations added by this plugin.
*
* The key of the returned {@link Map} is the name of the allocator, and the value
* is a function to construct the allocator.
*
* @param settings Settings for the node
* @param clusterSettings Settings for the cluster
* @return A map of allocator implementations
*/
default Map<String, Supplier<ShardsAllocator>> getShardsAllocators(Settings settings, ClusterSettings clusterSettings) {
return Collections.emptyMap();
}
}

View File

@ -20,7 +20,6 @@
package org.elasticsearch.rest.action.admin.cluster;
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusRequest;
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
@ -54,9 +53,10 @@ public class RestSnapshotsStatusAction extends BaseRestHandler {
if (snapshots.length == 1 && "_all".equalsIgnoreCase(snapshots[0])) {
snapshots = Strings.EMPTY_ARRAY;
}
SnapshotsStatusRequest snapshotsStatusResponse = snapshotsStatusRequest(repository).snapshots(snapshots);
SnapshotsStatusRequest snapshotsStatusRequest = snapshotsStatusRequest(repository).snapshots(snapshots);
snapshotsStatusRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", snapshotsStatusRequest.ignoreUnavailable()));
snapshotsStatusResponse.masterNodeTimeout(request.paramAsTime("master_timeout", snapshotsStatusResponse.masterNodeTimeout()));
client.admin().cluster().snapshotsStatus(snapshotsStatusResponse, new RestToXContentListener<SnapshotsStatusResponse>(channel));
snapshotsStatusRequest.masterNodeTimeout(request.paramAsTime("master_timeout", snapshotsStatusRequest.masterNodeTimeout()));
client.admin().cluster().snapshotsStatus(snapshotsStatusRequest, new RestToXContentListener<>(channel));
}
}

View File

@ -32,7 +32,7 @@ import java.util.function.Function;
public class ScriptSettings {
public static final String DEFAULT_LANG = "groovy";
public static final String DEFAULT_LANG = "painless";
private static final Map<ScriptService.ScriptType, Setting<Boolean>> SCRIPT_TYPE_SETTING_MAP;
@ -59,7 +59,7 @@ public class ScriptSettings {
this.scriptLanguageSettings = Collections.unmodifiableList(scriptLanguageSettings);
this.defaultScriptLanguageSetting = new Setting<>("script.default_lang", DEFAULT_LANG, setting -> {
if (!"groovy".equals(setting) && !scriptEngineRegistry.getRegisteredLanguages().containsKey(setting)) {
if (!DEFAULT_LANG.equals(setting) && !scriptEngineRegistry.getRegisteredLanguages().containsKey(setting)) {
throw new IllegalArgumentException("unregistered default language [" + setting + "]");
}
return setting;

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search;
import com.carrotsearch.hppc.ObjectFloatHashMap;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.TopDocs;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
@ -107,9 +106,6 @@ import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
/**
*
*/
public class SearchService extends AbstractLifecycleComponent implements IndexEventListener {
// we can have 5 minutes here, since we make sure to clean with search requests and when shard/index closes
@ -233,6 +229,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public DfsSearchResult executeDfsPhase(ShardSearchRequest request) throws IOException {
final SearchContext context = createAndPutContext(request);
context.incRef();
try {
contextProcessing(context);
dfsPhase.execute(context);
@ -262,6 +259,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public QuerySearchResultProvider executeQueryPhase(ShardSearchRequest request) throws IOException {
final SearchContext context = createAndPutContext(request);
final SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
context.incRef();
try {
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
@ -295,6 +293,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public ScrollQuerySearchResult executeQueryPhase(InternalScrollSearchRequest request) {
final SearchContext context = findContext(request.id());
SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
context.incRef();
try {
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
@ -316,11 +315,13 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public QuerySearchResult executeQueryPhase(QuerySearchRequest request) {
final SearchContext context = findContext(request.id());
contextProcessing(context);
context.searcher().setAggregatedDfs(request.dfs());
IndexShard indexShard = context.indexShard();
SearchOperationListener operationListener = indexShard.getSearchOperationListener();
context.incRef();
try {
contextProcessing(context);
context.searcher().setAggregatedDfs(request.dfs());
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
queryPhase.execute(context);
@ -354,8 +355,9 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public QueryFetchSearchResult executeFetchPhase(ShardSearchRequest request) throws IOException {
final SearchContext context = createAndPutContext(request);
contextProcessing(context);
context.incRef();
try {
contextProcessing(context);
SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
@ -393,9 +395,10 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public QueryFetchSearchResult executeFetchPhase(QuerySearchRequest request) {
final SearchContext context = findContext(request.id());
contextProcessing(context);
context.searcher().setAggregatedDfs(request.dfs());
context.incRef();
try {
contextProcessing(context);
context.searcher().setAggregatedDfs(request.dfs());
SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
operationListener.onPreQueryPhase(context);
long time = System.nanoTime();
@ -433,8 +436,9 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public ScrollQueryFetchSearchResult executeFetchPhase(InternalScrollSearchRequest request) {
final SearchContext context = findContext(request.id());
contextProcessing(context);
context.incRef();
try {
contextProcessing(context);
SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
processScroll(request, context);
operationListener.onPreQueryPhase(context);
@ -473,9 +477,10 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public FetchSearchResult executeFetchPhase(ShardFetchRequest request) {
final SearchContext context = findContext(request.id());
contextProcessing(context);
final SearchOperationListener operationListener = context.indexShard().getSearchOperationListener();
context.incRef();
try {
contextProcessing(context);
if (request.lastEmittedDoc() != null) {
context.scrollContext().lastEmittedDoc = request.lastEmittedDoc();
}
@ -593,6 +598,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
public boolean freeContext(long id) {
final SearchContext context = removeContext(id);
if (context != null) {
assert context.refCount() > 0 : " refCount must be > 0: " + context.refCount();
try {
context.indexShard().getSearchOperationListener().onFreeContext(context);
if (context.scrollContext() != null) {
@ -624,9 +630,13 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
}
private void cleanContext(SearchContext context) {
assert context == SearchContext.current();
context.clearReleasables(Lifetime.PHASE);
SearchContext.removeCurrent();
try {
assert context == SearchContext.current();
context.clearReleasables(Lifetime.PHASE);
SearchContext.removeCurrent();
} finally {
context.decRef();
}
}
private void processFailure(SearchContext context, Exception e) {

View File

@ -64,6 +64,14 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket, I
return Double.isInfinite(((Number) to).doubleValue()) ? null : new DateTime(((Number) to).longValue(), DateTimeZone.UTC);
}
private Double internalGetFrom() {
return from;
}
private Double internalGetTo() {
return to;
}
@Override
protected InternalRange.Factory<Bucket, ?> getFactory() {
return FACTORY;
@ -109,8 +117,8 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket, I
@Override
public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) {
return new Bucket(prototype.getKey(), ((Number) prototype.getFrom()).doubleValue(), ((Number) prototype.getTo()).doubleValue(),
prototype.getDocCount(), aggregations, prototype.getKeyed(), prototype.getFormat());
return new Bucket(prototype.getKey(), prototype.internalGetFrom(), prototype.internalGetTo(),
prototype.getDocCount(), aggregations, prototype.getKeyed(), prototype.getFormat());
}
}

View File

@ -22,7 +22,9 @@ package org.elasticsearch.search.internal;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.RefCount;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
@ -30,6 +32,8 @@ import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
import org.elasticsearch.common.util.concurrent.RefCounted;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
@ -67,7 +71,18 @@ import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
public abstract class SearchContext implements Releasable {
/**
* This class encapsulates the state needed to execute a search. It holds a reference to the
* shards point in time snapshot (IndexReader / ContextIndexSearcher) and allows passing on
* state from one query / fetch phase to another.
*
* This class also implements {@link RefCounted} since in some situations like in {@link org.elasticsearch.search.SearchService}
* a SearchContext can be closed concurrently due to independent events ie. when an index gets removed. To prevent accessing closed
* IndexReader / IndexSearcher instances the SearchContext can be guarded by a reference count and fail if it's been closed by
* an external event.
*/
// For reference why we use RefCounted here see #20095
public abstract class SearchContext extends AbstractRefCounted implements Releasable {
private static ThreadLocal<SearchContext> current = new ThreadLocal<>();
public static final int DEFAULT_TERMINATE_AFTER = 0;
@ -91,6 +106,7 @@ public abstract class SearchContext implements Releasable {
protected final ParseFieldMatcher parseFieldMatcher;
protected SearchContext(ParseFieldMatcher parseFieldMatcher) {
super("search_context");
this.parseFieldMatcher = parseFieldMatcher;
}
@ -100,17 +116,27 @@ public abstract class SearchContext implements Releasable {
@Override
public final void close() {
if (closed.compareAndSet(false, true)) { // prevent double release
try {
clearReleasables(Lifetime.CONTEXT);
} finally {
doClose();
}
if (closed.compareAndSet(false, true)) { // prevent double closing
decRef();
}
}
private boolean nowInMillisUsed;
@Override
protected final void closeInternal() {
try {
clearReleasables(Lifetime.CONTEXT);
} finally {
doClose();
}
}
@Override
protected void alreadyClosed() {
throw new IllegalStateException("search context is already closed can't increment refCount current count [" + refCount() + "]");
}
protected abstract void doClose();
/**

View File

@ -30,15 +30,15 @@ import java.io.IOException;
public class SnapshotMissingException extends SnapshotException {
public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId, final Throwable cause) {
super(repositoryName, snapshotId, "is missing", cause);
super(repositoryName, snapshotId, " is missing", cause);
}
public SnapshotMissingException(final String repositoryName, final SnapshotId snapshotId) {
super(repositoryName, snapshotId, "is missing");
super(repositoryName, snapshotId, " is missing");
}
public SnapshotMissingException(final String repositoryName, final String snapshotName) {
super(repositoryName, snapshotName, "is missing");
super(repositoryName, snapshotName, " is missing");
}
public SnapshotMissingException(StreamInput in) throws IOException {

View File

@ -38,6 +38,8 @@ grant codeBase "${codebase.lucene-test-framework-6.1.0.jar}" {
permission java.lang.reflect.ReflectPermission "suppressAccessChecks";
// needed for testing hardlinks in StoreRecoveryTests since we install MockFS
permission java.nio.file.LinkPermission "hard";
// needed for RAMUsageTester
permission java.lang.RuntimePermission "accessDeclaredMembers";
};
grant codeBase "${codebase.randomizedtesting-runner-2.3.2.jar}" {

View File

@ -158,7 +158,6 @@ public class MetaDataIndexTemplateServiceTests extends ESSingleNodeTestCase {
null,
null,
null,
new HashSet<>(),
null,
null, null, null);
MetaDataIndexTemplateService service = new MetaDataIndexTemplateService(Settings.EMPTY, null, createIndexService, new AliasValidator(Settings.EMPTY), null, null);
@ -188,7 +187,6 @@ public class MetaDataIndexTemplateServiceTests extends ESSingleNodeTestCase {
indicesService,
null,
null,
new HashSet<>(),
null,
nodeServicesProvider,
null,

View File

@ -385,15 +385,6 @@ public class BootstrapCheckTests extends ESTestCase {
BootstrapCheck.check(true, false, Collections.singletonList(check), "testMaxMapCountCheck");
}
public void testMinMasterNodes() {
boolean isSet = randomBoolean();
BootstrapCheck.Check check = new BootstrapCheck.MinMasterNodesCheck(isSet);
assertThat(check.check(), not(equalTo(isSet)));
List<BootstrapCheck.Check> defaultChecks = BootstrapCheck.checks(Settings.EMPTY);
expectThrows(RuntimeException.class, () -> BootstrapCheck.check(true, false, defaultChecks, "testMinMasterNodes"));
}
public void testClientJvmCheck() {
final AtomicReference<String> vmName = new AtomicReference<>("Java HotSpot(TM) 32-Bit Client VM");
final BootstrapCheck.Check check = new BootstrapCheck.ClientJvmCheck() {

View File

@ -44,6 +44,8 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.function.Supplier;
public class ClusterModuleTests extends ModuleTestCase {
private ClusterService clusterService = new ClusterService(Settings.EMPTY,
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), null);
@ -65,13 +67,6 @@ public class ClusterModuleTests extends ModuleTestCase {
}
}
static class FakeIndexTemplateFilter implements IndexTemplateFilter {
@Override
public boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template) {
return false;
}
}
public void testRegisterClusterDynamicSettingDuplicate() {
try {
new SettingsModule(Settings.EMPTY, EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING);
@ -105,7 +100,7 @@ public class ClusterModuleTests extends ModuleTestCase {
public void testRegisterAllocationDeciderDuplicate() {
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () ->
new ClusterModule(Settings.EMPTY, clusterService,
Collections.singletonList(new ClusterPlugin() {
Collections.<ClusterPlugin>singletonList(new ClusterPlugin() {
@Override
public Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
return Collections.singletonList(new EnableAllocationDecider(settings, clusterSettings));
@ -126,49 +121,39 @@ public class ClusterModuleTests extends ModuleTestCase {
assertTrue(module.allocationDeciders.stream().anyMatch(d -> d.getClass().equals(FakeAllocationDecider.class)));
}
private ClusterModule newClusterModuleWithShardsAllocator(Settings settings, String name, Supplier<ShardsAllocator> supplier) {
return new ClusterModule(settings, clusterService, Collections.singletonList(
new ClusterPlugin() {
@Override
public Map<String, Supplier<ShardsAllocator>> getShardsAllocators(Settings settings, ClusterSettings clusterSettings) {
return Collections.singletonMap(name, supplier);
}
}
));
}
public void testRegisterShardsAllocator() {
Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "custom").build();
ClusterModule module = new ClusterModule(settings, clusterService, Collections.emptyList());
module.registerShardsAllocator("custom", FakeShardsAllocator.class);
assertBinding(module, ShardsAllocator.class, FakeShardsAllocator.class);
ClusterModule module = newClusterModuleWithShardsAllocator(settings, "custom", FakeShardsAllocator::new);
assertEquals(FakeShardsAllocator.class, module.shardsAllocator.getClass());
}
public void testRegisterShardsAllocatorAlreadyRegistered() {
ClusterModule module = new ClusterModule(Settings.EMPTY, clusterService, Collections.emptyList());
try {
module.registerShardsAllocator(ClusterModule.BALANCED_ALLOCATOR, FakeShardsAllocator.class);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [shards_allocator] more than once for [balanced]");
}
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () ->
newClusterModuleWithShardsAllocator(Settings.EMPTY, ClusterModule.BALANCED_ALLOCATOR, FakeShardsAllocator::new));
assertEquals("ShardsAllocator [" + ClusterModule.BALANCED_ALLOCATOR + "] already defined", e.getMessage());
}
public void testUnknownShardsAllocator() {
Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "dne").build();
ClusterModule module = new ClusterModule(settings, clusterService, Collections.emptyList());
assertBindingFailure(module, "Unknown [shards_allocator]");
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () ->
new ClusterModule(settings, clusterService, Collections.emptyList()));
assertEquals("Unknown ShardsAllocator [dne]", e.getMessage());
}
public void testEvenShardsAllocatorBackcompat() {
Settings settings = Settings.builder()
.put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR).build();
ClusterModule module = new ClusterModule(settings, clusterService, Collections.emptyList());
assertBinding(module, ShardsAllocator.class, BalancedShardsAllocator.class);
}
public void testRegisterIndexTemplateFilterDuplicate() {
ClusterModule module = new ClusterModule(Settings.EMPTY, clusterService, Collections.emptyList());
try {
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(),
"Can't register the same [index_template_filter] more than once for [" + FakeIndexTemplateFilter.class.getName() + "]");
}
}
public void testRegisterIndexTemplateFilter() {
ClusterModule module = new ClusterModule(Settings.EMPTY, clusterService, Collections.emptyList());
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
assertSetMultiBinding(module, IndexTemplateFilter.class, FakeIndexTemplateFilter.class);
public void testShardsAllocatorFactoryNull() {
Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), "bad").build();
NullPointerException e = expectThrows(NullPointerException.class, () ->
newClusterModuleWithShardsAllocator(settings, "bad", () -> null));
}
}

View File

@ -161,9 +161,8 @@ public class ShardFailedClusterStateTaskExecutorTests extends ESAllocationTestCa
allocationService.reroute(stateAfterAddingNode, reason).routingTable();
ClusterState stateAfterReroute = ClusterState.builder(stateAfterAddingNode).routingTable(afterReroute).build();
RoutingNodes routingNodes = stateAfterReroute.getRoutingNodes();
RoutingTable afterStart =
allocationService.applyStartedShards(stateAfterReroute, routingNodes.shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
return ClusterState.builder(stateAfterReroute).routingTable(afterStart).build();
RoutingAllocation.Result afterStart = allocationService.applyStartedShards(stateAfterReroute, routingNodes.shardsWithState(ShardRoutingState.INITIALIZING));
return ClusterState.builder(stateAfterReroute).routingResult(afterStart).build();
}
private List<ShardStateAction.ShardEntry> createExistingShards(ClusterState currentState, String reason) {

View File

@ -215,7 +215,6 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase {
null,
null,
null,
new HashSet<>(),
null,
null,
null,

View File

@ -32,6 +32,9 @@ import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.cluster.ESAllocationTestCase;
import org.junit.Before;
import java.util.Set;
import java.util.stream.Collectors;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.hamcrest.Matchers.containsString;
import static org.hamcrest.Matchers.is;
@ -89,14 +92,14 @@ public class RoutingTableTests extends ESAllocationTestCase {
RoutingAllocation.Result rerouteResult = ALLOCATION_SERVICE.reroute(clusterState, "reroute");
this.testRoutingTable = rerouteResult.routingTable();
assertThat(rerouteResult.changed(), is(true));
this.clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
this.clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
}
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.getRoutingNodes().shardsWithState(index, INITIALIZING));
this.clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
this.clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
this.testRoutingTable = rerouteResult.routingTable();
}
@ -296,10 +299,11 @@ public class RoutingTableTests extends ESAllocationTestCase {
.numberOfShards(numShards)
.numberOfReplicas(numReplicas)
.build();
MetaData metaData = MetaData.builder().put(indexMetaData, true).build();
final RoutingTableGenerator routingTableGenerator = new RoutingTableGenerator();
final RoutingTableGenerator.ShardCounter counter = new RoutingTableGenerator.ShardCounter();
final IndexRoutingTable indexRoutingTable = routingTableGenerator.genIndexRoutingTable(indexMetaData, counter);
indexMetaData = updateActiveAllocations(indexRoutingTable, indexMetaData);
MetaData metaData = MetaData.builder().put(indexMetaData, true).build();
// test no validation errors
assertTrue(indexRoutingTable.validate(metaData));
// test wrong number of shards causes validation errors
@ -327,4 +331,16 @@ public class RoutingTableTests extends ESAllocationTestCase {
final MetaData metaData4 = MetaData.builder().put(indexMetaData, true).build();
expectThrows(IllegalStateException.class, () -> indexRoutingTable.validate(metaData4));
}
public static IndexMetaData updateActiveAllocations(IndexRoutingTable indexRoutingTable, IndexMetaData indexMetaData) {
IndexMetaData.Builder imdBuilder = IndexMetaData.builder(indexMetaData);
for (IndexShardRoutingTable shardTable : indexRoutingTable) {
for (ShardRouting shardRouting : shardTable) {
Set<String> activeAllocations = shardTable.activeShards().stream().map(
shr -> shr.allocationId().getId()).collect(Collectors.toSet());
imdBuilder.putActiveAllocationIds(shardRouting.id(), activeAllocations);
}
}
return imdBuilder.build();
}
}

View File

@ -111,50 +111,44 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
nodes.add(newNode("node2"));
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
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;
logger.error(clusterState.prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
assertTrue(routingResult.changed());
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();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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)));
assertTrue(routingResult.changed());
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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)));
assertTrue(routingResult.changed());
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(prev, Matchers.sameInstance(routingTable));
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertFalse(routingResult.changed());
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
}
@ -181,50 +175,44 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
nodes.add(newNode("node2"));
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
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();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.not(Matchers.sameInstance(routingTable)));
assertTrue(routingResult.changed());
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();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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)));
assertTrue(routingResult.changed());
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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)));
assertTrue(routingResult.changed());
prev = routingTable;
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prev, Matchers.sameInstance(routingTable));
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertFalse(routingResult.changed());
assertNumIndexShardsPerNode(clusterState, Matchers.equalTo(2));
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
}
@ -262,24 +250,11 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
RoutingTable routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
RoutingAllocation.Result routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// move initializing to started
RoutingTable prev = routingTable;
while (true) {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, service);
}
private ClusterState initCluster(AllocationService service, int numberOfNodes, int numberOfIndices, int numberOfShards,
@ -299,43 +274,30 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
routingTableBuilder.addAsNew(cursor.value);
}
RoutingTable routingTable = routingTableBuilder.build();
RoutingTable initialRoutingTable = routingTableBuilder.build();
logger.info("start {} nodes", numberOfNodes);
DiscoveryNodes.Builder nodes = DiscoveryNodes.builder();
for (int i = 0; i < numberOfNodes; i++) {
nodes.add(newNode("node" + i));
}
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(nodes).metaData(metaData).routingTable(routingTable).build();
routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(nodes).metaData(metaData).routingTable(initialRoutingTable).build();
RoutingAllocation.Result routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the replica shards");
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
while (true) {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, service);
}
private ClusterState addIndex(ClusterState clusterState, AllocationService service, int indexOrdinal, int numberOfShards,
@ -350,37 +312,22 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
routingTableBuilder.addAsNew(imd);
MetaData metaData = metaDataBuilder.build();
RoutingTable routingTable = routingTableBuilder.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTableBuilder.build()).build();
RoutingAllocation.Result routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the replica shards");
routingNodes = clusterState.getRoutingNodes();
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
while (true) {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, service);
}
private ClusterState removeNodes(ClusterState clusterState, AllocationService service, int numNodes) {
@ -399,34 +346,23 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(nodes.build()).build();
clusterState = ClusterState.builder(clusterState)
.routingResult(service.deassociateDeadNodes(clusterState, true, "reroute")).build();
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.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
RoutingAllocation.Result routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the replica shards");
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("rebalancing");
routingTable = service.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = service.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
while (true) {
logger.debug("ClusterState: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
clusterState = applyStartedShardsUntilNoChange(clusterState, service);
return clusterState;
}

View File

@ -81,11 +81,11 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("start primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("move the shard");
String existingNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
@ -97,13 +97,13 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
}
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, existingNodeId, toNodeId)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(existingNodeId).iterator().next().state(), equalTo(ShardRoutingState.RELOCATING));
assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.INITIALIZING));
logger.info("finish moving the shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(existingNodeId).isEmpty(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.STARTED));
@ -144,7 +144,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
.add(newNode("node4", singleton(DiscoveryNode.Role.MASTER)))
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> allocating to non-existent node, should fail");
@ -198,14 +198,14 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> allocating empty primary with acceptDataLoss flag set to true");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateEmptyPrimaryAllocationCommand("test", 0, "node1", true)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -220,7 +220,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateReplicaAllocationCommand("test", 0, "node2")), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -229,7 +229,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -265,13 +265,13 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
.add(newNode("node3"))
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
logger.info("--> allocating empty primary shard with accept_data_loss flag set to true");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateEmptyPrimaryAllocationCommand("test", 0, "node1", true)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -285,7 +285,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> start the primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -300,7 +300,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateReplicaAllocationCommand("test", 0, "node2")), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -309,7 +309,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> cancel the relocation allocation");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node2", false)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -318,7 +318,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateReplicaAllocationCommand("test", 0, "node2")), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -333,7 +333,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -342,7 +342,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> cancel allocation of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node2", false)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -350,7 +350,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> allocate the replica shard on on the second node");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new AllocateReplicaAllocationCommand("test", 0, "node2")), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").shardsWithState(STARTED).size(), equalTo(1));
@ -358,7 +358,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -366,7 +366,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> move the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, "node2", "node3")), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -377,7 +377,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
if (randomBoolean()) {
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node1", true)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().node("node2").shardsWithState(STARTED).iterator().next().primary(), equalTo(true));
@ -385,7 +385,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
} else {
logger.info("--> cancel the move of the replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node3", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -393,7 +393,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> move the replica shard again");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, "node2", "node3")), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -403,7 +403,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> cancel the source replica shard");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node2", false)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -413,7 +413,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> start the former target replica shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
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));
@ -421,7 +421,7 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
logger.info("--> cancel the primary allocation (with allow_primary set to true)");
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new CancelAllocationCommand("test", 0, "node1", true)), false, false);
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(rerouteResult.changed(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node3").shardsWithState(STARTED).iterator().next().primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));

View File

@ -61,36 +61,36 @@ public class AllocationPriorityTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("first").settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_PRIORITY, priorityFirst)).numberOfShards(2).numberOfReplicas(1))
.put(IndexMetaData.builder("second").settings(settings(Version.CURRENT).put(IndexMetaData.SETTING_PRIORITY, prioritySecond)).numberOfShards(2).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("first"))
.addAsNew(metaData.index("second"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
routingTable = allocation.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size());
assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName());
assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName());
routingTable = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size());
assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName());
assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName());
routingTable = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).toString(),2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size());
assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName());
assertEquals(highPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName());
routingTable = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(2, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size());
assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0).getIndexName());
assertEquals(lowPriorityName, clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(1).getIndexName());

View File

@ -64,28 +64,26 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -93,16 +91,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -113,9 +111,9 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@ -132,11 +130,9 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -144,17 +140,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.add(newNode("node2", singletonMap("rack_id", "1")))
.add(newNode("node3", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -162,16 +158,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -182,9 +178,9 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node5", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@ -206,19 +202,19 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Initializing shards: {}", clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
logger.info("Started shards: {}", clusterState.getRoutingNodes().shardsWithState(STARTED));
@ -228,12 +224,12 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
@ -241,8 +237,8 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(5));
@ -250,12 +246,12 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).get(0).relocatingNodeId(), equalTo("node3"));
logger.info("--> complete initializing");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> run it again, since we still might have relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
@ -266,13 +262,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
@ -296,29 +292,29 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
@ -326,8 +322,8 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(10));
@ -337,11 +333,11 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
logger.info("--> complete initializing");
for (int i = 0; i < 2; i++) {
logger.info("--> complete initializing round: [{}]", i);
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(10));
@ -355,15 +351,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(5));
@ -388,28 +384,28 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -417,16 +413,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
@ -437,15 +433,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(3));
@ -466,11 +462,11 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(3))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -479,17 +475,17 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.add(newNode("node3", singletonMap("rack_id", "1")))
.add(newNode("node4", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
@ -497,16 +493,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node5", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
@ -517,15 +513,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node6", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(4));
@ -547,24 +543,24 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> replica will not start because we have only one rack value");
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
@ -574,16 +570,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -594,9 +590,9 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@ -614,11 +610,11 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -626,13 +622,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.add(newNode("node2", singletonMap("rack_id", "1")))
.add(newNode("node3", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> replica will not start because we have only one rack value");
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
@ -642,16 +638,16 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
@ -662,9 +658,9 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node5", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(routingTable, sameInstance(clusterState.routingTable()));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
}
@ -688,25 +684,25 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes on same rack and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("rack_id", "1")))
.add(newNode("node2", singletonMap("rack_id", "1")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(10));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
@ -714,20 +710,20 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", singletonMap("rack_id", "2")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> run it again, since we still might have relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
@ -738,13 +734,13 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4", singletonMap("rack_id", "3")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), greaterThan(0));
logger.info("--> complete relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(20));
@ -768,30 +764,30 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes in different zones and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("A-0", singletonMap("zone", "a")))
.add(newNode("B-0", singletonMap("zone", "b")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(5));
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> all replicas are allocated and started since we have on node in each zone");
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
@ -800,15 +796,15 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("A-1", singletonMap("zone", "a")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(10));
assertThat(clusterState.getRoutingNodes().node("A-1").size(), equalTo(2));
@ -829,11 +825,11 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(4))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding 5 nodes in different zones and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -844,14 +840,14 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.add(newNode("A-4", singletonMap("zone", "a")))
.add(newNode("B-0", singletonMap("zone", "b")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shard (primary)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(1)); // Unassigned shard is expected.
@ -859,7 +855,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
// Cancel all initializing shards and move started primary to another node.
AllocationCommands commands = new AllocationCommands();
String primaryNode = null;
for (ShardRouting routing : routingTable.allShards()) {
for (ShardRouting routing : clusterState.routingTable().allShards()) {
if (routing.primary()) {
primaryNode = routing.currentNodeId();
} else if (routing.initializing()) {
@ -868,8 +864,8 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
}
commands.add(new MoveAllocationCommand("test", 0, primaryNode, "A-4"));
routingTable = strategy.reroute(clusterState, commands, false, false).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, commands, false, false);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(1));

View File

@ -81,7 +81,6 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
clusterState = removeNodes(clusterState, strategy);
assertIndexBalance(clusterState.getRoutingTable(), clusterState.getRoutingNodes(), (numberOfNodes + 1) - (numberOfNodes + 1) / 2, numberOfIndices, numberOfReplicas, numberOfShards, balanceTreshold);
}
public void testReplicaBalance() {
@ -124,7 +123,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
routingTableBuilder.addAsNew(cursor.value);
}
RoutingTable routingTable = routingTableBuilder.build();
RoutingTable initialRoutingTable = routingTableBuilder.build();
logger.info("start " + numberOfNodes + " nodes");
@ -132,35 +131,22 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
for (int i = 0; i < numberOfNodes; i++) {
nodes.add(newNode("node" + i));
}
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(nodes).metaData(metaData).routingTable(routingTable).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).nodes(nodes).metaData(metaData).routingTable(initialRoutingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("restart all the primary shards, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the replica shards");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, strategy);
}
private ClusterState addNode(ClusterState clusterState, AllocationService strategy) {
@ -171,21 +157,9 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
RoutingTable routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
// move initializing to started
RoutingTable prev = routingTable;
while (true) {
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, strategy);
}
private ClusterState removeNodes(ClusterState clusterState, AllocationService strategy) {
@ -204,35 +178,23 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
strategy.deassociateDeadNodes(clusterState, randomBoolean(), "removed nodes")
).build();
}
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.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
RoutingAllocation.Result routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the replica shards");
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("rebalancing");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes();
if (routingTable == prev)
break;
prev = routingTable;
}
return clusterState;
return applyStartedShardsUntilNoChange(clusterState, strategy);
}

View File

@ -60,22 +60,22 @@ public class BalanceUnbalancedClusterTests extends CatAllocationTestCase {
.put(IndexMetaData.builder(index).settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder(state.routingTable())
RoutingTable initialRoutingTable = RoutingTable.builder(state.routingTable())
.addAsNew(metaData.index(index))
.build();
ClusterState clusterState = ClusterState.builder(state).metaData(metaData).routingTable(routingTable).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(state).metaData(metaData).routingTable(initialRoutingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
while (true) {
if (routingTable.shardsWithState(INITIALIZING).isEmpty()) {
if (clusterState.routingTable().shardsWithState(INITIALIZING).isEmpty()) {
break;
}
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
Map<String, Integer> counts = new HashMap<>();
for (IndexShardRoutingTable table : routingTable.index(index)) {
for (IndexShardRoutingTable table : clusterState.routingTable().index(index)) {
for (ShardRouting r : table) {
String s = r.currentNodeId();
Integer count = counts.get(s);

View File

@ -96,8 +96,21 @@ public abstract class CatAllocationTestCase extends ESAllocationTestCase {
MetaData.Builder builder = MetaData.builder();
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
for(Idx idx : indices.values()) {
IndexMetaData idxMeta = IndexMetaData.builder(idx.name).settings(settings(Version.CURRENT))
.numberOfShards(idx.numShards()).numberOfReplicas(idx.numReplicas()).build();
IndexMetaData.Builder idxMetaBuilder = IndexMetaData.builder(idx.name).settings(settings(Version.CURRENT))
.numberOfShards(idx.numShards()).numberOfReplicas(idx.numReplicas());
for (ShardRouting shardRouting : idx.routing) {
if (shardRouting.active()) {
Set<String> allocationIds = idxMetaBuilder.getActiveAllocationIds(shardRouting.id());
if (allocationIds == null) {
allocationIds = new HashSet<>();
} else {
allocationIds = new HashSet<>(allocationIds);
}
allocationIds.add(shardRouting.allocationId().getId());
idxMetaBuilder.putActiveAllocationIds(shardRouting.id(), allocationIds);
}
}
IndexMetaData idxMeta = idxMetaBuilder.build();
builder.put(idxMeta, false);
IndexRoutingTable.Builder tableBuilder = new IndexRoutingTable.Builder(idxMeta.getIndex()).initializeAsRecovery(idxMeta);
Map<Integer, IndexShardRoutingTable> shardIdToRouting = new HashMap<>();
@ -107,7 +120,6 @@ public abstract class CatAllocationTestCase extends ESAllocationTestCase {
refData = new IndexShardRoutingTable.Builder(shardIdToRouting.get(r.getId())).addShard(r).build();
}
shardIdToRouting.put(r.getId(), refData);
}
for (IndexShardRoutingTable t: shardIdToRouting.values()) {
tableBuilder.addIndexShard(t);
@ -139,20 +151,18 @@ public abstract class CatAllocationTestCase extends ESAllocationTestCase {
private ClusterState rebalance(ClusterState clusterState) {
RoutingTable routingTable;AllocationService strategy = createAllocationService(Settings.builder()
.build());
RoutingAllocation.Result reroute = strategy.reroute(clusterState, "reroute");
routingTable = reroute.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingTable = clusterState.routingTable();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
int numRelocations = 0;
while (true) {
List<ShardRouting> initializing = routingTable.shardsWithState(INITIALIZING);
List<ShardRouting> initializing = clusterState.routingTable().shardsWithState(INITIALIZING);
if (initializing.isEmpty()) {
break;
}
logger.debug("Initializing shards: {}", initializing);
numRelocations += initializing.size();
routingTable = strategy.applyStartedShards(clusterState, initializing).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, initializing);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
logger.debug("--> num relocations to get balance: {}", numRelocations);
return clusterState;

View File

@ -56,76 +56,72 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
// assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
// assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("now, start 1 more node, check that rebalancing will happen (for test1) because we set it to always");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
@ -142,95 +138,90 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("now, start 1 more node, check that rebalancing happen (for test1) because we set it to primaries_active");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
@ -246,76 +237,72 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("now, start 1 more node, check that rebalancing will not happen (for test1) because we set it to primaries_active");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
@ -330,114 +317,108 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("start the test2 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
logger.info("now, start 1 more node, check that rebalancing happen (for test1) because we set it to all_active");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").size(), equalTo(1));
@ -453,76 +434,72 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("now, start 1 more node, check that rebalancing will not happen (for test1) because we set it to all_active");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
@ -537,95 +514,90 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test1, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start the test1 replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards for test2, replicas will start initializing");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test2", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test2").shards().size(); i++) {
assertThat(routingTable.index("test2").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test2").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test2").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test2").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test2").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("now, start 1 more node, check that rebalancing will not happen (for test1) because we set it to all_active");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node3").isEmpty(), equalTo(true));
@ -657,31 +629,31 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.addAsNew(metaData.index("test1"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
}
logger.debug("start all the primary shards for test");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
}
logger.debug("now, start 1 more node, check that rebalancing will not happen since we unassigned shards");
@ -691,44 +663,45 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
logger.debug("reroute and check that nothing has changed");
RoutingAllocation.Result reroute = strategy.reroute(clusterState, "reroute");
assertFalse(reroute.changed());
routingTable = reroute.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = reroute;
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
}
logger.debug("now set allocateTest1 to true and reroute we should see the [test1] index initializing");
allocateTest1.set(true);
reroute = strategy.reroute(clusterState, "reroute");
assertTrue(reroute.changed());
routingTable = reroute.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
routingResult = reroute;
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
}
logger.debug("now start initializing shards and expect exactly one rebalance from node1 to node 2 since index [test] is all on node1");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING)).routingTable();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test1", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
}
int numStarted = 0;
int numRelocating = 0;
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
if (routingTable.index("test").shard(i).primaryShard().state() == STARTED) {
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
if (clusterState.routingTable().index("test").shard(i).primaryShard().state() == STARTED) {
numStarted++;
} else if (routingTable.index("test").shard(i).primaryShard().state() == RELOCATING) {
} else if (clusterState.routingTable().index("test").shard(i).primaryShard().state() == RELOCATING) {
numRelocating++;
}
}
@ -756,31 +729,31 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.build();
// we use a second index here (test1) that never gets assigned otherwise allocateUnassigned is never called if we don't have unassigned shards.
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.addAsNew(metaData.index("test1"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
}
logger.debug("start all the primary shards for test");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState("test", INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
}
logger.debug("now, start 1 more node, check that rebalancing will not happen since we have shard sync going on");
@ -788,41 +761,38 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
.add(newNode("node2")))
.build();
logger.debug("reroute and check that nothing has changed");
RoutingAllocation.Result reroute = strategy.reroute(clusterState, "reroute");
assertFalse(reroute.changed());
routingTable = reroute.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
}
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
}
logger.debug("now set hasFetches to true and reroute we should now see exactly one relocating shard");
hasFetches.set(false);
reroute = strategy.reroute(clusterState, "reroute");
assertTrue(reroute.changed());
routingTable = reroute.routingTable();
routingResult = strategy.reroute(clusterState, "reroute");
assertTrue(routingResult.changed());
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
int numStarted = 0;
int numRelocating = 0;
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(1));
if (routingTable.index("test").shard(i).primaryShard().state() == STARTED) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(1));
if (clusterState.routingTable().index("test").shard(i).primaryShard().state() == STARTED) {
numStarted++;
} else if (routingTable.index("test").shard(i).primaryShard().state() == RELOCATING) {
} else if (clusterState.routingTable().index("test").shard(i).primaryShard().state() == RELOCATING) {
numRelocating++;
}
}
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
}
assertEquals(numStarted, 1);
assertEquals(numRelocating, 1);
}
}

View File

@ -53,96 +53,90 @@ public class ConcurrentRebalanceRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(5));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(5));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("start two nodes and fully start the shards");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("now, start 8 more nodes, and check that no rebalancing/relocation have happened");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")).add(newNode("node4")).add(newNode("node5")).add(newNode("node6")).add(newNode("node7")).add(newNode("node8")).add(newNode("node9")).add(newNode("node10")))
.build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("start the replica shards, rebalancing should start, but, only 3 should be rebalancing");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(7));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(3));
logger.info("finalize this session relocation, 3 more should relocate now");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(7));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(3));
logger.info("finalize this session relocation, 2 more should relocate now");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(8));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(2));
logger.info("finalize this session relocation, no more relocation");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// we only allow one relocation at a time
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(0));
}
}

View File

@ -65,14 +65,14 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
@ -88,7 +88,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
).build();
rerouteResult = allocation.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).iterator().next().primary(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node(nodeIdRemaining).iterator().next().state(), equalTo(STARTED));
@ -116,14 +116,14 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
@ -136,7 +136,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
.add(newNode("node3"))
).build();
rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").iterator().next().state(), equalTo(STARTED));
@ -152,7 +152,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
new MoveAllocationCommand("test", 0, clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), "node3")),
false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").iterator().next().state(), equalTo(INITIALIZING));
@ -162,7 +162,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
.add(newNode(origReplicaNodeId))
).build();
rerouteResult = allocation.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).iterator().next().state(), equalTo(STARTED));
@ -190,14 +190,14 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
@ -210,7 +210,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
.add(newNode("node3"))
).build();
rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").iterator().next().state(), equalTo(STARTED));
@ -226,7 +226,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
new MoveAllocationCommand("test",0 , clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), "node3")),
false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").iterator().next().state(), equalTo(INITIALIZING));
@ -236,7 +236,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
.add(newNode(origReplicaNodeId))
).build();
rerouteResult = allocation.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).iterator().next().state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").iterator().next().state(), equalTo(INITIALIZING));

View File

@ -51,61 +51,56 @@ public class ElectReplicaAsPrimaryDuringRelocationTests extends ESAllocationTest
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(2));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(2));
logger.info("Start another node and perform rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("find the replica shard that gets relocated");
IndexShardRoutingTable indexShardRoutingTable = null;
if (routingTable.index("test").shard(0).replicaShards().get(0).relocating()) {
indexShardRoutingTable = routingTable.index("test").shard(0);
} else if (routingTable.index("test").shard(1).replicaShards().get(0).relocating()) {
indexShardRoutingTable = routingTable.index("test").shard(1);
if (clusterState.routingTable().index("test").shard(0).replicaShards().get(0).relocating()) {
indexShardRoutingTable = clusterState.routingTable().index("test").shard(0);
} else if (clusterState.routingTable().index("test").shard(1).replicaShards().get(0).relocating()) {
indexShardRoutingTable = clusterState.routingTable().index("test").shard(1);
}
// we might have primary relocating, and the test is only for replicas, so only test in the case of replica allocation
if (indexShardRoutingTable != null) {
logger.info("kill the node [{}] of the primary shard for the relocating replica", indexShardRoutingTable.primaryShard().currentNodeId());
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove(indexShardRoutingTable.primaryShard().currentNodeId())).build();
prevRoutingTable = routingTable;
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("make sure all the primary shards are active");
assertThat(routingTable.index("test").shard(0).primaryShard().active(), equalTo(true));
assertThat(routingTable.index("test").shard(1).primaryShard().active(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().active(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shard(1).primaryShard().active(), equalTo(true));
}
}
}

View File

@ -81,23 +81,23 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(1, clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING));
assertEquals(byteSize, clusterState.getRoutingTable().shardsWithState(ShardRoutingState.INITIALIZING).get(0).getExpectedShardSize());
logger.info("Start the primary shard");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(1, clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED));
assertEquals(1, clusterState.getRoutingNodes().unassigned().size());
logger.info("Add another one node and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(1, clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING));
assertEquals(byteSize, clusterState.getRoutingTable().shardsWithState(ShardRoutingState.INITIALIZING).get(0).getExpectedShardSize());
@ -135,11 +135,11 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
logger.info("adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("start primary shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("move the shard");
String existingNodeId = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
@ -151,7 +151,7 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
}
rerouteResult = allocation.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", 0, existingNodeId, toNodeId)), false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertEquals(clusterState.getRoutingNodes().node(existingNodeId).iterator().next().state(), ShardRoutingState.RELOCATING);
assertEquals(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(),ShardRoutingState.INITIALIZING);
@ -160,7 +160,7 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
logger.info("finish moving the shard");
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(existingNodeId).isEmpty(), equalTo(true));
assertThat(clusterState.getRoutingNodes().node(toNodeId).iterator().next().state(), equalTo(ShardRoutingState.STARTED));

View File

@ -49,31 +49,28 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start 4 nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3")).add(newNode("node4"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
logger.info("start the replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(1));
@ -85,13 +82,12 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
logger.info("remove 2 nodes where primaries are allocated, reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.remove(routingTable.index("test1").shard(0).primaryShard().currentNodeId())
.remove(routingTable.index("test2").shard(0).primaryShard().currentNodeId())
.remove(clusterState.routingTable().index("test1").shard(0).primaryShard().currentNodeId())
.remove(clusterState.routingTable().index("test2").shard(0).primaryShard().currentNodeId())
)
.build();
prevRoutingTable = routingTable;
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (RoutingNode routingNode : routingNodes) {

View File

@ -78,14 +78,14 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
).build();
RoutingAllocation.Result rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting primaries
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// starting replicas
rerouteResult = allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
logger.info("--> verifying all is allocated");
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
@ -98,7 +98,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.add(newNode("node3"))
).build();
rerouteResult = allocation.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node1").iterator().next().state(), equalTo(STARTED));
@ -114,13 +114,13 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
new MoveAllocationCommand("test", 0, clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), "node3")),
false, false);
assertThat(rerouteResult.changed(), equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(RELOCATING));
assertThat(clusterState.getRoutingNodes().node("node3").iterator().next().state(), equalTo(INITIALIZING));
logger.info("--> fail primary shard recovering instance on node3 being initialized");
rerouteResult = allocation.applyFailedShard(clusterState, clusterState.getRoutingNodes().node("node3").iterator().next());
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
assertThat(clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next().state(), equalTo(STARTED));
assertThat(clusterState.getRoutingNodes().node("node3").size(), equalTo(0));
@ -136,7 +136,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
logger.info("--> fail primary shard recovering instance on node1 being relocated");
rerouteResult = allocation.applyFailedShard(clusterState, clusterState.getRoutingNodes().node(origPrimaryNodeId).iterator().next());
clusterState = ClusterState.builder(clusterState).routingTable(rerouteResult.routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(rerouteResult).build();
// check promotion of replica to primary
assertThat(clusterState.getRoutingNodes().node(origReplicaNodeId).iterator().next().state(), equalTo(STARTED));
@ -156,70 +156,66 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the shards (primaries)");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
}
logger.info("Start the shards (backups)");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
}
logger.info("fail the primary shard, will have no place to be rerouted to (single node), so stays unassigned");
ShardRouting shardToFail = routingTable.index("test").shard(0).primaryShard();
prevRoutingTable = routingTable;
routingTable = strategy.applyFailedShard(clusterState, shardToFail).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
ShardRouting shardToFail = clusterState.routingTable().index("test").shard(0).primaryShard();
routingResult = strategy.applyFailedShard(clusterState, shardToFail);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), not(equalTo(shardToFail.currentNodeId())));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), not(equalTo(shardToFail.currentNodeId())));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
public void testFirstAllocationFailureSingleNode() {
@ -234,44 +230,42 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding single node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("fail the first shard, will have no place to be rerouted to (single node), so stays unassigned");
prevRoutingTable = routingTable;
ShardRouting firstShard = clusterState.getRoutingNodes().node("node1").iterator().next();
routingTable = strategy.applyFailedShard(clusterState, firstShard).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyFailedShard(clusterState, firstShard);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
}
@ -287,11 +281,11 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(numberOfReplicas))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding {} nodes and performing rerouting", numberOfReplicas + 1);
DiscoveryNodes.Builder nodeBuilder = DiscoveryNodes.builder();
@ -302,12 +296,12 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
while (!clusterState.routingTable().shardsWithState(UNASSIGNED).isEmpty()) {
// start all initializing
clusterState = ClusterState.builder(clusterState)
.routingTable(strategy
.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING)).routingTable()
.routingResult(strategy
.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING))
)
.build();
// and assign more unassigned
clusterState = ClusterState.builder(clusterState).routingTable(strategy.reroute(clusterState, "reroute").routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(strategy.reroute(clusterState, "reroute")).build();
}
int shardsToFail = randomIntBetween(1, numberOfReplicas);
@ -326,9 +320,9 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
}
}
routingTable = strategy.applyFailedShards(clusterState, failedShards).routingTable();
RoutingAllocation.Result routingResult = strategy.applyFailedShards(clusterState, failedShards);
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (FailedRerouteAllocation.FailedShard failedShard : failedShards) {
if (routingNodes.getByAllocationId(failedShard.routingEntry.shardId(), failedShard.routingEntry.allocationId().getId()) != null) {
@ -355,48 +349,46 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
final String nodeHoldingPrimary = routingTable.index("test").shard(0).primaryShard().currentNodeId();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
final String nodeHoldingPrimary = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("fail the first shard, will start INITIALIZING on the second node");
prevRoutingTable = routingTable;
final ShardRouting firstShard = clusterState.getRoutingNodes().node(nodeHoldingPrimary).iterator().next();
routingTable = strategy.applyFailedShard(clusterState, firstShard).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
routingResult = strategy.applyFailedShard(clusterState, firstShard);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertTrue(routingResult.changed());
final String nodeHoldingPrimary2 = routingTable.index("test").shard(0).primaryShard().currentNodeId();
final String nodeHoldingPrimary2 = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
assertThat(nodeHoldingPrimary2, not(equalTo(nodeHoldingPrimary)));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), not(equalTo(nodeHoldingPrimary)));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), not(equalTo(nodeHoldingPrimary)));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
}
@ -412,63 +404,59 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the shards (primaries)");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
}
logger.info("Start the shards (backups)");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), anyOf(equalTo("node1"), equalTo("node2")));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), anyOf(equalTo("node2"), equalTo("node1")));
}
logger.info("Adding third node and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED, RELOCATING), equalTo(2));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), lessThan(3));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED, RELOCATING), equalTo(2));
@ -479,13 +467,12 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
logger.info("Fail the shards on node 3");
ShardRouting shardToFail = routingNodes.node("node3").iterator().next();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyFailedShard(clusterState, shardToFail).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyFailedShard(clusterState, shardToFail);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(2));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(2));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED, RELOCATING), equalTo(2));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), lessThan(3));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED, RELOCATING), equalTo(2));
@ -523,7 +510,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
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();
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// the primary gets allocated on another node, replicas are unassigned
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(2));
@ -532,7 +519,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
assertThat(newPrimaryShard, not(equalTo(primaryShardToFail)));
// start the primary shard
clusterState = ClusterState.builder(clusterState).routingTable(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
}
@ -553,16 +540,16 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
// add 4 nodes
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3")).add(newNode("node4"))).build();
clusterState = ClusterState.builder(clusterState).routingTable(allocation.reroute(clusterState, "reroute").routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.reroute(clusterState, "reroute")).build();
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.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))).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, Collections.singletonList(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0))).routingTable()).build();
clusterState = ClusterState.builder(clusterState).routingResult(allocation.applyStartedShards(clusterState, Collections.singletonList(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).get(0)))).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
@ -570,7 +557,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
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();
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));

View File

@ -58,11 +58,11 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding four nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -71,17 +71,17 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.add(newNode("node3", singletonMap("tag1", "value3")))
.add(newNode("node4", singletonMap("tag1", "value4")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> make sure shards are only allocated on tag1 with value1 and value2");
List<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
@ -97,7 +97,7 @@ public class FilterRoutingTests extends ESAllocationTestCase {
logger.info("Building initial routing table");
MetaData metaData = MetaData.builder()
MetaData initialMetaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)
.put("index.number_of_shards", 2)
.put("index.number_of_replicas", 1)
@ -106,11 +106,11 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.build()))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(initialMetaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(initialMetaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -119,17 +119,17 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.add(newNode("node3", singletonMap("tag1", "value3")))
.add(newNode("node4", singletonMap("tag1", "value4")))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> make sure shards are only allocated on tag1 with value1 and value2");
List<ShardRouting> startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
@ -140,24 +140,23 @@ public class FilterRoutingTests extends ESAllocationTestCase {
logger.info("--> switch between value2 and value4, shards should be relocating");
metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)
.put("index.number_of_shards", 2)
.put("index.number_of_replicas", 1)
.put("index.routing.allocation.include.tag1", "value1,value4")
.put("index.routing.allocation.exclude.tag1", "value2,value3")
.build()))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
IndexMetaData existingMetaData = clusterState.metaData().index("test");
MetaData updatedMetaData = MetaData.builder()
.put(IndexMetaData.builder(existingMetaData).settings(Settings.builder().put(existingMetaData.getSettings())
.put("index.routing.allocation.include.tag1", "value1,value4")
.put("index.routing.allocation.exclude.tag1", "value2,value3")
.build()))
.build();
clusterState = ClusterState.builder(clusterState).metaData(updatedMetaData).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(2));
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(2));
logger.info("--> finish relocation");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
startedShards = clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.STARTED);
assertThat(startedShards.size(), equalTo(4));
@ -175,25 +174,25 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes and performing rerouting");
DiscoveryNode node1 = newNode("node1", singletonMap("tag1", "value1"));
DiscoveryNode node2 = newNode("node2", singletonMap("tag1", "value2"));
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(node1).add(node2)).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node(node1.getId()).numberOfShardsWithState(INITIALIZING), equalTo(2));
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(INITIALIZING), equalTo(2));
logger.info("--> start the shards (only primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> make sure all shards are started");
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4));
@ -205,29 +204,29 @@ public class FilterRoutingTests extends ESAllocationTestCase {
.build());
logger.info("--> move shards from node1 to node2");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> check that concurrent recoveries only allows 1 shard to move");
assertThat(clusterState.getRoutingNodes().node(node1.getId()).numberOfShardsWithState(STARTED), equalTo(1));
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(INITIALIZING), equalTo(1));
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(2));
logger.info("--> start the shards (only primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> move second shard from node1 to node2");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(INITIALIZING), equalTo(1));
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(3));
logger.info("--> start the shards (only primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node(node2.getId()).numberOfShardsWithState(STARTED), equalTo(4));
}
}

View File

@ -56,109 +56,98 @@ public class IndexBalanceTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1))
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test1").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test1").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test1").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("Adding three node and performing rerouting");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
}
logger.info("Another round of rebalancing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they
// recover from primary *started* shards in the
// IndicesClusterStateService
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the more shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
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++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(4));
@ -186,142 +175,128 @@ public class IndexBalanceTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1))
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test1").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test1").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test1").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test1").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
}
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
logger.info("Start the primary shard");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they
// recover from primary *started* shards in the
// IndicesClusterStateService
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
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++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
}
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertTrue(routingResult.changed());
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(4));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(4));
@ -347,91 +322,79 @@ public class IndexBalanceTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("Adding three node and performing rerouting");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
}
logger.info("Another round of rebalancing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they
// recover from primary *started* shards in the
// IndicesClusterStateService
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the more shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
}
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(2));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(2));
assertThat(routingNodes.node("node3").numberOfShardsWithState(STARTED), equalTo(2));
@ -442,88 +405,75 @@ public class IndexBalanceTests extends ESAllocationTestCase {
logger.info("Add new index 3 shards 1 replica");
prevRoutingTable = routingTable;
metaData = MetaData.builder(metaData)
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 3)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
))
.build();
routingTable = RoutingTable.builder(routingTable)
.addAsNew(metaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
MetaData updatedMetaData = MetaData.builder(clusterState.metaData())
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 3)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
))
.build();
RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable())
.addAsNew(updatedMetaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(updatedMetaData).routingTable(updatedRoutingTable).build();
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
}
logger.info("Another round of rebalancing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
for (int i = 0; i < routingTable.index("test1").shards().size(); i++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they
// recover from primary *started* shards in the
// IndicesClusterStateService
assertThat(routingTable.index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
}
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the more shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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++) {
assertThat(routingTable.index("test1").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test1").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test1").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test1").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test1").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test1").shard(i).replicaShards().size(), equalTo(1));
}
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(4));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(4));
assertThat(routingNodes.node("node3").numberOfShardsWithState(STARTED), equalTo(4));
@ -531,6 +481,5 @@ public class IndexBalanceTests extends ESAllocationTestCase {
assertThat(routingNodes.node("node1").shardsWithState("test1", STARTED).size(), equalTo(2));
assertThat(routingNodes.node("node2").shardsWithState("test1", STARTED).size(), equalTo(2));
assertThat(routingNodes.node("node3").shardsWithState("test1", STARTED).size(), equalTo(2));
}
}

View File

@ -27,6 +27,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RestoreSource;
@ -47,6 +48,7 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
@ -89,104 +91,96 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(5));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(2).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(2).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(5));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).currentNodeId(), nullValue());
}
logger.info("start two nodes and fully start the shards");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(2));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(2));
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
}
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
}
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3", VersionUtils.getPreviousVersion())))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(UNASSIGNED).size(), equalTo(1));
}
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
}
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(2));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShardsWithState(STARTED).size(), equalTo(2));
}
}
@ -258,15 +252,15 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(5));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(2).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(2).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(5));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).currentNodeId(), nullValue());
}
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("old0", VersionUtils.getPreviousVersion()))
@ -294,15 +288,14 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
.add(newNode("new0"))).build();
clusterState = stabilize(clusterState, service);
routingTable = clusterState.routingTable();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(3));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).shards().get(2).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), notNullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), notNullValue());
assertThat(routingTable.index("test").shard(i).shards().get(2).currentNodeId(), notNullValue());
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), notNullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), notNullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(2).currentNodeId(), notNullValue());
}
}
@ -315,21 +308,25 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
MASTER_DATA_ROLES, VersionUtils.getPreviousVersion());
final DiscoveryNode oldNode2 = new DiscoveryNode("oldNode2", LocalTransportAddress.buildUnique(), emptyMap(),
MASTER_DATA_ROLES, VersionUtils.getPreviousVersion());
AllocationId allocationId1P = AllocationId.newInitializing();
AllocationId allocationId1R = AllocationId.newInitializing();
AllocationId allocationId2P = AllocationId.newInitializing();
AllocationId allocationId2R = AllocationId.newInitializing();
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder(shard1.getIndexName()).settings(settings(Version.CURRENT).put(Settings.EMPTY)).numberOfShards(1).numberOfReplicas(1))
.put(IndexMetaData.builder(shard2.getIndexName()).settings(settings(Version.CURRENT).put(Settings.EMPTY)).numberOfShards(1).numberOfReplicas(1))
.put(IndexMetaData.builder(shard1.getIndexName()).settings(settings(Version.CURRENT).put(Settings.EMPTY)).numberOfShards(1).numberOfReplicas(1).putActiveAllocationIds(0, Sets.newHashSet(allocationId1P.getId(), allocationId1R.getId())))
.put(IndexMetaData.builder(shard2.getIndexName()).settings(settings(Version.CURRENT).put(Settings.EMPTY)).numberOfShards(1).numberOfReplicas(1).putActiveAllocationIds(0, Sets.newHashSet(allocationId2P.getId(), allocationId2R.getId())))
.build();
RoutingTable routingTable = RoutingTable.builder()
.add(IndexRoutingTable.builder(shard1.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shard1)
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), newNode.getId(), true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), oldNode1.getId(), false, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), newNode.getId(), null, true, ShardRoutingState.STARTED, allocationId1P))
.addShard(TestShardRouting.newShardRouting(shard1.getIndexName(), shard1.getId(), oldNode1.getId(), null, false, ShardRoutingState.STARTED, allocationId1R))
.build())
)
.add(IndexRoutingTable.builder(shard2.getIndex())
.addIndexShard(new IndexShardRoutingTable.Builder(shard2)
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), newNode.getId(), true, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), oldNode1.getId(), false, ShardRoutingState.STARTED))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), newNode.getId(), null, true, ShardRoutingState.STARTED, allocationId2P))
.addShard(TestShardRouting.newShardRouting(shard2.getIndexName(), shard2.getId(), oldNode1.getId(), null, false, ShardRoutingState.STARTED, allocationId2R))
.build())
)
.build();
@ -385,27 +382,24 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
private ClusterState stabilize(ClusterState clusterState, AllocationService service) {
logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint());
RoutingTable routingTable = service.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = service.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertRecoveryNodeVersions(routingNodes);
logger.info("complete rebalancing");
RoutingTable prev = routingTable;
boolean stable = false;
for (int i = 0; i < 1000; i++) { // at most 200 iters - this should be enough for all tests
logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes().prettyPrint());
routingTable = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = service.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
if (stable = (routingTable == prev)) {
if (routingResult.changed() == false) {
break;
}
assertRecoveryNodeVersions(routingNodes);
prev = routingTable;
}
logger.info("stabilized success [{}]", stable);
assertThat(stable, is(true));
logger.info("stabilized success [{}]", routingResult.changed() == false);
assertFalse(routingResult.changed());
return clusterState;
}

View File

@ -57,43 +57,43 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(numberOfShards).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("adding two nodes and performing rerouting till all are allocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1", singletonMap("tag1", "value1")))
.add(newNode("node2", singletonMap("tag1", "value2")))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
logger.info("remove one of the nodes and apply filter to move everything from another node");
metaData = MetaData.builder()
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)
.put(IndexMetaData.builder(clusterState.metaData().index("test1")).settings(settings(Version.CURRENT)
.put("index.number_of_shards", numberOfShards)
.put("index.number_of_replicas", 0)
.put("index.routing.allocation.exclude.tag1", "value2")
.build()))
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)
.put(IndexMetaData.builder(clusterState.metaData().index("test2")).settings(settings(Version.CURRENT)
.put("index.number_of_shards", numberOfShards)
.put("index.number_of_replicas", 0)
.put("index.routing.allocation.exclude.tag1", "value2")
.build()))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build();
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("[{}] primaries should be still started but [{}] other primaries should be unassigned", numberOfShards, numberOfShards);
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(numberOfShards));
@ -103,8 +103,8 @@ public class PreferLocalPrimariesToRelocatingPrimariesTests extends ESAllocation
logger.info("start node back up");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node1", singletonMap("tag1", "value1")))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
while (clusterState.getRoutingNodes().shardsWithState(STARTED).size() < totalNumberOfShards) {
int localInitializations = 0;

View File

@ -54,30 +54,30 @@ public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test1"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("adding two nodes and performing rerouting till all are allocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
while (!clusterState.getRoutingNodes().shardsWithState(INITIALIZING).isEmpty()) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
logger.info("increasing the number of replicas to 1, and perform a reroute (to get the replicas allocation going)");
routingTable = RoutingTable.builder(routingTable).updateNumberOfReplicas(1).build();
RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()).updateNumberOfReplicas(1).build();
metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(1).build();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).metaData(metaData).build();
clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metaData(metaData).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("2 replicas should be initializing now for the existing indices (we throttle to 1)");
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
@ -87,15 +87,15 @@ public class PreferPrimaryAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("new_index").settings(settings(Version.CURRENT)).numberOfShards(4).numberOfReplicas(0))
.build();
routingTable = RoutingTable.builder(clusterState.routingTable())
updatedRoutingTable = RoutingTable.builder(clusterState.routingTable())
.addAsNew(metaData.index("new_index"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(updatedRoutingTable).build();
logger.info("reroute, verify that primaries for the new index primary shards are allocated");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.routingTable().index("new_index").shardsWithState(INITIALIZING).size(), equalTo(2));
}

View File

@ -54,38 +54,38 @@ public class PrimaryNotRelocatedWhileBeingRecoveredTests extends ESAllocationTes
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
logger.info("start another node, replica will start recovering form primary");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(5));
logger.info("start another node, make sure the primary is not relocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(5));
}
}

View File

@ -77,12 +77,13 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
routingTableBuilder.addAsNew(metaData.index("INDEX_" + i));
}
RoutingTable routingTable = routingTableBuilder.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
RoutingTable initialRoutingTable = routingTableBuilder.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
int numIters = scaledRandomIntBetween(5, 15);
int nodeIdCounter = 0;
int atMostNodes = scaledRandomIntBetween(Math.max(1, maxNumReplicas), 15);
final boolean frequentNodes = randomBoolean();
RoutingAllocation.Result routingResult;
for (int i = 0; i < numIters; i++) {
logger.info("Start iteration [{}]", i);
ClusterState.Builder stateBuilder = ClusterState.builder(clusterState);
@ -108,15 +109,14 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
stateBuilder.nodes(newNodesBuilder.build());
clusterState = stateBuilder.build();
if (nodesRemoved) {
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
} else {
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
routingResult = strategy.reroute(clusterState, "reroute");
}
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
}
logger.info("Fill up nodes such that every shard can be allocated");
@ -137,12 +137,11 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
int iterations = 0;
do {
iterations++;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
if (clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size() > 0) {
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
} while (clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size() != 0 ||

View File

@ -83,77 +83,73 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(5));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(5));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("start two nodes and fully start the shards");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
}
logger.info("start all the primary shards, replicas will start initializing");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertEquals(routingTable.index("test").shard(i).replicaShards().get(0).getExpectedShardSize(), sizes[i]);
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertEquals(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).getExpectedShardSize(), sizes[i]);
}
logger.info("now, start 8 more nodes, and check that no rebalancing/relocation have happened");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3")).add(newNode("node4")).add(newNode("node5")).add(newNode("node6")).add(newNode("node7")).add(newNode("node8")).add(newNode("node9")).add(newNode("node10")))
.build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertEquals(routingTable.index("test").shard(i).replicaShards().get(0).getExpectedShardSize(), sizes[i]);
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertEquals(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).getExpectedShardSize(), sizes[i]);
}
logger.info("start the replica shards, rebalancing should start");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(5));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
int num = 0;
for (ShardRouting routing : routingTable.index("test").shard(i).shards()) {
for (ShardRouting routing : clusterState.routingTable().index("test").shard(i).shards()) {
if (routing.state() == RELOCATING || routing.state() == INITIALIZING) {
assertEquals(routing.getExpectedShardSize(), sizes[i]);
num++;
@ -164,16 +160,15 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
logger.info("complete relocation, other half of relocation should happen");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
for (ShardRouting routing : routingTable.index("test").shard(i).shards()) {
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(7));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(3));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
for (ShardRouting routing : clusterState.routingTable().index("test").shard(i).shards()) {
if (routing.state() == RELOCATING || routing.state() == INITIALIZING) {
assertEquals(routing.getExpectedShardSize(), sizes[i]);
}
@ -183,12 +178,11 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
logger.info("complete relocation, that's it!");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10));
// make sure we have an even relocation
for (RoutingNode routingNode : routingNodes) {
assertThat(routingNode.size(), equalTo(1));

View File

@ -56,15 +56,14 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1))
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding three node and performing rerouting");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3"))).build();
routingNodes = clusterState.getRoutingNodes();
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
// all shards are unassigned. so no inactive shards or primaries.
@ -72,9 +71,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasInactivePrimaries(), equalTo(false));
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(true));
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -84,25 +82,21 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Another round of rebalancing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the more shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -110,9 +104,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasInactivePrimaries(), equalTo(false));
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false));
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingNodes = clusterState.getRoutingNodes();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
@ -128,72 +121,65 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1))
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).addAsNew(metaData.index("test1")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the primary shard");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
routingResult = strategy.reroute(clusterState, "reroute");
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(3));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(3));
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(4));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(4));
@ -220,9 +206,9 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
MetaData metaData = MetaData.builder().put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(1)).build();
RoutingTable routingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding three node and performing rerouting");
clusterState = ClusterState.builder(clusterState)
@ -234,9 +220,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasInactivePrimaries(), equalTo(false));
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(true));
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -246,20 +231,18 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Another round of rebalancing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
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));
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -271,15 +254,13 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.node("node3").numberOfShardsWithState(STARTED), equalTo(1));
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the more shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -297,17 +278,16 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Add new index 3 shards 1 replica");
prevRoutingTable = routingTable;
metaData = MetaData.builder(metaData)
metaData = MetaData.builder(clusterState.metaData())
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 3)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
))
.build();
routingTable = RoutingTable.builder(routingTable)
.addAsNew(metaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable())
.addAsNew(metaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(updatedRoutingTable).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -315,17 +295,15 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasInactivePrimaries(), equalTo(false));
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(true));
assertThat(routingTable.index("test1").shards().size(), equalTo(3));
assertThat(clusterState.routingTable().index("test1").shards().size(), equalTo(3));
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Reroute, assign");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -333,13 +311,12 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.hasInactivePrimaries(), equalTo(true));
assertThat(routingNodes.hasUnassignedPrimaries(), equalTo(false));
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
logger.info("Reroute, start the primaries");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -349,9 +326,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Reroute, start the replicas");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -369,10 +345,10 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
assertThat(routingNodes.node("node3").shardsWithState("test1", STARTED).size(), equalTo(2));
logger.info("kill one node");
IndexShardRoutingTable indexShardRoutingTable = routingTable.index("test").shard(0);
IndexShardRoutingTable indexShardRoutingTable = clusterState.routingTable().index("test").shard(0);
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove(indexShardRoutingTable.primaryShard().currentNodeId())).build();
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -383,9 +359,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Start Recovering shards round 1");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));
@ -395,9 +370,8 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
logger.info("Start Recovering shards round 2");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(assertShardStats(routingNodes), equalTo(true));

View File

@ -47,8 +47,8 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
private final ESLogger logger = Loggers.getLogger(SameShardRoutingTests.class);
public void testSameHost() {
AllocationService strategy = createAllocationService(Settings.builder()
.put(SameShardAllocationDecider.SAME_HOST_SETTING, true).build());
AllocationService strategy = createAllocationService(
Settings.builder().put(SameShardAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING.getKey(), true).build());
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(1))
@ -67,15 +67,14 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
MASTER_DATA_ROLES, Version.CURRENT))
.add(new DiscoveryNode("node2", "node2", "node2", "test1", "test1", LocalTransportAddress.buildUnique(), emptyMap(),
MASTER_DATA_ROLES, Version.CURRENT))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(0));
@ -84,8 +83,8 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(new DiscoveryNode("node3", "node3", "node3", "test2", "test2", LocalTransportAddress.buildUnique(), emptyMap(),
MASTER_DATA_ROLES, Version.CURRENT))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.STARTED), equalTo(2));
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), ShardRoutingState.INITIALIZING), equalTo(2));

View File

@ -64,16 +64,16 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(0));
@ -83,8 +83,8 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
logger.info("Do another reroute, make sure its still not allocated");
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
}
public void testClusterLevelShardsLimitAllocate() {
@ -108,16 +108,16 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1));
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(1));
@ -130,15 +130,15 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
.build());
logger.info("Do another reroute, make sure shards are now allocated");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.INITIALIZING), equalTo(1));
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(ShardRoutingState.STARTED), equalTo(2));
@ -164,44 +164,44 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("Adding one node and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start the primary shards");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(5));
logger.info("add another index with 5 shards");
metaData = MetaData.builder(metaData)
metaData = MetaData.builder(clusterState.metaData())
.put(IndexMetaData.builder("test1").settings(settings(Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
))
.build();
routingTable = RoutingTable.builder(routingTable)
.addAsNew(metaData.index("test1"))
.build();
RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable())
.addAsNew(metaData.index("test1"))
.build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).metaData(metaData).routingTable(updatedRoutingTable).build();
logger.info("Add another one node and reroute");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(numberOfShardsOfType(clusterState.getRoutingNodes(), STARTED), equalTo(10));
@ -213,8 +213,8 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
}
logger.info("update {} for test, see that things move", ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING.getKey());
metaData = MetaData.builder(metaData)
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)
metaData = MetaData.builder(clusterState.metaData())
.put(IndexMetaData.builder(clusterState.metaData().index("test")).settings(settings(Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 5)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), 3)
@ -225,8 +225,8 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).metaData(metaData).build();
logger.info("reroute after setting");
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(3));
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(RELOCATING), equalTo(2));
@ -235,8 +235,8 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
// 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.getRoutingNodes();
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// now we are done compared to EvenShardCountAllocator since the Balancer is not soely based on the average
assertThat(clusterState.getRoutingNodes().node("node1").numberOfShardsWithState(STARTED), equalTo(5));
assertThat(clusterState.getRoutingNodes().node("node2").numberOfShardsWithState(STARTED), equalTo(5));

View File

@ -67,96 +67,87 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metaData.index("test")).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
logger.info("Rerouting again, nothing should change");
prevRoutingTable = routingTable;
clusterState = ClusterState.builder(clusterState).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(routingTable == prevRoutingTable, equalTo(true));
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
assertThat(routingResult.changed(), equalTo(false));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Marking the shard as started");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable != prevRoutingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
assertThat(routingResult.changed(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
logger.info("Starting another node and making sure nothing changed");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable == prevRoutingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
assertThat(routingResult.changed(), equalTo(false));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
logger.info("Killing node1 where the shard is, checking the shard is relocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build();
prevRoutingTable = routingTable;
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable != prevRoutingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2"));
assertThat(routingResult.changed(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2"));
logger.info("Start another node, make sure that things remain the same (shard is in node2 and initializing)");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable == prevRoutingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingResult.changed(), equalTo(false));
logger.info("Start the shard on node 2");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable != prevRoutingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2"));
assertThat(routingResult.changed(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node2"));
}
public void testSingleIndexShardFailed() {
@ -168,44 +159,41 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable.Builder routingTableBuilder = RoutingTable.builder()
.addAsNew(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTableBuilder.build()).build();
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
logger.info("Adding one node and rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
assertThat(routingResult.changed(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), equalTo("node1"));
logger.info("Marking the shard as failed");
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();
routingResult = strategy.applyFailedShard(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING).get(0));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(routingResult.changed(), equalTo(true));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
}
public void testMultiIndexEvenDistribution() {
@ -228,16 +216,15 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
for (int i = 0; i < numberOfIndices; i++) {
routingTableBuilder.addAsNew(metaData.index("test" + i));
}
RoutingTable routingTable = routingTableBuilder.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTableBuilder.build()).build();
assertThat(routingTable.indicesRouting().size(), equalTo(numberOfIndices));
assertThat(clusterState.routingTable().indicesRouting().size(), equalTo(numberOfIndices));
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).currentNodeId(), nullValue());
}
logger.info("Adding " + (numberOfIndices / 2) + " nodes");
@ -246,21 +233,20 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
for (int i = 0; i < (numberOfIndices / 2); i++) {
nodesBuilder.add(newNode("node" + i));
}
RoutingTable prevRoutingTable = routingTable;
clusterState = ClusterState.builder(clusterState).nodes(nodesBuilder).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(true));
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).primary(), equalTo(true));
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).primary(), equalTo(true));
// make sure we still have 2 shards initializing per node on the first 25 nodes
String nodeId = routingTable.index("test" + i).shard(0).shards().get(0).currentNodeId();
String nodeId = clusterState.routingTable().index("test" + i).shard(0).shards().get(0).currentNodeId();
int nodeIndex = Integer.parseInt(nodeId.substring("node".length()));
assertThat(nodeIndex, lessThan(25));
}
@ -284,35 +270,33 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
for (int i = (numberOfIndices / 2); i < numberOfIndices; i++) {
nodesBuilder.add(newNode("node" + i));
}
prevRoutingTable = routingTable;
clusterState = ClusterState.builder(clusterState).nodes(nodesBuilder).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(false));
assertThat(routingResult.changed(), equalTo(false));
logger.info("Marking the shard as started");
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(true));
int numberOfRelocatingShards = 0;
int numberOfStartedShards = 0;
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(STARTED), equalTo(RELOCATING)));
if (routingTable.index("test" + i).shard(0).shards().get(0).state() == STARTED) {
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).unassigned(), equalTo(false));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(STARTED), equalTo(RELOCATING)));
if (clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state() == STARTED) {
numberOfStartedShards++;
} else if (routingTable.index("test" + i).shard(0).shards().get(0).state() == RELOCATING) {
} else if (clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state() == RELOCATING) {
numberOfRelocatingShards++;
}
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).primary(), equalTo(true));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).primary(), equalTo(true));
// make sure we still have 2 shards either relocating or started on the first 25 nodes (still)
String nodeId = routingTable.index("test" + i).shard(0).shards().get(0).currentNodeId();
String nodeId = clusterState.routingTable().index("test" + i).shard(0).shards().get(0).currentNodeId();
int nodeIndex = Integer.parseInt(nodeId.substring("node".length()));
assertThat(nodeIndex, lessThan(25));
}
@ -340,26 +324,24 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
for (int i = 0; i < numberOfIndices; i++) {
routingTableBuilder.addAsNew(metaData.index("test" + i));
}
RoutingTable routingTable = routingTableBuilder.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTableBuilder.build()).build();
assertThat(routingTable.indicesRouting().size(), equalTo(numberOfIndices));
assertThat(clusterState.routingTable().indicesRouting().size(), equalTo(numberOfIndices));
logger.info("Starting 3 nodes and rerouting");
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3")))
.build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(true));
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).shards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), equalTo(INITIALIZING));
}
RoutingNodes routingNodes = clusterState.getRoutingNodes();
assertThat(numberOfShardsOfType(routingNodes, INITIALIZING), equalTo(numberOfIndices));
@ -371,41 +353,36 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState)
.nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node4")).add(newNode("node5")))
.build();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(false));
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(true));
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
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)));
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(RELOCATING), equalTo(STARTED)));
}
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));
logger.info("Now, mark the relocated as started");
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// routingTable = strategy.reroute(new RoutingStrategyInfo(metaData, routingTable), nodes);
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingResult.changed(), equalTo(true));
for (int i = 0; i < numberOfIndices; i++) {
assertThat(routingTable.index("test" + i).shards().size(), equalTo(1));
assertThat(routingTable.index("test" + i).shard(0).size(), equalTo(1));
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)));
assertThat(clusterState.routingTable().index("test" + i).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test" + i).shard(0).shards().get(0).state(), anyOf(equalTo(RELOCATING), equalTo(STARTED)));
}
routingNodes = clusterState.getRoutingNodes();
assertThat(numberOfShardsOfType(routingNodes, STARTED), equalTo(numberOfIndices));

View File

@ -52,118 +52,111 @@ public class SingleShardOneReplicaRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(0).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(0).shards().get(1).currentNodeId(), nullValue());
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), nullValue());
logger.info("Add another node and perform rerouting, nothing will happen since primary shards not started");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they recover from primary *started* shards in the IndicesClusterStateService
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node2"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node2"));
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node2"));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node2"));
logger.info("Kill node1, backup shard should become primary");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).remove("node1")).build();
prevRoutingTable = routingTable;
routingTable = strategy.deassociateDeadNodes(clusterState, true, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.deassociateDeadNodes(clusterState, true, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they recover from primary *started* shards in the IndicesClusterStateService
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), nullValue());
logger.info("Start another node, backup shard should start initializing");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo("node2"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they recover from primary *started* shards in the IndicesClusterStateService
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node3"));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo("node3"));
}
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.AllocationId;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingTable;
@ -36,6 +37,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.cluster.ESAllocationTestCase;
import java.util.Arrays;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.nullValue;
@ -45,9 +47,11 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
AllocationService allocation = createAllocationService();
logger.info("--> building initial cluster state");
AllocationId allocationId = AllocationId.newRelocation(AllocationId.newInitializing());
final IndexMetaData indexMetaData = IndexMetaData.builder("test")
.settings(settings(Version.CURRENT))
.numberOfShards(2).numberOfReplicas(0)
.putActiveAllocationIds(1, Collections.singleton(allocationId.getId()))
.build();
final Index index = indexMetaData.getIndex();
ClusterState.Builder stateBuilder = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
@ -55,7 +59,7 @@ public class StartedShardsRoutingTests extends ESAllocationTestCase {
.metaData(MetaData.builder().put(indexMetaData, false));
final ShardRouting initShard = TestShardRouting.newShardRouting(new ShardId(index, 0), "node1", true, ShardRoutingState.INITIALIZING);
final ShardRouting relocatingShard = TestShardRouting.newShardRouting(new ShardId(index, 1), "node1", "node2", true, ShardRoutingState.RELOCATING);
final ShardRouting relocatingShard = TestShardRouting.newShardRouting(new ShardId(index, 1), "node1", "node2", true, ShardRoutingState.RELOCATING, allocationId);
stateBuilder.routingTable(RoutingTable.builder().add(IndexRoutingTable.builder(index)
.addIndexShard(new IndexShardRoutingTable.Builder(initShard.shardId()).addShard(initShard).build())
.addIndexShard(new IndexShardRoutingTable.Builder(relocatingShard.shardId()).addShard(relocatingShard).build())).build());

View File

@ -63,103 +63,97 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(10));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(0).currentNodeId(), nullValue());
assertThat(clusterState.routingTable().index("test").shard(i).shards().get(1).currentNodeId(), nullValue());
}
logger.info("Adding one node and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), nullValue());
}
logger.info("Add another node and perform rerouting, nothing will happen since primary not started");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable == routingTable, equalTo(true));
assertFalse(routingResult.changed());
logger.info("Start the primary shard (on node1)");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node1").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
// backup shards are initializing as well, we make sure that they recover from primary *started* shards in the IndicesClusterStateService
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), equalTo("node2"));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(INITIALIZING));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), equalTo("node2"));
}
logger.info("Reroute, nothing should change");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
assertThat(prevRoutingTable == routingTable, equalTo(true));
routingResult = strategy.reroute(clusterState, "reroute");
assertFalse(routingResult.changed());
logger.info("Start the backup shard");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node2").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
for (int i = 0; i < routingTable.index("test").shards().size(); i++) {
assertThat(routingTable.index("test").shard(i).size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(routingTable.index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(i).replicaShards().get(0).currentNodeId(), equalTo("node2"));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
for (int i = 0; i < clusterState.routingTable().index("test").shards().size(); i++) {
assertThat(clusterState.routingTable().index("test").shard(i).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).primaryShard().currentNodeId(), equalTo("node1"));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(i).replicaShards().get(0).currentNodeId(), equalTo("node2"));
}
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(10));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(10));
logger.info("Add another node and perform rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED, RELOCATING), equalTo(10));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), lessThan(10));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED, RELOCATING), equalTo(10));
@ -168,13 +162,12 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
logger.info("Start the shards on node 3");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.node("node3").shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.node("node3").shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingNodes = clusterState.getRoutingNodes();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(10));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(10));
assertThat(routingNodes.node("node1").numberOfShardsWithState(STARTED), equalTo(7));
assertThat(routingNodes.node("node2").numberOfShardsWithState(STARTED), equalTo(7));
assertThat(routingNodes.node("node3").numberOfShardsWithState(STARTED), equalTo(6));

View File

@ -61,50 +61,50 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(1))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start one node, do reroute, only 3 should initialize");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(17));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(17));
logger.info("start initializing, another 3 should initialize");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(3));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(14));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(14));
logger.info("start initializing, another 3 should initialize");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(6));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(11));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(11));
logger.info("start initializing, another 1 should initialize");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(9));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(9));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(10));
logger.info("start initializing, all primaries should be started");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(10));
}
public void testReplicaAndPrimaryRecoveryThrottling() {
@ -120,59 +120,59 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start one node, do reroute, only 3 should initialize");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(7));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(7));
logger.info("start initializing, another 2 should initialize");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(3));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(5));
logger.info("start initializing, all primaries should be started");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(5));
logger.info("start another node, replicas should start being allocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(2));
logger.info("start initializing replicas");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(8));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
logger.info("start initializing replicas, all should be started");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(10));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(10));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
}
public void testThrottleIncomingAndOutgoing() {
@ -188,52 +188,52 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(9).numberOfReplicas(0))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start one node, do reroute, only 5 should initialize");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(4));
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(4));
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 5);
logger.info("start initializing, all primaries should be started");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(4));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(4));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start another 2 nodes, 5 shards should be relocating - at most 5 are allowed per node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2")).add(newNode("node3"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(4));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(5));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(4));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(5));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 3);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 2);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 5);
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("start the relocating shards, one more shard should relocate away from node1");
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(8));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
@ -251,59 +251,59 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
RoutingTable initialRoutingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("start one node, do reroute, only 1 should initialize");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(2));
logger.info("start initializing");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(2));
logger.info("start one more node, first non-primary should start being allocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
logger.info("start initializing non-primary");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(1));
routingResult = strategy.applyStartedShards(clusterState, clusterState.routingTable().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(1));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 0);
logger.info("start one more node, initializing second non-primary");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
logger.info("start one more node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node4"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
@ -315,11 +315,10 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
// even though it is throttled, move command still forces allocation
clusterState = ClusterState.builder(clusterState).routingResult(reroute).build();
routingTable = clusterState.routingTable();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertThat(clusterState.routingTable().shardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(clusterState.routingTable().shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(clusterState.routingTable().shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 2);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
}

View File

@ -54,127 +54,118 @@ public class UpdateNumberOfReplicasTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(routingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(routingTable.index("test").shard(0).shards().get(1).currentNodeId(), nullValue());
assertThat(initialRoutingTable.index("test").shards().size(), equalTo(1));
assertThat(initialRoutingTable.index("test").shard(0).size(), equalTo(2));
assertThat(initialRoutingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(initialRoutingTable.index("test").shard(0).shards().get(0).state(), equalTo(UNASSIGNED));
assertThat(initialRoutingTable.index("test").shard(0).shards().get(1).state(), equalTo(UNASSIGNED));
assertThat(initialRoutingTable.index("test").shard(0).shards().get(0).currentNodeId(), nullValue());
assertThat(initialRoutingTable.index("test").shard(0).shards().get(1).currentNodeId(), nullValue());
logger.info("Adding two nodes and performing rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2"))).build();
RoutingTable prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start all the primary shards");
RoutingNodes routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("Start all the replica shards");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
final String nodeHoldingPrimary = routingTable.index("test").shard(0).primaryShard().currentNodeId();
final String nodeHoldingReplica = routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
final String nodeHoldingPrimary = clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId();
final String nodeHoldingReplica = clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId();
assertThat(nodeHoldingPrimary, not(equalTo(nodeHoldingReplica)));
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).shards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo(nodeHoldingReplica));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).shards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo(nodeHoldingReplica));
logger.info("add another replica");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = RoutingTable.builder(routingTable).updateNumberOfReplicas(2).build();
RoutingTable updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()).updateNumberOfReplicas(2).build();
metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(2).build();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).metaData(metaData).build();
clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metaData(metaData).build();
assertThat(clusterState.metaData().index("test").getNumberOfReplicas(), equalTo(2));
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(3));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo(nodeHoldingReplica));
assertThat(routingTable.index("test").shard(0).replicaShards().get(1).state(), equalTo(UNASSIGNED));
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), equalTo(nodeHoldingReplica));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(1).state(), equalTo(UNASSIGNED));
logger.info("Add another node and start the added replica");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).add(newNode("node3"))).build();
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(3));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).get(0).currentNodeId(), equalTo(nodeHoldingReplica));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(STARTED).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(STARTED).get(0).currentNodeId(), equalTo(nodeHoldingReplica));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(INITIALIZING).get(0).currentNodeId(), equalTo("node3"));
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, routingNodes.shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(3));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).get(0).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
assertThat(routingTable.index("test").shard(0).replicaShardsWithState(STARTED).get(1).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(3));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(STARTED).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(STARTED).get(0).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShardsWithState(STARTED).get(1).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
logger.info("now remove a replica");
routingNodes = clusterState.getRoutingNodes();
prevRoutingTable = routingTable;
routingTable = RoutingTable.builder(routingTable).updateNumberOfReplicas(1).build();
updatedRoutingTable = RoutingTable.builder(clusterState.routingTable()).updateNumberOfReplicas(1).build();
metaData = MetaData.builder(clusterState.metaData()).updateNumberOfReplicas(1).build();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).metaData(metaData).build();
clusterState = ClusterState.builder(clusterState).routingTable(updatedRoutingTable).metaData(metaData).build();
assertThat(clusterState.metaData().index("test").getNumberOfReplicas(), equalTo(1));
assertThat(prevRoutingTable != routingTable, equalTo(true));
assertThat(routingTable.index("test").shards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).size(), equalTo(2));
assertThat(routingTable.index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(routingTable.index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(routingTable.index("test").shard(0).replicaShards().get(0).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
assertTrue(routingResult.changed());
assertThat(clusterState.routingTable().index("test").shards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).size(), equalTo(2));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).primaryShard().currentNodeId(), equalTo(nodeHoldingPrimary));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().size(), equalTo(1));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).state(), equalTo(STARTED));
assertThat(clusterState.routingTable().index("test").shard(0).replicaShards().get(0).currentNodeId(), anyOf(equalTo(nodeHoldingReplica), equalTo("node3")));
logger.info("do a reroute, should remain the same");
prevRoutingTable = routingTable;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(prevRoutingTable != routingTable, equalTo(false));
assertFalse(routingResult.changed());
}
}

View File

@ -120,29 +120,28 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
final RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData)
.routingTable(routingTable).build();
.routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1"))
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Primary shard should be initializing, replica should not
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that we're able to start the primary
@ -151,9 +150,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that the replica couldn't be started since node1 doesn't have enough space
@ -164,8 +162,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that the replica is initialized now that node3 is available with enough space
@ -173,9 +171,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that the replica couldn't be started since node1 doesn't have enough space
@ -205,8 +202,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -236,8 +233,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -252,8 +249,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -261,9 +258,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(0));
@ -321,12 +317,12 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData)
.routingTable(routingTable).build();
.routingTable(initialRoutingTable).build();
logger.info("--> adding node1 and node2 node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
@ -334,8 +330,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Primary should initialize, even though both nodes are over the limit initialize
@ -375,17 +371,16 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Now the replica should be able to initialize
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(2));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that we're able to start the primary and replica, since they were both initializing
@ -403,8 +398,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node3"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that the replica is initialized now that node3 is available with enough space
@ -412,9 +407,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that all replicas could be started
@ -444,8 +438,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -475,8 +469,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, NoopGatewayAllocator.INSTANCE, new BalancedShardsAllocator(Settings.EMPTY), cis);
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -491,8 +485,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node4"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started
@ -502,9 +496,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// primary shard already has been relocated away
@ -519,8 +512,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes())
.add(newNode("node5"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Shards remain started on node3 and node4
@ -531,9 +524,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> apply INITIALIZING shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> final cluster state:");
logShardStates(clusterState);
@ -766,30 +758,29 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test2").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.addAsNew(metaData.index("test2"))
.build();
ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData)
.routingTable(routingTable).build();
.routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1"))
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// shards should be initializing
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
logger.info("--> start the shards");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING))
.routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
// Assert that we're able to start the primary and replicas
@ -803,8 +794,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationCommand relocate1 = new MoveAllocationCommand("test", 0, "node2", "node3");
AllocationCommands cmds = new AllocationCommands(relocate1);
routingTable = strategy.reroute(clusterState, cmds, false, false).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, cmds, false, false);
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logShardStates(clusterState);
AllocationCommand relocate2 = new MoveAllocationCommand("test2", 0, "node2", "node3");
@ -850,7 +841,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
@ -862,7 +853,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
ClusterState baseClusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
.metaData(metaData)
.routingTable(routingTable)
.routingTable(initialRoutingTable)
.nodes(discoveryNodes)
.build();
@ -965,7 +956,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
@ -978,7 +969,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(discoveryNode1).add(discoveryNode2).build();
ClusterState baseClusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY))
.metaData(metaData)
.routingTable(routingTable)
.routingTable(initialRoutingTable)
.nodes(discoveryNodes)
.build();

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Allocation;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Rebalance;
import org.elasticsearch.common.logging.ESLogger;
@ -125,27 +126,27 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("enabled").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("disabled"))
.addAsNew(metaData.index("enabled"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding two nodes and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1"))
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(1));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> start the shards (replicas)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
logger.info("--> verify only enabled index has been routed");
assertThat(clusterState.getRoutingNodes().shardsWithState("enabled", STARTED).size(), equalTo(2));
@ -170,29 +171,29 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("always_disabled").settings(settings(Version.CURRENT).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), Rebalance.NONE)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.addAsNew(metaData.index("always_disabled"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding one nodes and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1"))
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(4));
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
@ -202,29 +203,26 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.add(newNode("node2"))
.add(newNode("node3"))
).build();
ClusterState prevState = clusterState;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
if (useClusterSetting) {
prevState = clusterState;
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(Settings.builder()
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(clusterState.metaData()).transientSettings(Settings.builder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), allowedOnes)
.build())).build();
} else {
prevState = clusterState;
IndexMetaData meta = clusterState.getMetaData().index("test");
IndexMetaData meta1 = clusterState.getMetaData().index("always_disabled");
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).removeAllIndices().put(IndexMetaData.builder(meta1))
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(clusterState.metaData()).removeAllIndices().put(IndexMetaData.builder(meta1))
.put(IndexMetaData.builder(meta).settings(Settings.builder().put(meta.getSettings()).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), allowedOnes).build())))
.build();
}
clusterSettings.applySettings(clusterState.metaData().settings());
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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);
@ -249,8 +247,8 @@ public class EnableAllocationTests extends ESAllocationTestCase {
default:
fail("only replicas, primaries or all are allowed");
}
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(8));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
@ -271,23 +269,23 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT).put(indexSettings)).numberOfShards(6).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
RoutingTable initialRoutingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(routingTable).build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).metaData(metaData).routingTable(initialRoutingTable).build();
logger.info("--> adding one nodes and do rerouting");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.add(newNode("node1"))
.add(newNode("node2"))
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(6));
logger.info("--> start the shards (primaries)");
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.getRoutingNodes().shardsWithState(INITIALIZING).size(), equalTo(0));
@ -297,25 +295,23 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.add(newNode("node2"))
.add(newNode("node3"))
).build();
ClusterState prevState = clusterState;
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
assertThat(clusterState.getRoutingNodes().shardsWithState(RELOCATING).size(), equalTo(0));
metaData = clusterState.metaData();
if (useClusterSetting) {
prevState = clusterState;
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(Settings.builder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL)
.build())).build();
} else {
prevState = clusterState;
IndexMetaData meta = clusterState.getMetaData().index("test");
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).removeAllIndices()
.put(IndexMetaData.builder(meta).settings(Settings.builder().put(meta.getSettings()).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL).build()))).build();
}
clusterSettings.applySettings(clusterState.metaData().settings());
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
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

@ -35,6 +35,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardShuffler;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
@ -250,14 +251,14 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
.add(newNode("node2", unmodifiableMap(node2Attributes)))
.localNodeId("node1")
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// after all are started, check routing iteration
ShardIterator shardIterator = clusterState.routingTable().index("test").shard(0).preferAttributesActiveInitializingShardsIt(new String[]{"rack_id"}, clusterState.nodes());
@ -299,11 +300,11 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
.localNodeId("node1")
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
ShardsIterator shardsIterator = clusterState.routingTable().index("test").shard(0).onlyNodeSelectorActiveInitializingShardsIt("disk:ebs",clusterState.nodes());
assertThat(shardsIterator.size(), equalTo(1));
@ -372,14 +373,14 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
.add(newNode("node2"))
.localNodeId("node1")
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
OperationRouting operationRouting = new OperationRouting(Settings.EMPTY, new ClusterSettings(Settings.EMPTY,
ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
@ -446,11 +447,11 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
.add(newNode("node3"))
.localNodeId("node1")
).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
RoutingAllocation.Result routingResult = strategy.reroute(clusterState, "reroute");
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
// When replicas haven't initialized, it comes back with the primary first, then initializing replicas
GroupShardsIterator shardIterators = operationRouting.searchShards(clusterState, new String[]{"test"}, null, "_replica_first");
@ -471,11 +472,11 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
assertFalse(routing.primary());
assertTrue(routing.initializing());
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
routingTable = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
routingResult = strategy.applyStartedShards(clusterState, clusterState.getRoutingNodes().shardsWithState(INITIALIZING));
clusterState = ClusterState.builder(clusterState).routingResult(routingResult).build();
shardIterators = operationRouting.searchShards(clusterState, new String[]{"test"}, null, "_replica");

View File

@ -80,6 +80,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
import static org.elasticsearch.cluster.routing.RoutingTableTests.updateActiveAllocations;
import static org.elasticsearch.test.ClusterServiceUtils.createClusterService;
import static org.elasticsearch.test.ClusterServiceUtils.setState;
import static org.elasticsearch.cluster.ESAllocationTestCase.createAllocationService;
@ -596,7 +597,6 @@ public class NodeJoinControllerTests extends ESTestCase {
.put(SETTING_VERSION_CREATED, Version.CURRENT)
.put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 1)
.put(SETTING_CREATION_DATE, System.currentTimeMillis())).build();
stateBuilder.metaData(MetaData.builder().put(indexMetaData, false).generateClusterUuidIfNeeded());
IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(indexMetaData.getIndex());
RoutingTable.Builder routing = new RoutingTable.Builder();
routing.addAsNew(indexMetaData);
@ -619,7 +619,10 @@ public class NodeJoinControllerTests extends ESTestCase {
ShardRoutingState.UNASSIGNED, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "life sucks")));
}
indexRoutingTableBuilder.addIndexShard(indexShardRoutingBuilder.build());
stateBuilder.routingTable(RoutingTable.builder().add(indexRoutingTableBuilder.build()).build());
IndexRoutingTable indexRoutingTable = indexRoutingTableBuilder.build();
IndexMetaData updatedIndexMetaData = updateActiveAllocations(indexRoutingTable, indexMetaData);
stateBuilder.metaData(MetaData.builder().put(updatedIndexMetaData, false).generateClusterUuidIfNeeded())
.routingTable(RoutingTable.builder().add(indexRoutingTable).build());
}
setState(clusterService, stateBuilder.build());

View File

@ -0,0 +1,58 @@
/*
* 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.engine;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.RamUsageTester;
import org.apache.lucene.util.TestUtil;
import org.elasticsearch.test.ESTestCase;
public class LiveVersionMapTests extends ESTestCase {
public void testRamBytesUsed() throws Exception {
LiveVersionMap map = new LiveVersionMap();
for (int i = 0; i < 100000; ++i) {
BytesRefBuilder uid = new BytesRefBuilder();
uid.copyChars(TestUtil.randomSimpleString(random(), 10, 20));
VersionValue version = new VersionValue(randomLong(), null);
map.putUnderLock(uid.toBytesRef(), version);
}
long actualRamBytesUsed = RamUsageTester.sizeOf(map);
long estimatedRamBytesUsed = map.ramBytesUsed();
// less than 25% off
assertEquals(actualRamBytesUsed, estimatedRamBytesUsed, actualRamBytesUsed / 4);
// now refresh
map.beforeRefresh();
map.afterRefresh(true);
for (int i = 0; i < 100000; ++i) {
BytesRefBuilder uid = new BytesRefBuilder();
uid.copyChars(TestUtil.randomSimpleString(random(), 10, 20));
VersionValue version = new VersionValue(randomLong(), null);
map.putUnderLock(uid.toBytesRef(), version);
}
actualRamBytesUsed = RamUsageTester.sizeOf(map);
estimatedRamBytesUsed = map.ramBytesUsed();
// less than 25% off
assertEquals(actualRamBytesUsed, estimatedRamBytesUsed, actualRamBytesUsed / 4);
}
}

View File

@ -0,0 +1,45 @@
/*
* 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.engine;
import org.apache.lucene.util.RamUsageTester;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogTests;
import org.elasticsearch.test.ESTestCase;
public class VersionValueTests extends ESTestCase {
public void testRamBytesUsed() {
VersionValue versionValue = new VersionValue(randomLong(), null);
assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed());
Translog.Location location = TranslogTests.randomTranslogLocation();
versionValue = new VersionValue(randomLong(), location);
assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed());
}
public void testDeleteRamBytesUsed() {
DeleteVersionValue versionValue = new DeleteVersionValue(randomLong(), randomLong(), null);
assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed());
Translog.Location location = TranslogTests.randomTranslogLocation();
versionValue = new DeleteVersionValue(randomLong(), randomLong(), location);
assertEquals(RamUsageTester.sizeOf(versionValue), versionValue.ramBytesUsed());
}
}

View File

@ -71,10 +71,6 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
qb.setValidationMethod(randomFrom(GeoValidationMethod.values()));
}
if (randomBoolean()) {
qb.optimizeBbox(randomFrom("none", "memory", "indexed"));
}
if (randomBoolean()) {
qb.geoDistance(randomFrom(GeoDistance.values()));
}
@ -118,9 +114,6 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
e = expectThrows(IllegalArgumentException.class, () -> query.geoDistance(null));
assertEquals("geoDistance must not be null", e.getMessage());
e = expectThrows(IllegalArgumentException.class, () -> query.optimizeBbox(null));
assertEquals("optimizeBbox must not be null", e.getMessage());
}
/**
@ -368,7 +361,6 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
" \"pin.location\" : [ -70.0, 40.0 ],\n" +
" \"distance\" : 12000.0,\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"validation_method\" : \"STRICT\",\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
@ -381,6 +373,23 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
assertEquals(json, 12000.0, parsed.distance(), 0.0001);
}
public void testOptimizeBboxFails() throws IOException {
String json =
"{\n" +
" \"geo_distance\" : {\n" +
" \"pin.location\" : [ -70.0, 40.0 ],\n" +
" \"distance\" : 12000.0,\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"validation_method\" : \"STRICT\",\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
"}";
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> parseQuery(json));
assertTrue(e.getMessage().startsWith("Deprecated field "));
}
public void testFromCoerceFails() throws IOException {
String json =
"{\n" +
@ -388,7 +397,6 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
" \"pin.location\" : [ -70.0, 40.0 ],\n" +
" \"distance\" : 12000.0,\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"coerce\" : true,\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
@ -405,7 +413,6 @@ public class GeoDistanceQueryBuilderTests extends AbstractQueryTestCase<GeoDista
" \"pin.location\" : [ -70.0, 40.0 ],\n" +
" \"distance\" : 12000.0,\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"ignore_malformed\" : true,\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +

View File

@ -46,7 +46,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
@Override
protected GeoDistanceRangeQueryBuilder doCreateTestQueryBuilder() {
Version version = createShardContext().indexVersionCreated();
GeoDistanceRangeQueryBuilder builder;
GeoPoint randomPoint = RandomGeoGenerator.randomPointIn(random(), -180.0, -89.9, 180.0, 89.9);
if (randomBoolean()) {
@ -106,9 +105,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
if (randomBoolean()) {
builder.geoDistance(randomFrom(GeoDistance.values()));
}
if (randomBoolean() && version.before(Version.V_2_2_0)) {
builder.optimizeBbox(randomFrom("none", "memory", "indexed"));
}
builder.unit(fromToUnits);
if (randomBoolean()) {
builder.setValidationMethod(randomFrom(GeoValidationMethod.values()));
@ -245,14 +241,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
assertEquals("[to] must not be null", e.getMessage());
}
public void testInvalidOptimizeBBox() {
GeoDistanceRangeQueryBuilder builder = new GeoDistanceRangeQueryBuilder(GEO_POINT_FIELD_NAME, new GeoPoint());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> builder.optimizeBbox(null));
assertEquals("optimizeBbox must not be null", e.getMessage());
e = expectThrows(IllegalArgumentException.class, () -> builder.optimizeBbox("foo"));
assertEquals("optimizeBbox must be one of [none, memory, indexed]", e.getMessage());
}
public void testInvalidGeoDistance() {
GeoDistanceRangeQueryBuilder builder = new GeoDistanceRangeQueryBuilder(GEO_POINT_FIELD_NAME, new GeoPoint());
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> builder.geoDistance(null));
@ -306,7 +294,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
" \"include_upper\" : true,\n" +
" \"unit\" : \"m\",\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"validation_method\" : \"STRICT\",\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
@ -317,6 +304,26 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
assertEquals(json, -70.0, parsed.point().lon(), 0.0001);
}
public void testFromJsonOptimizeBboxFails() throws IOException {
String json =
"{\n" +
" \"geo_distance_range\" : {\n" +
" \"pin.location\" : [ -70.0, 40.0 ],\n" +
" \"from\" : \"200km\",\n" +
" \"to\" : \"400km\",\n" +
" \"include_lower\" : true,\n" +
" \"include_upper\" : true,\n" +
" \"unit\" : \"m\",\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
"}";
IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> parseQuery(json));
assertTrue(e.getMessage().startsWith("Deprecated field "));
}
public void testFromJsonCoerceFails() throws IOException {
String json =
"{\n" +
@ -328,7 +335,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
" \"include_upper\" : true,\n" +
" \"unit\" : \"m\",\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"coerce\" : true,\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +
@ -349,7 +355,6 @@ public class GeoDistanceRangeQueryTests extends AbstractQueryTestCase<GeoDistanc
" \"include_upper\" : true,\n" +
" \"unit\" : \"m\",\n" +
" \"distance_type\" : \"sloppy_arc\",\n" +
" \"optimize_bbox\" : \"memory\",\n" +
" \"ignore_malformed\" : true,\n" +
" \"ignore_unmapped\" : false,\n" +
" \"boost\" : 1.0\n" +

View File

@ -29,6 +29,7 @@ import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LineFileDocs;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.RamUsageTester;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesArray;
@ -1925,4 +1926,13 @@ public class TranslogTests extends ESTestCase {
IOUtils.close(view);
translog = new Translog(config, generation);
}
public static Translog.Location randomTranslogLocation() {
return new Translog.Location(randomLong(), randomLong(), randomInt());
}
public void testLocationRamBytesUsed() {
Translog.Location location = randomTranslogLocation();
assertEquals(RamUsageTester.sizeOf(location), location.ramBytesUsed());
}
}

View File

@ -165,7 +165,7 @@ public class ClusterStateChanges extends AbstractComponent {
MetaDataUpdateSettingsService metaDataUpdateSettingsService = new MetaDataUpdateSettingsService(settings, clusterService,
allocationService, IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, indicesService, nodeServicesProvider);
MetaDataCreateIndexService createIndexService = new MetaDataCreateIndexService(settings, clusterService, indicesService,
allocationService, new AliasValidator(settings), Collections.emptySet(), environment,
allocationService, new AliasValidator(settings), environment,
nodeServicesProvider, IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, threadPool);
transportCloseIndexAction = new TransportCloseIndexAction(settings, transportService, clusterService, threadPool,

View File

@ -1,83 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.indices.template;
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsResponse;
import org.elasticsearch.cluster.ClusterModule;
import org.elasticsearch.cluster.metadata.IndexTemplateFilter;
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope;
import java.util.Arrays;
import java.util.Collection;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.core.IsNull.notNullValue;
@ClusterScope(scope = Scope.SUITE)
public class IndexTemplateFilteringIT extends ESIntegTestCase {
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return Arrays.asList(TestPlugin.class);
}
public void testTemplateFiltering() throws Exception {
client().admin().indices().preparePutTemplate("template1")
.setTemplate("test*")
.addMapping("type1", "field1", "type=text").get();
client().admin().indices().preparePutTemplate("template2")
.setTemplate("test*")
.addMapping("type2", "field2", "type=text").get();
client().admin().indices().preparePutTemplate("template3")
.setTemplate("no_match")
.addMapping("type3", "field3", "type=text").get();
assertAcked(prepareCreate("test"));
GetMappingsResponse response = client().admin().indices().prepareGetMappings("test").get();
assertThat(response, notNullValue());
ImmutableOpenMap<String, MappingMetaData> metadata = response.getMappings().get("test");
assertThat(metadata.size(), is(1));
assertThat(metadata.get("type2"), notNullValue());
}
public static class TestFilter implements IndexTemplateFilter {
@Override
public boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template) {
//make sure that no_match template is filtered out before the custom filters as it doesn't match the index name
return (template.name().equals("template2") || template.name().equals("no_match"));
}
}
public static class TestPlugin extends Plugin {
public void onModule(ClusterModule module) {
module.registerIndexTemplateFilter(TestFilter.class);
}
}
}

View File

@ -32,12 +32,12 @@ import static org.hamcrest.Matchers.equalTo;
public class ScriptSettingsTests extends ESTestCase {
public void testDefaultLanguageIsGroovy() {
public void testDefaultLanguageIsPainless() {
ScriptEngineRegistry scriptEngineRegistry =
new ScriptEngineRegistry(Collections.singletonList(new CustomScriptEngineService()));
ScriptContextRegistry scriptContextRegistry = new ScriptContextRegistry(Collections.emptyList());
ScriptSettings scriptSettings = new ScriptSettings(scriptEngineRegistry, scriptContextRegistry);
assertThat(scriptSettings.getDefaultScriptLanguageSetting().get(Settings.EMPTY), equalTo("groovy"));
assertThat(scriptSettings.getDefaultScriptLanguageSetting().get(Settings.EMPTY), equalTo("painless"));
}
public void testCustomDefaultLanguage() {

View File

@ -18,10 +18,15 @@
*/
package org.elasticsearch.search;
import com.carrotsearch.hppc.IntArrayList;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -34,12 +39,19 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.search.fetch.ShardFetchRequest;
import org.elasticsearch.search.internal.ShardSearchLocalRequest;
import org.elasticsearch.search.query.QuerySearchResultProvider;
import org.elasticsearch.test.ESSingleNodeTestCase;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import static java.util.Collections.singletonList;
import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE;
@ -112,6 +124,75 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
assertEquals(activeRefs, indexShard.store().refCount());
}
public void testSearchWhileIndexDeleted() throws IOException, InterruptedException {
createIndex("index");
client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get();
SearchService service = getInstanceFromNode(SearchService.class);
IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index"));
IndexShard indexShard = indexService.getShard(0);
AtomicBoolean running = new AtomicBoolean(true);
CountDownLatch startGun = new CountDownLatch(1);
Semaphore semaphore = new Semaphore(Integer.MAX_VALUE);
final Thread thread = new Thread() {
@Override
public void run() {
startGun.countDown();
while(running.get()) {
service.afterIndexDeleted(indexService.index(), indexService.getIndexSettings().getSettings());
if (randomBoolean()) {
// here we trigger some refreshes to ensure the IR go out of scope such that we hit ACE if we access a search
// context in a non-sane way.
try {
semaphore.acquire();
} catch (InterruptedException e) {
throw new AssertionError(e);
}
client().prepareIndex("index", "type").setSource("field", "value")
.setRefreshPolicy(randomFrom(WriteRequest.RefreshPolicy.values())).execute(new ActionListener<IndexResponse>() {
@Override
public void onResponse(IndexResponse indexResponse) {
semaphore.release();
}
@Override
public void onFailure(Exception e) {
semaphore.release();
}
});
}
}
}
};
thread.start();
startGun.await();
try {
final int rounds = scaledRandomIntBetween(100, 10000);
for (int i = 0; i < rounds; i++) {
try {
QuerySearchResultProvider querySearchResultProvider = service.executeQueryPhase(
new ShardSearchLocalRequest(indexShard.shardId(), 1, SearchType.DEFAULT,
new SearchSourceBuilder(), new String[0], false));
IntArrayList intCursors = new IntArrayList(1);
intCursors.add(0);
ShardFetchRequest req = new ShardFetchRequest(querySearchResultProvider.id(), intCursors, null /* not a scroll */);
service.executeFetchPhase(req);
} catch (AlreadyClosedException ex) {
throw ex;
} catch (IllegalStateException ex) {
assertEquals("search context is already closed can't increment refCount current count [0]", ex.getMessage());
} catch (SearchContextMissingException ex) {
// that's fine
}
}
} finally {
running.set(false);
thread.join();
semaphore.acquire(Integer.MAX_VALUE);
}
}
public static class FailOnRewriteQueryPlugin extends Plugin implements SearchPlugin {
@Override
public List<QuerySpec<?>> getQueries() {

View File

@ -28,9 +28,11 @@ import org.elasticsearch.script.MockScriptPlugin;
import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.aggregations.bucket.histogram.Histogram;
import org.elasticsearch.search.aggregations.bucket.range.Range;
import org.elasticsearch.search.aggregations.metrics.sum.Sum;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.test.ESIntegTestCase;
import org.joda.time.DateTime;
import java.io.IOException;
import java.util.ArrayList;
@ -42,6 +44,7 @@ import java.util.Map;
import java.util.function.Function;
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
import static org.elasticsearch.search.aggregations.AggregationBuilders.dateRange;
import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram;
import static org.elasticsearch.search.aggregations.AggregationBuilders.sum;
import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorBuilders.bucketScript;
@ -58,11 +61,13 @@ public class BucketScriptIT extends ESIntegTestCase {
private static final String FIELD_2_NAME = "field2";
private static final String FIELD_3_NAME = "field3";
private static final String FIELD_4_NAME = "field4";
private static final String FIELD_5_NAME = "field5";
private static int interval;
private static int numDocs;
private static int minNumber;
private static int maxNumber;
private static long date;
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
@ -125,6 +130,7 @@ public class BucketScriptIT extends ESIntegTestCase {
numDocs = randomIntBetween(10, 500);
minNumber = -200;
maxNumber = 200;
date = randomLong();
List<IndexRequestBuilder> builders = new ArrayList<>();
for (int docs = 0; docs < numDocs; docs++) {
@ -142,6 +148,7 @@ public class BucketScriptIT extends ESIntegTestCase {
jsonBuilder.field(FIELD_2_NAME, randomIntBetween(minNumber, maxNumber));
jsonBuilder.field(FIELD_3_NAME, randomIntBetween(minNumber, maxNumber));
jsonBuilder.field(FIELD_4_NAME, randomIntBetween(minNumber, maxNumber));
jsonBuilder.field(FIELD_5_NAME, date);
jsonBuilder.endObject();
return jsonBuilder;
}
@ -238,6 +245,52 @@ public class BucketScriptIT extends ESIntegTestCase {
}
}
public void testInlineScriptWithDateRange() {
SearchResponse response = client()
.prepareSearch("idx")
.addAggregation(
dateRange("range")
.field(FIELD_5_NAME)
.addUnboundedFrom(date)
.subAggregation(sum("field2Sum").field(FIELD_2_NAME))
.subAggregation(sum("field3Sum").field(FIELD_3_NAME))
.subAggregation(sum("field4Sum").field(FIELD_4_NAME))
.subAggregation(
bucketScript("seriesArithmetic",
new Script("_value0 + _value1 + _value2", ScriptType.INLINE, CustomScriptPlugin.NAME, null)
, "field2Sum", "field3Sum", "field4Sum")))
.execute().actionGet();
assertSearchResponse(response);
Range range = response.getAggregations().get("range");
assertThat(range, notNullValue());
assertThat(range.getName(), equalTo("range"));
List<? extends Range.Bucket> buckets = range.getBuckets();
for (int i = 0; i < buckets.size(); ++i) {
Range.Bucket bucket = buckets.get(i);
if (bucket.getDocCount() == 0) {
SimpleValue seriesArithmetic = bucket.getAggregations().get("seriesArithmetic");
assertThat(seriesArithmetic, nullValue());
} else {
Sum field2Sum = bucket.getAggregations().get("field2Sum");
assertThat(field2Sum, notNullValue());
double field2SumValue = field2Sum.getValue();
Sum field3Sum = bucket.getAggregations().get("field3Sum");
assertThat(field3Sum, notNullValue());
double field3SumValue = field3Sum.getValue();
Sum field4Sum = bucket.getAggregations().get("field4Sum");
assertThat(field4Sum, notNullValue());
double field4SumValue = field4Sum.getValue();
SimpleValue seriesArithmetic = bucket.getAggregations().get("seriesArithmetic");
assertThat(seriesArithmetic, notNullValue());
double seriesArithmeticValue = seriesArithmetic.value();
assertThat(seriesArithmeticValue, equalTo(field2SumValue + field3SumValue + field4SumValue));
}
}
}
public void testInlineScriptSingleVariable() {
SearchResponse response = client()
.prepareSearch("idx")

View File

@ -1504,12 +1504,24 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
logger.info("--> checking that _current no longer returns the snapshot");
assertThat(client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("_current").execute().actionGet().getSnapshots().isEmpty(), equalTo(true));
try {
client.admin().cluster().prepareSnapshotStatus("test-repo").addSnapshots("test-snap-doesnt-exist").execute().actionGet();
fail();
} catch (SnapshotMissingException ex) {
// Expected
}
// test that getting an unavailable snapshot status throws an exception if ignoreUnavailable is false on the request
SnapshotMissingException ex = expectThrows(SnapshotMissingException.class, () ->
client.admin().cluster().prepareSnapshotStatus("test-repo").addSnapshots("test-snap-doesnt-exist").get());
assertEquals("[test-repo:test-snap-doesnt-exist] is missing", ex.getMessage());
// test that getting an unavailable snapshot status does not throw an exception if ignoreUnavailable is true on the request
response = client.admin().cluster().prepareSnapshotStatus("test-repo")
.addSnapshots("test-snap-doesnt-exist")
.setIgnoreUnavailable(true)
.get();
assertTrue(response.getSnapshots().isEmpty());
// test getting snapshot status for available and unavailable snapshots where ignoreUnavailable is true
// (available one should be returned)
response = client.admin().cluster().prepareSnapshotStatus("test-repo")
.addSnapshots("test-snap", "test-snap-doesnt-exist")
.setIgnoreUnavailable(true)
.get();
assertEquals(1, response.getSnapshots().size());
assertEquals("test-snap", response.getSnapshots().get(0).getSnapshot().getSnapshotId().getName());
}
public void testSnapshotRelocatingPrimary() throws Exception {

View File

@ -479,6 +479,8 @@ task run(type: RunTask) {
* </dl>
*/
Map<String, String> expansionsForDistribution(distributionType) {
final String defaultHeapSize = "2g"
String footer = "# Built for ${project.name}-${project.version} " +
"(${distributionType})"
Map<String, Object> expansions = [
@ -498,8 +500,8 @@ Map<String, String> expansionsForDistribution(distributionType) {
'def': '',
],
'heap.min': "256m",
'heap.max': "2g",
'heap.min': defaultHeapSize,
'heap.max': defaultHeapSize,
'stopping.timeout': [
'rpm': 86400,

View File

@ -14,33 +14,33 @@
#
# Use a descriptive name for your cluster:
#
# cluster.name: my-application
#cluster.name: my-application
#
# ------------------------------------ Node ------------------------------------
#
# Use a descriptive name for the node:
#
# node.name: node-1
#node.name: node-1
#
# Add custom attributes to the node:
#
# node.rack: r1
#node.rack: r1
#
# ----------------------------------- Paths ------------------------------------
#
# Path to directory where to store the data (separate multiple locations by comma):
#
# path.data: /path/to/data
#path.data: /path/to/data
#
# Path to log files:
#
# path.logs: /path/to/logs
#path.logs: /path/to/logs
#
# ----------------------------------- Memory -----------------------------------
#
# Lock the memory on startup:
#
# bootstrap.memory_lock: true
#bootstrap.memory_lock: true
#
# Make sure that the heap size is set to about half the memory available
# on the system and that the owner of the process is allowed to use this
@ -52,11 +52,11 @@
#
# Set the bind address to a specific IP (IPv4 or IPv6):
#
# network.host: 192.168.0.1
#network.host: 192.168.0.1
#
# Set a custom port for HTTP:
#
# http.port: 9200
#http.port: 9200
#
# For more information, see the documentation at:
# <http://www.elastic.co/guide/en/elasticsearch/reference/current/modules-network.html>
@ -66,11 +66,11 @@
# Pass an initial list of hosts to perform discovery when new node is started:
# The default list of hosts is ["127.0.0.1", "[::1]"]
#
# discovery.zen.ping.unicast.hosts: ["host1", "host2"]
#discovery.zen.ping.unicast.hosts: ["host1", "host2"]
#
# Prevent the "split brain" by configuring the majority of nodes (total number of nodes / 2 + 1):
#
# discovery.zen.minimum_master_nodes: 3
#discovery.zen.minimum_master_nodes: 3
#
# For more information, see the documentation at:
# <http://www.elastic.co/guide/en/elasticsearch/reference/current/modules-discovery.html>
@ -79,7 +79,7 @@
#
# Block initial recovery after a full cluster restart until N nodes are started:
#
# gateway.recover_after_nodes: 3
#gateway.recover_after_nodes: 3
#
# For more information, see the documentation at:
# <http://www.elastic.co/guide/en/elasticsearch/reference/current/modules-gateway.html>
@ -88,8 +88,8 @@
#
# Disable starting multiple nodes on a single system:
#
# node.max_local_storage_nodes: 1
#node.max_local_storage_nodes: 1
#
# Require explicit names when deleting indices:
#
# action.destructive_requires_name: true
#action.destructive_requires_name: true

View File

@ -11,14 +11,28 @@ manner. They range from adding custom mapping types, custom analyzers, native
scripts, custom discovery and more.
Plugins contain JAR files, but may also contain scripts and config files, and
must be installed on every node in the cluster. After installation, each
must be installed on every node in the cluster. After installation, each
node must be restarted before the plugin becomes visible.
This documentation distinguishes two categories of plugins:
Core Plugins:: This category identifies plugins that are part of Elasticsearch
project. Delivered at the same time as Elasticsearch, their version number always
matches the version number of Elasticsearch itself. These plugins are maintained
by the Elastic team with the appreciated help of amazing community members (for
open source plugins). Issues and bug reports can be reported on the
https://github.com/elastic/elasticsearch[Github project page].
Community contributed:: This category identifies plugins that are external to
the Elasticsearch project. They are provided by individual developers or private
companies and have their own licenses as well as their own versioning system.
Issues and bug reports can usually be reported on the community plugin's web site.
For advice on writing your own plugin, see <<plugin-authors>>.
IMPORTANT: Site plugins -- plugins containing HTML, CSS and Javascript -- are
no longer supported.
For advice on writing your own plugin, see <<plugin-authors>>.
include::plugin-script.asciidoc[]
include::api.asciidoc[]

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