Merge pull request #15278 from s1monw/settings_prototype

Add infrastructure to transactionally apply and reset dynamic settings
This commit is contained in:
Simon Willnauer 2015-12-18 10:57:29 +01:00
commit 55f77db45b
132 changed files with 2943 additions and 1781 deletions

View File

@ -0,0 +1,127 @@
/*
* 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.action.admin.cluster.settings;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.cluster.ClusterState.builder;
/**
* Updates transient and persistent cluster state settings if there are any changes
* due to the update.
*/
final class SettingsUpdater {
final Settings.Builder transientUpdates = Settings.settingsBuilder();
final Settings.Builder persistentUpdates = Settings.settingsBuilder();
private final ClusterSettings clusterSettings;
SettingsUpdater(ClusterSettings clusterSettings) {
this.clusterSettings = clusterSettings;
}
synchronized Settings getTransientUpdates() {
return transientUpdates.build();
}
synchronized Settings getPersistentUpdate() {
return persistentUpdates.build();
}
synchronized ClusterState updateSettings(final ClusterState currentState, Settings transientToApply, Settings persistentToApply) {
boolean changed = false;
Settings.Builder transientSettings = Settings.settingsBuilder();
transientSettings.put(currentState.metaData().transientSettings());
changed |= apply(transientToApply, transientSettings, transientUpdates, "transient");
Settings.Builder persistentSettings = Settings.settingsBuilder();
persistentSettings.put(currentState.metaData().persistentSettings());
changed |= apply(persistentToApply, persistentSettings, persistentUpdates, "persistent");
if (!changed) {
return currentState;
}
MetaData.Builder metaData = MetaData.builder(currentState.metaData())
.persistentSettings(persistentSettings.build())
.transientSettings(transientSettings.build());
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
boolean updatedReadOnly = MetaData.SETTING_READ_ONLY_SETTING.get(metaData.persistentSettings()) || MetaData.SETTING_READ_ONLY_SETTING.get(metaData.transientSettings());
if (updatedReadOnly) {
blocks.addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
} else {
blocks.removeGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
}
ClusterState build = builder(currentState).metaData(metaData).blocks(blocks).build();
Settings settings = build.metaData().settings();
// now we try to apply things and if they are invalid we fail
// this dryRun will validate & parse settings but won't actually apply them.
clusterSettings.dryRun(settings);
return build;
}
private boolean apply(Settings toApply, Settings.Builder target, Settings.Builder updates, String type) {
boolean changed = false;
final Set<String> toRemove = new HashSet<>();
Settings.Builder settingsBuilder = Settings.settingsBuilder();
for (Map.Entry<String, String> entry : toApply.getAsMap().entrySet()) {
if (entry.getValue() == null) {
toRemove.add(entry.getKey());
} else if (clusterSettings.isLoggerSetting(entry.getKey()) || clusterSettings.hasDynamicSetting(entry.getKey())) {
settingsBuilder.put(entry.getKey(), entry.getValue());
updates.put(entry.getKey(), entry.getValue());
changed = true;
} else {
throw new IllegalArgumentException(type + " setting [" + entry.getKey() + "], not dynamically updateable");
}
}
changed |= applyDeletes(toRemove, target);
target.put(settingsBuilder.build());
return changed;
}
private final boolean applyDeletes(Set<String> deletes, Settings.Builder builder) {
boolean changed = false;
for (String entry : deletes) {
Set<String> keysToRemove = new HashSet<>();
Set<String> keySet = builder.internalMap().keySet();
for (String key : keySet) {
if (Regex.simpleMatch(entry, key)) {
keysToRemove.add(key);
}
}
for (String key : keysToRemove) {
builder.remove(key);
changed = true;
}
}
return changed;
}
}

View File

@ -28,23 +28,19 @@ import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
import org.elasticsearch.cluster.settings.DynamicSettings;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.Map;
import static org.elasticsearch.cluster.ClusterState.builder;
/**
@ -54,15 +50,14 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
private final AllocationService allocationService;
private final DynamicSettings dynamicSettings;
private final ClusterSettings clusterSettings;
@Inject
public TransportClusterUpdateSettingsAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool,
AllocationService allocationService, @ClusterDynamicSettings DynamicSettings dynamicSettings,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
AllocationService allocationService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, ClusterSettings clusterSettings) {
super(settings, ClusterUpdateSettingsAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, ClusterUpdateSettingsRequest::new);
this.allocationService = allocationService;
this.dynamicSettings = dynamicSettings;
this.clusterSettings = clusterSettings;
}
@Override
@ -73,8 +68,8 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
@Override
protected ClusterBlockException checkBlock(ClusterUpdateSettingsRequest request, ClusterState state) {
// allow for dedicated changes to the metadata blocks, so we don't block those to allow to "re-enable" it
if ((request.transientSettings().getAsMap().isEmpty() && request.persistentSettings().getAsMap().size() == 1 && request.persistentSettings().get(MetaData.SETTING_READ_ONLY) != null) ||
request.persistentSettings().getAsMap().isEmpty() && request.transientSettings().getAsMap().size() == 1 && request.transientSettings().get(MetaData.SETTING_READ_ONLY) != null) {
if ((request.transientSettings().getAsMap().isEmpty() && request.persistentSettings().getAsMap().size() == 1 && MetaData.SETTING_READ_ONLY_SETTING.exists(request.persistentSettings())) ||
request.persistentSettings().getAsMap().isEmpty() && request.transientSettings().getAsMap().size() == 1 && MetaData.SETTING_READ_ONLY_SETTING.exists(request.transientSettings())) {
return null;
}
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
@ -88,9 +83,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
@Override
protected void masterOperation(final ClusterUpdateSettingsRequest request, final ClusterState state, final ActionListener<ClusterUpdateSettingsResponse> listener) {
final Settings.Builder transientUpdates = Settings.settingsBuilder();
final Settings.Builder persistentUpdates = Settings.settingsBuilder();
final SettingsUpdater updater = new SettingsUpdater(clusterSettings);
clusterService.submitStateUpdateTask("cluster_update_settings",
new AckedClusterStateUpdateTask<ClusterUpdateSettingsResponse>(Priority.IMMEDIATE, request, listener) {
@ -98,7 +91,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
@Override
protected ClusterUpdateSettingsResponse newResponse(boolean acknowledged) {
return new ClusterUpdateSettingsResponse(acknowledged, transientUpdates.build(), persistentUpdates.build());
return new ClusterUpdateSettingsResponse(acknowledged, updater.getTransientUpdates(), updater.getPersistentUpdate());
}
@Override
@ -125,7 +118,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
// so we should *not* execute the reroute.
if (!clusterService.state().nodes().localNodeMaster()) {
logger.debug("Skipping reroute after cluster update settings, because node is no longer master");
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, transientUpdates.build(), persistentUpdates.build()));
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, updater.getTransientUpdates(), updater.getPersistentUpdate()));
return;
}
@ -145,13 +138,13 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
@Override
//we return when the cluster reroute is acked or it times out but the acknowledged flag depends on whether the update settings was acknowledged
protected ClusterUpdateSettingsResponse newResponse(boolean acknowledged) {
return new ClusterUpdateSettingsResponse(updateSettingsAcked && acknowledged, transientUpdates.build(), persistentUpdates.build());
return new ClusterUpdateSettingsResponse(updateSettingsAcked && acknowledged, updater.getTransientUpdates(), updater.getPersistentUpdate());
}
@Override
public void onNoLongerMaster(String source) {
logger.debug("failed to preform reroute after cluster settings were updated - current node is no longer a master");
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, transientUpdates.build(), persistentUpdates.build()));
listener.onResponse(new ClusterUpdateSettingsResponse(updateSettingsAcked, updater.getTransientUpdates(), updater.getPersistentUpdate()));
}
@Override
@ -181,58 +174,11 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
@Override
public ClusterState execute(final ClusterState currentState) {
Settings.Builder transientSettings = Settings.settingsBuilder();
transientSettings.put(currentState.metaData().transientSettings());
for (Map.Entry<String, String> entry : request.transientSettings().getAsMap().entrySet()) {
if (dynamicSettings.isDynamicOrLoggingSetting(entry.getKey())) {
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue(), clusterService.state());
if (error == null) {
transientSettings.put(entry.getKey(), entry.getValue());
transientUpdates.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring transient setting [{}], [{}]", entry.getKey(), error);
}
} else {
logger.warn("ignoring transient setting [{}], not dynamically updateable", entry.getKey());
}
}
Settings.Builder persistentSettings = Settings.settingsBuilder();
persistentSettings.put(currentState.metaData().persistentSettings());
for (Map.Entry<String, String> entry : request.persistentSettings().getAsMap().entrySet()) {
if (dynamicSettings.isDynamicOrLoggingSetting(entry.getKey())) {
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue(), clusterService.state());
if (error == null) {
persistentSettings.put(entry.getKey(), entry.getValue());
persistentUpdates.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring persistent setting [{}], [{}]", entry.getKey(), error);
}
} else {
logger.warn("ignoring persistent setting [{}], not dynamically updateable", entry.getKey());
}
}
if (!changed) {
return currentState;
}
MetaData.Builder metaData = MetaData.builder(currentState.metaData())
.persistentSettings(persistentSettings.build())
.transientSettings(transientSettings.build());
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
boolean updatedReadOnly = metaData.persistentSettings().getAsBoolean(MetaData.SETTING_READ_ONLY, false) || metaData.transientSettings().getAsBoolean(MetaData.SETTING_READ_ONLY, false);
if (updatedReadOnly) {
blocks.addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
} else {
blocks.removeGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
}
return builder(currentState).metaData(metaData).blocks(blocks).build();
ClusterState clusterState = updater.updateSettings(currentState, request.transientSettings(), request.persistentSettings());
changed = clusterState != currentState;
return clusterState;
}
});
}
}

View File

@ -31,31 +31,36 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
/**
* Close index action
*/
public class TransportCloseIndexAction extends TransportMasterNodeAction<CloseIndexRequest, CloseIndexResponse> implements NodeSettingsService.Listener {
public class TransportCloseIndexAction extends TransportMasterNodeAction<CloseIndexRequest, CloseIndexResponse> {
private final MetaDataIndexStateService indexStateService;
private final DestructiveOperations destructiveOperations;
private volatile boolean closeIndexEnabled;
public static final String SETTING_CLUSTER_INDICES_CLOSE_ENABLE = "cluster.indices.close.enable";
public static final Setting<Boolean> CLUSTER_INDICES_CLOSE_ENABLE_SETTING = Setting.boolSetting("cluster.indices.close.enable", true, true, Setting.Scope.CLUSTER);
@Inject
public TransportCloseIndexAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, MetaDataIndexStateService indexStateService,
NodeSettingsService nodeSettingsService, ActionFilters actionFilters,
ClusterSettings clusterSettings, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, DestructiveOperations destructiveOperations) {
super(settings, CloseIndexAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, CloseIndexRequest::new);
this.indexStateService = indexStateService;
this.destructiveOperations = destructiveOperations;
this.closeIndexEnabled = settings.getAsBoolean(SETTING_CLUSTER_INDICES_CLOSE_ENABLE, true);
nodeSettingsService.addListener(this);
this.closeIndexEnabled = CLUSTER_INDICES_CLOSE_ENABLE_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_INDICES_CLOSE_ENABLE_SETTING, this::setCloseIndexEnabled);
}
private void setCloseIndexEnabled(boolean closeIndexEnabled) {
this.closeIndexEnabled = closeIndexEnabled;
}
@Override
@ -73,7 +78,7 @@ public class TransportCloseIndexAction extends TransportMasterNodeAction<CloseIn
protected void doExecute(CloseIndexRequest request, ActionListener<CloseIndexResponse> listener) {
destructiveOperations.failDestructive(request.indices());
if (closeIndexEnabled == false) {
throw new IllegalStateException("closing indices is disabled - set [" + SETTING_CLUSTER_INDICES_CLOSE_ENABLE + ": true] to enable it. NOTE: closed indices still consume a significant amount of diskspace");
throw new IllegalStateException("closing indices is disabled - set [" + CLUSTER_INDICES_CLOSE_ENABLE_SETTING.getKey() + ": true] to enable it. NOTE: closed indices still consume a significant amount of diskspace");
}
super.doExecute(request, listener);
}
@ -104,13 +109,4 @@ public class TransportCloseIndexAction extends TransportMasterNodeAction<CloseIn
}
});
}
@Override
public void onRefreshSettings(Settings settings) {
final boolean enable = settings.getAsBoolean(SETTING_CLUSTER_INDICES_CLOSE_ENABLE, this.closeIndexEnabled);
if (enable != this.closeIndexEnabled) {
logger.info("updating [{}] from [{}] to [{}]", SETTING_CLUSTER_INDICES_CLOSE_ENABLE, this.closeIndexEnabled, enable);
this.closeIndexEnabled = enable;
}
}
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -45,8 +44,7 @@ public class TransportDeleteIndexAction extends TransportMasterNodeAction<Delete
@Inject
public TransportDeleteIndexAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, MetaDataDeleteIndexService deleteIndexService,
NodeSettingsService nodeSettingsService, ActionFilters actionFilters,
ThreadPool threadPool, MetaDataDeleteIndexService deleteIndexService, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, DestructiveOperations destructiveOperations) {
super(settings, DeleteIndexAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, DeleteIndexRequest::new);
this.deleteIndexService = deleteIndexService;

View File

@ -32,12 +32,9 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.Arrays;
/**
* Open index action
*/
@ -49,7 +46,7 @@ public class TransportOpenIndexAction extends TransportMasterNodeAction<OpenInde
@Inject
public TransportOpenIndexAction(Settings settings, TransportService transportService, ClusterService clusterService,
ThreadPool threadPool, MetaDataIndexStateService indexStateService,
NodeSettingsService nodeSettingsService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
DestructiveOperations destructiveOperations) {
super(settings, OpenIndexAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, OpenIndexRequest::new);
this.indexStateService = indexStateService;

View File

@ -21,25 +21,30 @@ package org.elasticsearch.action.support;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* Helper for dealing with destructive operations and wildcard usage.
*/
public final class DestructiveOperations extends AbstractComponent implements NodeSettingsService.Listener {
public final class DestructiveOperations extends AbstractComponent {
/**
* Setting which controls whether wildcard usage (*, prefix*, _all) is allowed.
*/
public static final String REQUIRES_NAME = "action.destructive_requires_name";
public static final Setting<Boolean> REQUIRES_NAME_SETTING = Setting.boolSetting("action.destructive_requires_name", false, true, Setting.Scope.CLUSTER);
private volatile boolean destructiveRequiresName;
@Inject
public DestructiveOperations(Settings settings, NodeSettingsService nodeSettingsService) {
public DestructiveOperations(Settings settings, ClusterSettings clusterSettings) {
super(settings);
destructiveRequiresName = settings.getAsBoolean(DestructiveOperations.REQUIRES_NAME, false);
nodeSettingsService.addListener(this);
destructiveRequiresName = REQUIRES_NAME_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(REQUIRES_NAME_SETTING, this::setDestructiveRequiresName);
}
private void setDestructiveRequiresName(boolean destructiveRequiresName) {
this.destructiveRequiresName = destructiveRequiresName;
}
/**
@ -65,15 +70,6 @@ public final class DestructiveOperations extends AbstractComponent implements No
}
}
@Override
public void onRefreshSettings(Settings settings) {
boolean newValue = settings.getAsBoolean(DestructiveOperations.REQUIRES_NAME, destructiveRequiresName);
if (destructiveRequiresName != newValue) {
logger.info("updating [action.operate_all_indices] from [{}] to [{}]", destructiveRequiresName, newValue);
this.destructiveRequiresName = newValue;
}
}
private static boolean hasWildcardUsage(String aliasOrIndex) {
return "_all".equals(aliasOrIndex) || aliasOrIndex.indexOf('*') != -1;
}

View File

@ -19,9 +19,6 @@
package org.elasticsearch.cluster;
import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction;
import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction;
@ -29,7 +26,6 @@ import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.IndexTemplateFilter;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService;
import org.elasticsearch.cluster.metadata.MetaDataIndexAliasesService;
@ -60,17 +56,15 @@ import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocatio
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.cluster.settings.DynamicSettings;
import org.elasticsearch.cluster.settings.Validator;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.ExtensionPoint;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.gateway.PrimaryShardAllocator;
import org.elasticsearch.index.engine.EngineConfig;
@ -81,21 +75,13 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.MergePolicyConfig;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.search.internal.DefaultSearchContext;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.*;
/**
* Configures classes and services that affect the entire cluster.
@ -122,7 +108,6 @@ public class ClusterModule extends AbstractModule {
SnapshotInProgressAllocationDecider.class));
private final Settings settings;
private final DynamicSettings.Builder clusterDynamicSettings = new DynamicSettings.Builder();
private final DynamicSettings.Builder indexDynamicSettings = new DynamicSettings.Builder();
private final ExtensionPoint.SelectedType<ShardsAllocator> shardsAllocators = new ExtensionPoint.SelectedType<>("shards_allocator", ShardsAllocator.class);
private final ExtensionPoint.ClassSet<AllocationDecider> allocationDeciders = new ExtensionPoint.ClassSet<>("allocation_decider", AllocationDecider.class, AllocationDeciders.class);
@ -134,7 +119,6 @@ public class ClusterModule extends AbstractModule {
public ClusterModule(Settings settings) {
this.settings = settings;
registerBuiltinClusterSettings();
registerBuiltinIndexSettings();
for (Class<? extends AllocationDecider> decider : ClusterModule.DEFAULT_ALLOCATION_DECIDERS) {
@ -144,70 +128,11 @@ public class ClusterModule extends AbstractModule {
registerShardsAllocator(ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR, BalancedShardsAllocator.class);
}
private void registerBuiltinClusterSettings() {
registerClusterDynamicSetting(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES, Validator.EMPTY);
registerClusterDynamicSetting(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP + "*", Validator.EMPTY);
registerClusterDynamicSetting(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, Validator.FLOAT);
registerClusterDynamicSetting(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, Validator.FLOAT);
registerClusterDynamicSetting(BalancedShardsAllocator.SETTING_THRESHOLD, Validator.NON_NEGATIVE_FLOAT);
registerClusterDynamicSetting(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ALLOCATION_ALLOW_REBALANCE_VALIDATOR);
registerClusterDynamicSetting(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, Validator.INTEGER);
registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, Validator.EMPTY);
registerClusterDynamicSetting(ZenDiscovery.SETTING_REJOIN_ON_MASTER_GONE, Validator.BOOLEAN);
registerClusterDynamicSetting(DiscoverySettings.NO_MASTER_BLOCK, Validator.EMPTY);
registerClusterDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP + "*", Validator.EMPTY);
registerClusterDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP + "*", Validator.EMPTY);
registerClusterDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP + "*", Validator.EMPTY);
registerClusterDynamicSetting(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE, Validator.EMPTY);
registerClusterDynamicSetting(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
registerClusterDynamicSetting(IndicesTTLService.INDICES_TTL_INTERVAL, Validator.TIME);
registerClusterDynamicSetting(MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT, Validator.TIME);
registerClusterDynamicSetting(MetaData.SETTING_READ_ONLY, Validator.EMPTY);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, Validator.POSITIVE_INTEGER);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, Validator.POSITIVE_INTEGER);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(ThreadPool.THREADPOOL_GROUP + "*", ThreadPool.THREAD_POOL_TYPE_SETTINGS_VALIDATOR);
registerClusterDynamicSetting(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, Validator.INTEGER);
registerClusterDynamicSetting(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, Validator.INTEGER);
registerClusterDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, Validator.EMPTY);
registerClusterDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, Validator.EMPTY);
registerClusterDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, Validator.BOOLEAN);
registerClusterDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, Validator.BOOLEAN);
registerClusterDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(SnapshotInProgressAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED, Validator.EMPTY);
registerClusterDynamicSetting(DestructiveOperations.REQUIRES_NAME, Validator.EMPTY);
registerClusterDynamicSetting(DiscoverySettings.PUBLISH_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(DiscoverySettings.PUBLISH_DIFF_ENABLE, Validator.BOOLEAN);
registerClusterDynamicSetting(DiscoverySettings.COMMIT_TIMEOUT, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
registerClusterDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
registerClusterDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, Validator.NON_NEGATIVE_DOUBLE);
registerClusterDynamicSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
registerClusterDynamicSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, Validator.NON_NEGATIVE_DOUBLE);
registerClusterDynamicSetting(InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, Validator.TIME_NON_NEGATIVE);
registerClusterDynamicSetting(SearchService.DEFAULT_SEARCH_TIMEOUT, Validator.TIMEOUT);
registerClusterDynamicSetting(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_VALIDATOR);
registerClusterDynamicSetting(TransportService.SETTING_TRACE_LOG_INCLUDE, Validator.EMPTY);
registerClusterDynamicSetting(TransportService.SETTING_TRACE_LOG_INCLUDE + ".*", Validator.EMPTY);
registerClusterDynamicSetting(TransportService.SETTING_TRACE_LOG_EXCLUDE, Validator.EMPTY);
registerClusterDynamicSetting(TransportService.SETTING_TRACE_LOG_EXCLUDE + ".*", Validator.EMPTY);
registerClusterDynamicSetting(TransportCloseIndexAction.SETTING_CLUSTER_INDICES_CLOSE_ENABLE, Validator.BOOLEAN);
registerClusterDynamicSetting(ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE, Validator.INTEGER);
registerClusterDynamicSetting(TransportReplicationAction.SHARD_FAILURE_TIMEOUT, Validator.TIME_NON_NEGATIVE);
}
private void registerBuiltinIndexSettings() {
registerIndexDynamicSetting(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
registerIndexDynamicSetting(IndexStore.INDEX_STORE_THROTTLE_TYPE, Validator.EMPTY);
registerIndexDynamicSetting(MergeSchedulerConfig.MAX_THREAD_COUNT, Validator.EMPTY);
registerIndexDynamicSetting(MergeSchedulerConfig.MAX_THREAD_COUNT, Validator.NON_NEGATIVE_INTEGER);
registerIndexDynamicSetting(MergeSchedulerConfig.MAX_MERGE_COUNT, Validator.EMPTY);
registerIndexDynamicSetting(MergeSchedulerConfig.AUTO_THROTTLE, Validator.EMPTY);
registerIndexDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_REQUIRE_GROUP + "*", Validator.EMPTY);
@ -272,9 +197,6 @@ public class ClusterModule extends AbstractModule {
indexDynamicSettings.addSetting(setting, validator);
}
public void registerClusterDynamicSetting(String setting, Validator validator) {
clusterDynamicSettings.addSetting(setting, validator);
}
public void registerAllocationDecider(Class<? extends AllocationDecider> allocationDecider) {
allocationDeciders.registerExtension(allocationDecider);
@ -290,7 +212,6 @@ public class ClusterModule extends AbstractModule {
@Override
protected void configure() {
bind(DynamicSettings.class).annotatedWith(ClusterDynamicSettings.class).toInstance(clusterDynamicSettings.build());
bind(DynamicSettings.class).annotatedWith(IndexDynamicSettings.class).toInstance(indexDynamicSettings.build());
// bind ShardsAllocator

View File

@ -37,11 +37,12 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ReceiveTimeoutTransportException;
@ -63,8 +64,8 @@ import java.util.concurrent.TimeUnit;
*/
public class InternalClusterInfoService extends AbstractComponent implements ClusterInfoService, LocalNodeMasterListener, ClusterStateListener {
public static final String INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL = "cluster.info.update.interval";
public static final String INTERNAL_CLUSTER_INFO_TIMEOUT = "cluster.info.update.timeout";
public static final Setting<TimeValue> INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING = Setting.timeSetting("cluster.info.update.interval", TimeValue.timeValueSeconds(30), TimeValue.timeValueSeconds(10), true, Setting.Scope.CLUSTER);
public static final Setting<TimeValue> INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING = Setting.positiveTimeSetting("cluster.info.update.timeout", TimeValue.timeValueSeconds(15), true, Setting.Scope.CLUSTER);
private volatile TimeValue updateFrequency;
@ -82,7 +83,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
private final List<Listener> listeners = new CopyOnWriteArrayList<>();
@Inject
public InternalClusterInfoService(Settings settings, NodeSettingsService nodeSettingsService,
public InternalClusterInfoService(Settings settings, ClusterSettings clusterSettings,
TransportNodesStatsAction transportNodesStatsAction,
TransportIndicesStatsAction transportIndicesStatsAction, ClusterService clusterService,
ThreadPool threadPool) {
@ -95,10 +96,12 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
this.transportIndicesStatsAction = transportIndicesStatsAction;
this.clusterService = clusterService;
this.threadPool = threadPool;
this.updateFrequency = settings.getAsTime(INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, TimeValue.timeValueSeconds(30));
this.fetchTimeout = settings.getAsTime(INTERNAL_CLUSTER_INFO_TIMEOUT, TimeValue.timeValueSeconds(15));
this.enabled = settings.getAsBoolean(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true);
nodeSettingsService.addListener(new ApplySettings());
this.updateFrequency = INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.get(settings);
this.fetchTimeout = INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.get(settings);
this.enabled = DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING, this::setFetchTimeout);
clusterSettings.addSettingsUpdateConsumer(INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING, this::setUpdateFrequency);
clusterSettings.addSettingsUpdateConsumer(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled);
// Add InternalClusterInfoService to listen for Master changes
this.clusterService.add((LocalNodeMasterListener)this);
@ -106,35 +109,16 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
this.clusterService.add((ClusterStateListener)this);
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
TimeValue newUpdateFrequency = settings.getAsTime(INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, null);
// ClusterInfoService is only enabled if the DiskThresholdDecider is enabled
Boolean newEnabled = settings.getAsBoolean(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, null);
if (newUpdateFrequency != null) {
if (newUpdateFrequency.getMillis() < TimeValue.timeValueSeconds(10).getMillis()) {
logger.warn("[{}] set too low [{}] (< 10s)", INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, newUpdateFrequency);
throw new IllegalStateException("Unable to set " + INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL + " less than 10 seconds");
} else {
logger.info("updating [{}] from [{}] to [{}]", INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, updateFrequency, newUpdateFrequency);
InternalClusterInfoService.this.updateFrequency = newUpdateFrequency;
}
private void setEnabled(boolean enabled) {
this.enabled = enabled;
}
TimeValue newFetchTimeout = settings.getAsTime(INTERNAL_CLUSTER_INFO_TIMEOUT, null);
if (newFetchTimeout != null) {
logger.info("updating fetch timeout [{}] from [{}] to [{}]", INTERNAL_CLUSTER_INFO_TIMEOUT, fetchTimeout, newFetchTimeout);
InternalClusterInfoService.this.fetchTimeout = newFetchTimeout;
private void setFetchTimeout(TimeValue fetchTimeout) {
this.fetchTimeout = fetchTimeout;
}
// We don't log about enabling it here, because the DiskThresholdDecider will already be logging about enable/disable
if (newEnabled != null) {
InternalClusterInfoService.this.enabled = newEnabled;
}
}
void setUpdateFrequency(TimeValue updateFrequency) {
this.updateFrequency = updateFrequency;
}
@Override

View File

@ -26,11 +26,12 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.client.IndicesAdminClient;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.concurrent.TimeoutException;
@ -40,30 +41,23 @@ import java.util.concurrent.TimeoutException;
*/
public class MappingUpdatedAction extends AbstractComponent {
public static final String INDICES_MAPPING_DYNAMIC_TIMEOUT = "indices.mapping.dynamic_timeout";
public static final Setting<TimeValue> INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING = Setting.positiveTimeSetting("indices.mapping.dynamic_timeout", TimeValue.timeValueSeconds(30), true, Setting.Scope.CLUSTER);
private IndicesAdminClient client;
private volatile TimeValue dynamicMappingUpdateTimeout;
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
TimeValue current = MappingUpdatedAction.this.dynamicMappingUpdateTimeout;
TimeValue newValue = settings.getAsTime(INDICES_MAPPING_DYNAMIC_TIMEOUT, current);
if (!current.equals(newValue)) {
logger.info("updating " + INDICES_MAPPING_DYNAMIC_TIMEOUT + " from [{}] to [{}]", current, newValue);
MappingUpdatedAction.this.dynamicMappingUpdateTimeout = newValue;
}
}
@Inject
public MappingUpdatedAction(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.dynamicMappingUpdateTimeout = INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING, this::setDynamicMappingUpdateTimeout);
}
@Inject
public MappingUpdatedAction(Settings settings, NodeSettingsService nodeSettingsService) {
super(settings);
this.dynamicMappingUpdateTimeout = settings.getAsTime(INDICES_MAPPING_DYNAMIC_TIMEOUT, TimeValue.timeValueSeconds(30));
nodeSettingsService.addListener(new ApplySettings());
private void setDynamicMappingUpdateTimeout(TimeValue dynamicMappingUpdateTimeout) {
this.dynamicMappingUpdateTimeout = dynamicMappingUpdateTimeout;
}
public void setClient(Client client) {
this.client = client.admin().indices();
}

View File

@ -40,8 +40,8 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.support.LoggerMessageFormat;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.xcontent.FromXContentBuilder;
@ -140,7 +140,7 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
}
public static final String SETTING_READ_ONLY = "cluster.blocks.read_only";
public static final Setting<Boolean> SETTING_READ_ONLY_SETTING = Setting.boolSetting("cluster.blocks.read_only", false, true, Setting.Scope.CLUSTER);
public static final ClusterBlock CLUSTER_READ_ONLY_BLOCK = new ClusterBlock(6, "cluster read-only (api)", false, false, RestStatus.FORBIDDEN, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
@ -745,23 +745,23 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
/** All known byte-sized cluster settings. */
public static final Set<String> CLUSTER_BYTES_SIZE_SETTINGS = unmodifiableSet(newHashSet(
IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC,
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC));
IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()));
/** All known time cluster settings. */
public static final Set<String> CLUSTER_TIME_SETTINGS = unmodifiableSet(newHashSet(
IndicesTTLService.INDICES_TTL_INTERVAL,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK,
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT,
RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT,
RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT,
DiscoverySettings.PUBLISH_TIMEOUT,
InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD));
IndicesTTLService.INDICES_TTL_INTERVAL_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING.getKey(),
RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING.getKey(),
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(),
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(),
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(),
DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(),
InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.getKey()));
/** As of 2.0 we require units for time and byte-sized settings. This methods adds default units to any cluster settings that don't
* specify a unit. */

View File

@ -34,12 +34,13 @@ import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.gateway.PriorityComparator;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.ArrayList;
import java.util.Collection;
@ -72,42 +73,32 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
*/
public class BalancedShardsAllocator extends AbstractComponent implements ShardsAllocator {
public static final String SETTING_THRESHOLD = "cluster.routing.allocation.balance.threshold";
public static final String SETTING_INDEX_BALANCE_FACTOR = "cluster.routing.allocation.balance.index";
public static final String SETTING_SHARD_BALANCE_FACTOR = "cluster.routing.allocation.balance.shard";
private static final float DEFAULT_INDEX_BALANCE_FACTOR = 0.55f;
private static final float DEFAULT_SHARD_BALANCE_FACTOR = 0.45f;
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
final float indexBalance = settings.getAsFloat(SETTING_INDEX_BALANCE_FACTOR, weightFunction.indexBalance);
final float shardBalance = settings.getAsFloat(SETTING_SHARD_BALANCE_FACTOR, weightFunction.shardBalance);
float threshold = settings.getAsFloat(SETTING_THRESHOLD, BalancedShardsAllocator.this.threshold);
if (threshold <= 0.0f) {
throw new IllegalArgumentException("threshold must be greater than 0.0f but was: " + threshold);
}
BalancedShardsAllocator.this.threshold = threshold;
BalancedShardsAllocator.this.weightFunction = new WeightFunction(indexBalance, shardBalance);
}
}
private volatile WeightFunction weightFunction = new WeightFunction(DEFAULT_INDEX_BALANCE_FACTOR, DEFAULT_SHARD_BALANCE_FACTOR);
private volatile float threshold = 1.0f;
public static final Setting<Float> INDEX_BALANCE_FACTOR_SETTING = Setting.floatSetting("cluster.routing.allocation.balance.index", 0.55f, true, Setting.Scope.CLUSTER);
public static final Setting<Float> SHARD_BALANCE_FACTOR_SETTING = Setting.floatSetting("cluster.routing.allocation.balance.shard", 0.45f, true, Setting.Scope.CLUSTER);
public static final Setting<Float> THRESHOLD_SETTING = Setting.floatSetting("cluster.routing.allocation.balance.threshold", 1.0f, 0.0f, true, Setting.Scope.CLUSTER);
private volatile WeightFunction weightFunction;
private volatile float threshold;
public BalancedShardsAllocator(Settings settings) {
this(settings, new NodeSettingsService(settings));
this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
}
@Inject
public BalancedShardsAllocator(Settings settings, NodeSettingsService nodeSettingsService) {
public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSettings) {
super(settings);
ApplySettings applySettings = new ApplySettings();
applySettings.onRefreshSettings(settings);
nodeSettingsService.addListener(applySettings);
setWeightFunction(INDEX_BALANCE_FACTOR_SETTING.get(settings), SHARD_BALANCE_FACTOR_SETTING.get(settings));
setThreshold(THRESHOLD_SETTING.get(settings));
clusterSettings.addSettingsUpdateConsumer(INDEX_BALANCE_FACTOR_SETTING, SHARD_BALANCE_FACTOR_SETTING, this::setWeightFunction);
clusterSettings.addSettingsUpdateConsumer(THRESHOLD_SETTING, this::setThreshold);
}
private void setWeightFunction(float indexBalance, float shardBalanceFactor) {
weightFunction = new WeightFunction(indexBalance, shardBalanceFactor);
}
private void setThreshold(float threshold) {
this.threshold = threshold;
}
@Override

View File

@ -24,10 +24,11 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.HashMap;
import java.util.Map;
@ -76,37 +77,12 @@ public class AwarenessAllocationDecider extends AllocationDecider {
public static final String NAME = "awareness";
public static final String CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES = "cluster.routing.allocation.awareness.attributes";
public static final String CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP = "cluster.routing.allocation.awareness.force.";
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
String[] awarenessAttributes = settings.getAsArray(CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES, null);
if (awarenessAttributes == null && "".equals(settings.get(CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES, null))) {
awarenessAttributes = Strings.EMPTY_ARRAY; // the empty string resets this
}
if (awarenessAttributes != null) {
logger.info("updating [cluster.routing.allocation.awareness.attributes] from [{}] to [{}]", AwarenessAllocationDecider.this.awarenessAttributes, awarenessAttributes);
AwarenessAllocationDecider.this.awarenessAttributes = awarenessAttributes;
}
Map<String, String[]> forcedAwarenessAttributes = new HashMap<>(AwarenessAllocationDecider.this.forcedAwarenessAttributes);
Map<String, Settings> forceGroups = settings.getGroups(CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP);
if (!forceGroups.isEmpty()) {
for (Map.Entry<String, Settings> entry : forceGroups.entrySet()) {
String[] aValues = entry.getValue().getAsArray("values");
if (aValues.length > 0) {
forcedAwarenessAttributes.put(entry.getKey(), aValues);
}
}
}
AwarenessAllocationDecider.this.forcedAwarenessAttributes = forcedAwarenessAttributes;
}
}
public static final Setting<String[]> CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING = new Setting<>("cluster.routing.allocation.awareness.attributes", "", Strings::splitStringByCommaToArray , true, Setting.Scope.CLUSTER);
public static final Setting<Settings> CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING = Setting.groupSetting("cluster.routing.allocation.awareness.force.", true, Setting.Scope.CLUSTER);
private String[] awarenessAttributes;
private Map<String, String[]> forcedAwarenessAttributes;
private volatile Map<String, String[]> forcedAwarenessAttributes;
/**
* Creates a new {@link AwarenessAllocationDecider} instance
@ -121,24 +97,28 @@ public class AwarenessAllocationDecider extends AllocationDecider {
* @param settings {@link Settings} to use
*/
public AwarenessAllocationDecider(Settings settings) {
this(settings, new NodeSettingsService(settings));
this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
}
@Inject
public AwarenessAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public AwarenessAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.awarenessAttributes = settings.getAsArray(CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES);
this.awarenessAttributes = CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING, this::setAwarenessAttributes);
setForcedAwarenessAttributes(CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING.get(settings));
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING, this::setForcedAwarenessAttributes);
}
forcedAwarenessAttributes = new HashMap<>();
Map<String, Settings> forceGroups = settings.getGroups(CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP);
private void setForcedAwarenessAttributes(Settings forceSettings) {
Map<String, String[]> forcedAwarenessAttributes = new HashMap<>();
Map<String, Settings> forceGroups = forceSettings.getAsGroups();
for (Map.Entry<String, Settings> entry : forceGroups.entrySet()) {
String[] aValues = entry.getValue().getAsArray("values");
if (aValues.length > 0) {
forcedAwarenessAttributes.put(entry.getKey(), aValues);
}
}
nodeSettingsService.addListener(new ApplySettings());
this.forcedAwarenessAttributes = forcedAwarenessAttributes;
}
/**
@ -150,6 +130,10 @@ public class AwarenessAllocationDecider extends AllocationDecider {
return this.awarenessAttributes;
}
private void setAwarenessAttributes(String[] awarenessAttributes) {
this.awarenessAttributes = awarenessAttributes;
}
@Override
public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
return underCapacity(shardRouting, node, allocation, true);

View File

@ -19,13 +19,12 @@
package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.settings.Validator;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Locale;
@ -49,16 +48,7 @@ import java.util.Locale;
public class ClusterRebalanceAllocationDecider extends AllocationDecider {
public static final String NAME = "cluster_rebalance";
public static final String CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE = "cluster.routing.allocation.allow_rebalance";
public static final Validator ALLOCATION_ALLOW_REBALANCE_VALIDATOR = (setting, value, clusterState) -> {
try {
ClusterRebalanceType.parseString(value);
return null;
} catch (IllegalArgumentException e) {
return "the value of " + setting + " must be one of: [always, indices_primaries_active, indices_all_active]";
}
};
public static final Setting<ClusterRebalanceType> CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING = new Setting<>("cluster.routing.allocation.allow_rebalance", ClusterRebalanceType.INDICES_ALL_ACTIVE.name().toLowerCase(Locale.ROOT), ClusterRebalanceType::parseString, true, Setting.Scope.CLUSTER);
/**
* An enum representation for the configured re-balance type.
@ -85,48 +75,28 @@ public class ClusterRebalanceAllocationDecider extends AllocationDecider {
} else if ("indices_all_active".equalsIgnoreCase(typeString) || "indicesAllActive".equalsIgnoreCase(typeString)) {
return ClusterRebalanceType.INDICES_ALL_ACTIVE;
}
throw new IllegalArgumentException("Illegal value for " + CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE + ": " + typeString);
throw new IllegalArgumentException("Illegal value for " + CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING + ": " + typeString);
}
}
private ClusterRebalanceType type;
private volatile ClusterRebalanceType type;
@Inject
public ClusterRebalanceAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public ClusterRebalanceAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
String allowRebalance = settings.get(CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "indices_all_active");
try {
type = ClusterRebalanceType.parseString(allowRebalance);
type = CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.get(settings);
} catch (IllegalStateException e) {
logger.warn("[{}] has a wrong value {}, defaulting to 'indices_all_active'", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, allowRebalance);
logger.warn("[{}] has a wrong value {}, defaulting to 'indices_all_active'", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getRaw(settings));
type = ClusterRebalanceType.INDICES_ALL_ACTIVE;
}
logger.debug("using [{}] with [{}]", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type.toString().toLowerCase(Locale.ROOT));
logger.debug("using [{}] with [{}]", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, type.toString().toLowerCase(Locale.ROOT));
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, this::setType);
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
String newAllowRebalance = settings.get(CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, null);
if (newAllowRebalance != null) {
ClusterRebalanceType newType = null;
try {
newType = ClusterRebalanceType.parseString(newAllowRebalance);
} catch (IllegalArgumentException e) {
// ignore
}
if (newType != null && newType != ClusterRebalanceAllocationDecider.this.type) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
ClusterRebalanceAllocationDecider.this.type.toString().toLowerCase(Locale.ROOT),
newType.toString().toLowerCase(Locale.ROOT));
ClusterRebalanceAllocationDecider.this.type = newType;
}
}
}
private void setType(ClusterRebalanceType type) {
this.type = type;
}
@Override

View File

@ -22,8 +22,9 @@ package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* Similar to the {@link ClusterRebalanceAllocationDecider} this
@ -41,27 +42,19 @@ public class ConcurrentRebalanceAllocationDecider extends AllocationDecider {
public static final String NAME = "concurrent_rebalance";
public static final String CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE = "cluster.routing.allocation.cluster_concurrent_rebalance";
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int clusterConcurrentRebalance = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, ConcurrentRebalanceAllocationDecider.this.clusterConcurrentRebalance);
if (clusterConcurrentRebalance != ConcurrentRebalanceAllocationDecider.this.clusterConcurrentRebalance) {
logger.info("updating [cluster.routing.allocation.cluster_concurrent_rebalance] from [{}], to [{}]", ConcurrentRebalanceAllocationDecider.this.clusterConcurrentRebalance, clusterConcurrentRebalance);
ConcurrentRebalanceAllocationDecider.this.clusterConcurrentRebalance = clusterConcurrentRebalance;
}
}
}
public static final Setting<Integer> CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING = Setting.intSetting("cluster.routing.allocation.cluster_concurrent_rebalance", 2, -1, true, Setting.Scope.CLUSTER);
private volatile int clusterConcurrentRebalance;
@Inject
public ConcurrentRebalanceAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public ConcurrentRebalanceAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.clusterConcurrentRebalance = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 2);
this.clusterConcurrentRebalance = CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.get(settings);
logger.debug("using [cluster_concurrent_rebalance] with [{}]", clusterConcurrentRebalance);
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING, this::setClusterConcurrentRebalance);
}
private void setClusterConcurrentRebalance(int concurrentRebalance) {
clusterConcurrentRebalance = concurrentRebalance;
}
@Override

View File

@ -33,15 +33,16 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.RatioValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.set.Sets;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Set;
@ -80,53 +81,11 @@ public class DiskThresholdDecider extends AllocationDecider {
private volatile boolean enabled;
private volatile TimeValue rerouteInterval;
public static final String CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED = "cluster.routing.allocation.disk.threshold_enabled";
public static final String CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK = "cluster.routing.allocation.disk.watermark.low";
public static final String CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK = "cluster.routing.allocation.disk.watermark.high";
public static final String CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS = "cluster.routing.allocation.disk.include_relocations";
public static final String CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL = "cluster.routing.allocation.disk.reroute_interval";
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
String newLowWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, null);
String newHighWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, null);
Boolean newRelocationsSetting = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, null);
Boolean newEnableSetting = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, null);
TimeValue newRerouteInterval = settings.getAsTime(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, null);
if (newEnableSetting != null) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED,
DiskThresholdDecider.this.enabled, newEnableSetting);
DiskThresholdDecider.this.enabled = newEnableSetting;
}
if (newRelocationsSetting != null) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS,
DiskThresholdDecider.this.includeRelocations, newRelocationsSetting);
DiskThresholdDecider.this.includeRelocations = newRelocationsSetting;
}
if (newLowWatermark != null) {
if (!validWatermarkSetting(newLowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
throw new ElasticsearchParseException("unable to parse low watermark [{}]", newLowWatermark);
}
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, newLowWatermark);
DiskThresholdDecider.this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(newLowWatermark);
DiskThresholdDecider.this.freeBytesThresholdLow = thresholdBytesFromWatermark(newLowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
}
if (newHighWatermark != null) {
if (!validWatermarkSetting(newHighWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
throw new ElasticsearchParseException("unable to parse high watermark [{}]", newHighWatermark);
}
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, newHighWatermark);
DiskThresholdDecider.this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(newHighWatermark);
DiskThresholdDecider.this.freeBytesThresholdHigh = thresholdBytesFromWatermark(newHighWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
}
if (newRerouteInterval != null) {
logger.info("updating [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, newRerouteInterval);
DiskThresholdDecider.this.rerouteInterval = newRerouteInterval;
}
}
}
public static final Setting<Boolean> CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING = Setting.boolSetting("cluster.routing.allocation.disk.threshold_enabled", true, true, Setting.Scope.CLUSTER);
public static final Setting<String> CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING = new Setting<>("cluster.routing.allocation.disk.watermark.low", "85%", (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.low"), true, Setting.Scope.CLUSTER);
public static final Setting<String> CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING = new Setting<>("cluster.routing.allocation.disk.watermark.high", "90%", (s) -> validWatermarkSetting(s, "cluster.routing.allocation.disk.watermark.high"), true, Setting.Scope.CLUSTER);
public static final Setting<Boolean> CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING = Setting.boolSetting("cluster.routing.allocation.disk.include_relocations", true, true, Setting.Scope.CLUSTER);;
public static final Setting<TimeValue> CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING = Setting.positiveTimeSetting("cluster.routing.allocation.disk.reroute_interval", TimeValue.timeValueSeconds(60), true, Setting.Scope.CLUSTER);
/**
* Listens for a node to go over the high watermark and kicks off an empty
@ -231,38 +190,49 @@ public class DiskThresholdDecider extends AllocationDecider {
// It's okay the Client is null here, because the empty cluster info
// service will never actually call the listener where the client is
// needed. Also this constructor is only used for tests
this(settings, new NodeSettingsService(settings), EmptyClusterInfoService.INSTANCE, null);
this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), EmptyClusterInfoService.INSTANCE, null);
}
@Inject
public DiskThresholdDecider(Settings settings, NodeSettingsService nodeSettingsService, ClusterInfoService infoService, Client client) {
public DiskThresholdDecider(Settings settings, ClusterSettings clusterSettings, ClusterInfoService infoService, Client client) {
super(settings);
String lowWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "85%");
String highWatermark = settings.get(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "90%");
if (!validWatermarkSetting(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK)) {
throw new ElasticsearchParseException("unable to parse low watermark [{}]", lowWatermark);
}
if (!validWatermarkSetting(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK)) {
throw new ElasticsearchParseException("unable to parse high watermark [{}]", highWatermark);
}
// Watermark is expressed in terms of used data, but we need "free" data watermark
this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark);
this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark);
this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
this.includeRelocations = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true);
this.rerouteInterval = settings.getAsTime(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, TimeValue.timeValueSeconds(60));
this.enabled = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true);
nodeSettingsService.addListener(new ApplySettings());
final String lowWatermark = CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.get(settings);
final String highWatermark = CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.get(settings);
setHighWatermark(highWatermark);
setLowWatermark(lowWatermark);
this.includeRelocations = CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.get(settings);
this.rerouteInterval = CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.get(settings);
this.enabled = CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, this::setLowWatermark);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, this::setHighWatermark);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, this::setIncludeRelocations);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, this::setRerouteInterval);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled);
infoService.addListener(new DiskListener(client));
}
// For Testing
ApplySettings newApplySettings() {
return new ApplySettings();
private void setIncludeRelocations(boolean includeRelocations) {
this.includeRelocations = includeRelocations;
}
private void setRerouteInterval(TimeValue rerouteInterval) {
this.rerouteInterval = rerouteInterval;
}
private void setEnabled(boolean enabled) {
this.enabled = enabled;
}
private void setLowWatermark(String lowWatermark) {
// Watermark is expressed in terms of used data, but we need "free" data watermark
this.freeDiskThresholdLow = 100.0 - thresholdPercentageFromWatermark(lowWatermark);
this.freeBytesThresholdLow = thresholdBytesFromWatermark(lowWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey());
}
private void setHighWatermark(String highWatermark) {
// Watermark is expressed in terms of used data, but we need "free" data watermark
this.freeDiskThresholdHigh = 100.0 - thresholdPercentageFromWatermark(highWatermark);
this.freeBytesThresholdHigh = thresholdBytesFromWatermark(highWatermark, CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey());
}
// For Testing
@ -581,20 +551,21 @@ public class DiskThresholdDecider extends AllocationDecider {
/**
* Checks if a watermark string is a valid percentage or byte size value,
* returning true if valid, false if invalid.
* @return the watermark value given
*/
public boolean validWatermarkSetting(String watermark, String settingName) {
public static String validWatermarkSetting(String watermark, String settingName) {
try {
RatioValue.parseRatioValue(watermark);
return true;
} catch (ElasticsearchParseException e) {
try {
ByteSizeValue.parseBytesSizeValue(watermark, settingName);
return true;
} catch (ElasticsearchParseException ex) {
return false;
ex.addSuppressed(e);
throw ex;
}
}
return watermark;
}
private Decision earlyTerminate(RoutingAllocation allocation, ImmutableOpenMap<String, DiskUsage> usages) {

View File

@ -24,14 +24,15 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Locale;
/**
* This allocation decider allows shard allocations / rebalancing via the cluster wide settings {@link #CLUSTER_ROUTING_ALLOCATION_ENABLE} /
* {@link #CLUSTER_ROUTING_REBALANCE_ENABLE} and the per index setting {@link #INDEX_ROUTING_ALLOCATION_ENABLE} / {@link #INDEX_ROUTING_REBALANCE_ENABLE}.
* This allocation decider allows shard allocations / rebalancing via the cluster wide settings {@link #CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING} /
* {@link #CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING} and the per index setting {@link #INDEX_ROUTING_ALLOCATION_ENABLE} / {@link #INDEX_ROUTING_REBALANCE_ENABLE}.
* The per index settings overrides the cluster wide setting.
*
* <p>
@ -55,26 +56,34 @@ import java.util.Locale;
* @see Rebalance
* @see Allocation
*/
public class EnableAllocationDecider extends AllocationDecider implements NodeSettingsService.Listener {
public class EnableAllocationDecider extends AllocationDecider {
public static final String NAME = "enable";
public static final String CLUSTER_ROUTING_ALLOCATION_ENABLE = "cluster.routing.allocation.enable";
public static final Setting<Allocation> CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING = new Setting<>("cluster.routing.allocation.enable", Allocation.ALL.name(), Allocation::parse, true, Setting.Scope.CLUSTER);
public static final String INDEX_ROUTING_ALLOCATION_ENABLE= "index.routing.allocation.enable";
public static final String CLUSTER_ROUTING_REBALANCE_ENABLE = "cluster.routing.rebalance.enable";
public static final Setting<Rebalance> CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING = new Setting<>("cluster.routing.rebalance.enable", Rebalance.ALL.name(), Rebalance::parse, true, Setting.Scope.CLUSTER);
public static final String INDEX_ROUTING_REBALANCE_ENABLE = "index.routing.rebalance.enable";
private volatile Rebalance enableRebalance;
private volatile Allocation enableAllocation;
@Inject
public EnableAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public EnableAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.enableAllocation = Allocation.parse(settings.get(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.ALL.name()));
this.enableRebalance = Rebalance.parse(settings.get(CLUSTER_ROUTING_REBALANCE_ENABLE, Rebalance.ALL.name()));
nodeSettingsService.addListener(this);
this.enableAllocation = CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.get(settings);
this.enableRebalance = CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, this::setEnableAllocation);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, this::setEnableRebalance);
}
public void setEnableRebalance(Rebalance enableRebalance) {
this.enableRebalance = enableRebalance;
}
public void setEnableAllocation(Allocation enableAllocation) {
this.enableAllocation = enableAllocation;
}
@Override
@ -149,25 +158,9 @@ public class EnableAllocationDecider extends AllocationDecider implements NodeSe
}
}
@Override
public void onRefreshSettings(Settings settings) {
final Allocation enable = Allocation.parse(settings.get(CLUSTER_ROUTING_ALLOCATION_ENABLE, this.enableAllocation.name()));
if (enable != this.enableAllocation) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_ROUTING_ALLOCATION_ENABLE, this.enableAllocation, enable);
EnableAllocationDecider.this.enableAllocation = enable;
}
final Rebalance enableRebalance = Rebalance.parse(settings.get(CLUSTER_ROUTING_REBALANCE_ENABLE, this.enableRebalance.name()));
if (enableRebalance != this.enableRebalance) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_ROUTING_REBALANCE_ENABLE, this.enableRebalance, enableRebalance);
EnableAllocationDecider.this.enableRebalance = enableRebalance;
}
}
/**
* Allocation values or rather their string representation to be used used with
* {@link EnableAllocationDecider#CLUSTER_ROUTING_ALLOCATION_ENABLE} / {@link EnableAllocationDecider#INDEX_ROUTING_ALLOCATION_ENABLE}
* {@link EnableAllocationDecider#CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING} / {@link EnableAllocationDecider#INDEX_ROUTING_ALLOCATION_ENABLE}
* via cluster / index settings.
*/
public enum Allocation {
@ -193,7 +186,7 @@ public class EnableAllocationDecider extends AllocationDecider implements NodeSe
/**
* Rebalance values or rather their string representation to be used used with
* {@link EnableAllocationDecider#CLUSTER_ROUTING_REBALANCE_ENABLE} / {@link EnableAllocationDecider#INDEX_ROUTING_REBALANCE_ENABLE}
* {@link EnableAllocationDecider#CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING} / {@link EnableAllocationDecider#INDEX_ROUTING_REBALANCE_ENABLE}
* via cluster / index settings.
*/
public enum Rebalance {

View File

@ -25,10 +25,9 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.Map;
import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.AND;
import static org.elasticsearch.cluster.node.DiscoveryNodeFilters.OpType.OR;
@ -65,36 +64,23 @@ public class FilterAllocationDecider extends AllocationDecider {
public static final String INDEX_ROUTING_INCLUDE_GROUP = "index.routing.allocation.include.";
public static final String INDEX_ROUTING_EXCLUDE_GROUP = "index.routing.allocation.exclude.";
public static final String CLUSTER_ROUTING_REQUIRE_GROUP = "cluster.routing.allocation.require.";
public static final String CLUSTER_ROUTING_INCLUDE_GROUP = "cluster.routing.allocation.include.";
public static final String CLUSTER_ROUTING_EXCLUDE_GROUP = "cluster.routing.allocation.exclude.";
public static final Setting<Settings> CLUSTER_ROUTING_REQUIRE_GROUP_SETTING = Setting.groupSetting("cluster.routing.allocation.require.", true, Setting.Scope.CLUSTER);
public static final Setting<Settings> CLUSTER_ROUTING_INCLUDE_GROUP_SETTING = Setting.groupSetting("cluster.routing.allocation.include.", true, Setting.Scope.CLUSTER);
public static final Setting<Settings> CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING = Setting.groupSetting("cluster.routing.allocation.exclude.", true, Setting.Scope.CLUSTER);
private volatile DiscoveryNodeFilters clusterRequireFilters;
private volatile DiscoveryNodeFilters clusterIncludeFilters;
private volatile DiscoveryNodeFilters clusterExcludeFilters;
@Inject
public FilterAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public FilterAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
Map<String, String> requireMap = settings.getByPrefix(CLUSTER_ROUTING_REQUIRE_GROUP).getAsMap();
if (requireMap.isEmpty()) {
clusterRequireFilters = null;
} else {
clusterRequireFilters = DiscoveryNodeFilters.buildFromKeyValue(AND, requireMap);
}
Map<String, String> includeMap = settings.getByPrefix(CLUSTER_ROUTING_INCLUDE_GROUP).getAsMap();
if (includeMap.isEmpty()) {
clusterIncludeFilters = null;
} else {
clusterIncludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, includeMap);
}
Map<String, String> excludeMap = settings.getByPrefix(CLUSTER_ROUTING_EXCLUDE_GROUP).getAsMap();
if (excludeMap.isEmpty()) {
clusterExcludeFilters = null;
} else {
clusterExcludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, excludeMap);
}
nodeSettingsService.addListener(new ApplySettings());
setClusterRequireFilters(CLUSTER_ROUTING_REQUIRE_GROUP_SETTING.get(settings));
setClusterExcludeFilters(CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING.get(settings));
setClusterIncludeFilters(CLUSTER_ROUTING_INCLUDE_GROUP_SETTING.get(settings));
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_REQUIRE_GROUP_SETTING, this::setClusterRequireFilters);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING, this::setClusterExcludeFilters);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_INCLUDE_GROUP_SETTING, this::setClusterIncludeFilters);
}
@Override
@ -144,21 +130,13 @@ public class FilterAllocationDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME, "node passes include/exclude/require filters");
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
Map<String, String> requireMap = settings.getByPrefix(CLUSTER_ROUTING_REQUIRE_GROUP).getAsMap();
if (!requireMap.isEmpty()) {
clusterRequireFilters = DiscoveryNodeFilters.buildFromKeyValue(AND, requireMap);
}
Map<String, String> includeMap = settings.getByPrefix(CLUSTER_ROUTING_INCLUDE_GROUP).getAsMap();
if (!includeMap.isEmpty()) {
clusterIncludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, includeMap);
}
Map<String, String> excludeMap = settings.getByPrefix(CLUSTER_ROUTING_EXCLUDE_GROUP).getAsMap();
if (!excludeMap.isEmpty()) {
clusterExcludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, excludeMap);
private void setClusterRequireFilters(Settings settings) {
clusterRequireFilters = DiscoveryNodeFilters.buildFromKeyValue(AND, settings.getAsMap());
}
private void setClusterIncludeFilters(Settings settings) {
clusterIncludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, settings.getAsMap());
}
private void setClusterExcludeFilters(Settings settings) {
clusterExcludeFilters = DiscoveryNodeFilters.buildFromKeyValue(OR, settings.getAsMap());
}
}

View File

@ -24,16 +24,16 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* This {@link AllocationDecider} limits the number of shards per node on a per
* index or node-wide basis. The allocator prevents a single node to hold more
* than {@value #INDEX_TOTAL_SHARDS_PER_NODE} per index and
* {@value #CLUSTER_TOTAL_SHARDS_PER_NODE} globally during the allocation
* <tt>cluster.routing.allocation.total_shards_per_node</tt> globally during the allocation
* process. The limits of this decider can be changed in real-time via a the
* index settings API.
* <p>
@ -64,26 +64,18 @@ public class ShardsLimitAllocationDecider extends AllocationDecider {
* Controls the maximum number of shards per node on a global level.
* Negative values are interpreted as unlimited.
*/
public static final String CLUSTER_TOTAL_SHARDS_PER_NODE = "cluster.routing.allocation.total_shards_per_node";
public static final Setting<Integer> CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING = Setting.intSetting("cluster.routing.allocation.total_shards_per_node", -1, true, Setting.Scope.CLUSTER);
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
Integer newClusterLimit = settings.getAsInt(CLUSTER_TOTAL_SHARDS_PER_NODE, null);
if (newClusterLimit != null) {
logger.info("updating [{}] from [{}] to [{}]", CLUSTER_TOTAL_SHARDS_PER_NODE,
ShardsLimitAllocationDecider.this.clusterShardLimit, newClusterLimit);
ShardsLimitAllocationDecider.this.clusterShardLimit = newClusterLimit;
}
}
}
@Inject
public ShardsLimitAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public ShardsLimitAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.clusterShardLimit = settings.getAsInt(CLUSTER_TOTAL_SHARDS_PER_NODE, -1);
nodeSettingsService.addListener(new ApplySettings());
this.clusterShardLimit = CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING, this::setClusterShardLimit);
}
private void setClusterShardLimit(int clusterShardLimit) {
this.clusterShardLimit = clusterShardLimit;
}
@Override

View File

@ -23,9 +23,10 @@ import org.elasticsearch.cluster.SnapshotsInProgress;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* This {@link org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider} prevents shards that
@ -38,18 +39,7 @@ public class SnapshotInProgressAllocationDecider extends AllocationDecider {
/**
* Disables relocation of shards that are currently being snapshotted.
*/
public static final String CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED = "cluster.routing.allocation.snapshot.relocation_enabled";
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
boolean newEnableRelocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED, enableRelocation);
if (newEnableRelocation != enableRelocation) {
logger.info("updating [{}] from [{}], to [{}]", CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED, enableRelocation, newEnableRelocation);
enableRelocation = newEnableRelocation;
}
}
}
public static final Setting<Boolean> CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING = Setting.boolSetting("cluster.routing.allocation.snapshot.relocation_enabled", false, true, Setting.Scope.CLUSTER);
private volatile boolean enableRelocation = false;
@ -66,14 +56,18 @@ public class SnapshotInProgressAllocationDecider extends AllocationDecider {
* @param settings {@link org.elasticsearch.common.settings.Settings} to use
*/
public SnapshotInProgressAllocationDecider(Settings settings) {
this(settings, new NodeSettingsService(settings));
this(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
}
@Inject
public SnapshotInProgressAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public SnapshotInProgressAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
enableRelocation = settings.getAsBoolean(CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED, enableRelocation);
nodeSettingsService.addListener(new ApplySettings());
enableRelocation = CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING, this::setEnableRelocation);
}
private void setEnableRelocation(boolean enableRelocation) {
this.enableRelocation = enableRelocation;
}
/**

View File

@ -19,13 +19,14 @@
package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* {@link ThrottlingAllocationDecider} controls the recovery process per node in
@ -47,27 +48,33 @@ import org.elasticsearch.node.settings.NodeSettingsService;
*/
public class ThrottlingAllocationDecider extends AllocationDecider {
public static final String NAME = "throttling";
public static final String CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES = "cluster.routing.allocation.node_initial_primaries_recoveries";
public static final String CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES = "cluster.routing.allocation.node_concurrent_recoveries";
public static final String CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES = "cluster.routing.allocation.concurrent_recoveries";
public static final int DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES = 2;
public static final int DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES = 4;
public static final String NAME = "throttling";
public static final String CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES = "cluster.routing.allocation.concurrent_recoveries";
public static final Setting<Integer> CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING = Setting.intSetting("cluster.routing.allocation.node_initial_primaries_recoveries", DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, 0, true, Setting.Scope.CLUSTER);
public static final Setting<Integer> CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING = new Setting<>("cluster.routing.allocation.node_concurrent_recoveries", (s) -> s.get(CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES,Integer.toString(DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES)), (s) -> Setting.parseInt(s, 0, "cluster.routing.allocation.node_concurrent_recoveries"), true, Setting.Scope.CLUSTER);
private volatile int primariesInitialRecoveries;
private volatile int concurrentRecoveries;
@Inject
public ThrottlingAllocationDecider(Settings settings, NodeSettingsService nodeSettingsService) {
public ThrottlingAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.primariesInitialRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES);
this.concurrentRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, DEFAULT_CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES));
this.primariesInitialRecoveries = CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING.get(settings);
this.concurrentRecoveries = CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.get(settings);
logger.debug("using node_concurrent_recoveries [{}], node_initial_primaries_recoveries [{}]", concurrentRecoveries, primariesInitialRecoveries);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING, this::setPrimariesInitialRecoveries);
clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING, this::setConcurrentRecoveries);
}
nodeSettingsService.addListener(new ApplySettings());
private void setConcurrentRecoveries(int concurrentRecoveries) {
this.concurrentRecoveries = concurrentRecoveries;
}
private void setPrimariesInitialRecoveries(int primariesInitialRecoveries) {
this.primariesInitialRecoveries = primariesInitialRecoveries;
}
@Override
@ -115,21 +122,4 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME, "below shard recovery limit of [%d]", concurrentRecoveries);
}
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int primariesInitialRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, ThrottlingAllocationDecider.this.primariesInitialRecoveries);
if (primariesInitialRecoveries != ThrottlingAllocationDecider.this.primariesInitialRecoveries) {
logger.info("updating [cluster.routing.allocation.node_initial_primaries_recoveries] from [{}] to [{}]", ThrottlingAllocationDecider.this.primariesInitialRecoveries, primariesInitialRecoveries);
ThrottlingAllocationDecider.this.primariesInitialRecoveries = primariesInitialRecoveries;
}
int concurrentRecoveries = settings.getAsInt(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, ThrottlingAllocationDecider.this.concurrentRecoveries);
if (concurrentRecoveries != ThrottlingAllocationDecider.this.concurrentRecoveries) {
logger.info("updating [cluster.routing.allocation.node_concurrent_recoveries] from [{}] to [{}]", ThrottlingAllocationDecider.this.concurrentRecoveries, concurrentRecoveries);
ThrottlingAllocationDecider.this.concurrentRecoveries = concurrentRecoveries;
}
}
}
}

View File

@ -49,6 +49,8 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.transport.TransportAddress;
@ -63,7 +65,6 @@ import org.elasticsearch.common.util.concurrent.PrioritizedRunnable;
import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoveryService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
@ -91,8 +92,8 @@ import static org.elasticsearch.common.util.concurrent.EsExecutors.daemonThreadF
*/
public class InternalClusterService extends AbstractLifecycleComponent<ClusterService> implements ClusterService {
public static final String SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD = "cluster.service.slow_task_logging_threshold";
public static final String SETTING_CLUSTER_SERVICE_RECONNECT_INTERVAL = "cluster.service.reconnect_interval";
public static final Setting<TimeValue> CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING = Setting.positiveTimeSetting("cluster.service.slow_task_logging_threshold", TimeValue.timeValueSeconds(30), true, Setting.Scope.CLUSTER);
public static final Setting<TimeValue> CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING = Setting.positiveTimeSetting("cluster.service.reconnect_interval", TimeValue.timeValueSeconds(10), false, Setting.Scope.CLUSTER);
public static final String UPDATE_THREAD_NAME = "clusterService#updateTask";
private final ThreadPool threadPool;
@ -103,7 +104,7 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
private final TransportService transportService;
private final NodeSettingsService nodeSettingsService;
private final ClusterSettings clusterSettings;
private final DiscoveryNodeService discoveryNodeService;
private final Version version;
@ -136,33 +137,32 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
@Inject
public InternalClusterService(Settings settings, DiscoveryService discoveryService, OperationRouting operationRouting, TransportService transportService,
NodeSettingsService nodeSettingsService, ThreadPool threadPool, ClusterName clusterName, DiscoveryNodeService discoveryNodeService, Version version) {
ClusterSettings clusterSettings, ThreadPool threadPool, ClusterName clusterName, DiscoveryNodeService discoveryNodeService, Version version) {
super(settings);
this.operationRouting = operationRouting;
this.transportService = transportService;
this.discoveryService = discoveryService;
this.threadPool = threadPool;
this.nodeSettingsService = nodeSettingsService;
this.clusterSettings = clusterSettings;
this.discoveryNodeService = discoveryNodeService;
this.version = version;
// will be replaced on doStart.
this.clusterState = ClusterState.builder(clusterName).build();
this.nodeSettingsService.setClusterService(this);
this.nodeSettingsService.addListener(new ApplySettings());
this.clusterSettings.addSettingsUpdateConsumer(CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING, this::setSlowTaskLoggingThreshold);
this.reconnectInterval = this.settings.getAsTime(SETTING_CLUSTER_SERVICE_RECONNECT_INTERVAL, TimeValue.timeValueSeconds(10));
this.reconnectInterval = CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING.get(settings);
this.slowTaskLoggingThreshold = this.settings.getAsTime(SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, TimeValue.timeValueSeconds(30));
this.slowTaskLoggingThreshold = CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(settings);
localNodeMasterListeners = new LocalNodeMasterListeners(threadPool);
initialBlocks = ClusterBlocks.builder().addGlobalBlock(discoveryService.getNoMasterBlock());
}
public NodeSettingsService settingsService() {
return this.nodeSettingsService;
private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) {
this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
@Override
@ -566,6 +566,15 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
// update the current cluster state
clusterState = newClusterState;
logger.debug("set local cluster state to version {}", newClusterState.version());
try {
// nothing to do until we actually recover from the gateway or any other block indicates we need to disable persistency
if (clusterChangedEvent.state().blocks().disableStatePersistence() == false && clusterChangedEvent.metaDataChanged()) {
final Settings incomingSettings = clusterChangedEvent.state().metaData().settings();
clusterSettings.applySettings(incomingSettings);
}
} catch (Exception ex) {
logger.warn("failed to apply cluster settings", ex);
}
for (ClusterStateListener listener : preAppliedListeners) {
try {
listener.clusterChanged(clusterChangedEvent);
@ -893,12 +902,4 @@ public class InternalClusterService extends AbstractLifecycleComponent<ClusterSe
}
}
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
final TimeValue slowTaskLoggingThreshold = settings.getAsTime(SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, InternalClusterService.this.slowTaskLoggingThreshold);
InternalClusterService.this.slowTaskLoggingThreshold = slowTaskLoggingThreshold;
}
}
}

View File

@ -1,38 +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.cluster.settings;
import org.elasticsearch.common.inject.BindingAnnotation;
import java.lang.annotation.Documented;
import java.lang.annotation.Retention;
import java.lang.annotation.Target;
import static java.lang.annotation.ElementType.FIELD;
import static java.lang.annotation.ElementType.PARAMETER;
import static java.lang.annotation.RetentionPolicy.RUNTIME;
@BindingAnnotation
@Target({FIELD, PARAMETER})
@Retention(RUNTIME)
@Documented
public @interface ClusterDynamicSettings {
}

View File

@ -84,7 +84,6 @@ public class Booleans {
* throws exception if string cannot be parsed to boolean
*/
public static Boolean parseBooleanExact(String value) {
boolean isFalse = isExplicitFalse(value);
if (isFalse) {
return false;
@ -94,7 +93,7 @@ public class Booleans {
return true;
}
throw new IllegalArgumentException("value cannot be parsed to boolean [ true/1/on/yes OR false/0/off/no ] ");
throw new IllegalArgumentException("Failed to parse value [" + value + "] cannot be parsed to boolean [ true/1/on/yes OR false/0/off/no ]");
}
public static Boolean parseBoolean(String value, Boolean defaultValue) {

View File

@ -0,0 +1,252 @@
/*
* 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.common.settings;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.common.component.AbstractComponent;
import java.util.*;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
/**
* A basic setting service that can be used for per-index and per-cluster settings.
* This service offers transactional application of updates settings.
*/
public abstract class AbstractScopedSettings extends AbstractComponent {
private Settings lastSettingsApplied = Settings.EMPTY;
private final List<SettingUpdater> settingUpdaters = new ArrayList<>();
private final Map<String, Setting<?>> complexMatchers = new HashMap<>();
private final Map<String, Setting<?>> keySettings = new HashMap<>();
private final Setting.Scope scope;
protected AbstractScopedSettings(Settings settings, Set<Setting<?>> settingsSet, Setting.Scope scope) {
super(settings);
for (Setting<?> entry : settingsSet) {
if (entry.getScope() != scope) {
throw new IllegalArgumentException("Setting must be a cluster setting but was: " + entry.getScope());
}
if (entry.hasComplexMatcher()) {
complexMatchers.put(entry.getKey(), entry);
} else {
keySettings.put(entry.getKey(), entry);
}
}
this.scope = scope;
}
public Setting.Scope getScope() {
return this.scope;
}
/**
* Applies the given settings to all listeners and rolls back the result after application. This
* method will not change any settings but will fail if any of the settings can't be applied.
*/
public synchronized Settings dryRun(Settings settings) {
final Settings current = Settings.builder().put(this.settings).put(settings).build();
final Settings previous = Settings.builder().put(this.settings).put(this.lastSettingsApplied).build();
List<RuntimeException> exceptions = new ArrayList<>();
for (SettingUpdater settingUpdater : settingUpdaters) {
try {
if (settingUpdater.hasChanged(current, previous)) {
settingUpdater.getValue(current, previous);
}
} catch (RuntimeException ex) {
exceptions.add(ex);
logger.debug("failed to prepareCommit settings for [{}]", ex, settingUpdater);
}
}
// here we are exhaustive and record all settings that failed.
ExceptionsHelper.rethrowAndSuppress(exceptions);
return current;
}
/**
* Applies the given settings to all the settings consumers or to none of them. The settings
* will be merged with the node settings before they are applied while given settings override existing node
* settings.
* @param newSettings the settings to apply
* @return the unmerged applied settings
*/
public synchronized Settings applySettings(Settings newSettings) {
if (lastSettingsApplied != null && newSettings.equals(lastSettingsApplied)) {
// nothing changed in the settings, ignore
return newSettings;
}
final Settings current = Settings.builder().put(this.settings).put(newSettings).build();
final Settings previous = Settings.builder().put(this.settings).put(this.lastSettingsApplied).build();
try {
List<Runnable> applyRunnables = new ArrayList<>();
for (SettingUpdater settingUpdater : settingUpdaters) {
try {
applyRunnables.add(settingUpdater.updater(current, previous));
} catch (Exception ex) {
logger.warn("failed to prepareCommit settings for [{}]", ex, settingUpdater);
throw ex;
}
}
for (Runnable settingUpdater : applyRunnables) {
settingUpdater.run();
}
} catch (Exception ex) {
logger.warn("failed to apply settings", ex);
throw ex;
} finally {
}
return lastSettingsApplied = newSettings;
}
/**
* Adds a settings consumer with a predicate that is only evaluated at update time.
* <p>
* Note: Only settings registered in {@link SettingsModule} can be changed dynamically.
* </p>
* @param validator an additional validator that is only applied to updates of this setting.
* This is useful to add additional validation to settings at runtime compared to at startup time.
*/
public synchronized <T> void addSettingsUpdateConsumer(Setting<T> setting, Consumer<T> consumer, Consumer<T> validator) {
if (setting != get(setting.getKey())) {
throw new IllegalArgumentException("Setting is not registered for key [" + setting.getKey() + "]");
}
this.settingUpdaters.add(setting.newUpdater(consumer, logger, validator));
}
/**
* Adds a settings consumer that accepts the values for two settings. The consumer if only notified if one or both settings change.
* <p>
* Note: Only settings registered in {@link SettingsModule} can be changed dynamically.
* </p>
* This method registers a compound updater that is useful if two settings are depending on each other. The consumer is always provided
* with both values even if only one of the two changes.
*/
public synchronized <A, B> void addSettingsUpdateConsumer(Setting<A> a, Setting<B> b, BiConsumer<A, B> consumer) {
if (a != get(a.getKey())) {
throw new IllegalArgumentException("Setting is not registered for key [" + a.getKey() + "]");
}
if (b != get(b.getKey())) {
throw new IllegalArgumentException("Setting is not registered for key [" + b.getKey() + "]");
}
this.settingUpdaters.add(Setting.compoundUpdater(consumer, a, b, logger));
}
/**
* Adds a settings consumer.
* <p>
* Note: Only settings registered in {@link org.elasticsearch.cluster.ClusterModule} can be changed dynamically.
* </p>
*/
public synchronized <T> void addSettingsUpdateConsumer(Setting<T> setting, Consumer<T> consumer) {
addSettingsUpdateConsumer(setting, consumer, (s) -> {});
}
/**
* Transactional interface to update settings.
* @see Setting
*/
public interface SettingUpdater<T> {
/**
* Returns true if this updaters setting has changed with the current update
* @param current the current settings
* @param previous the previous setting
* @return true if this updaters setting has changed with the current update
*/
boolean hasChanged(Settings current, Settings previous);
/**
* Returns the instance value for the current settings. This method is stateless and idempotent.
* This method will throw an exception if the source of this value is invalid.
*/
T getValue(Settings current, Settings previous);
/**
* Applies the given value to the updater. This methods will actually run the update.
*/
void apply(T value, Settings current, Settings previous);
/**
* Updates this updaters value if it has changed.
* @return <code>true</code> iff the value has been updated.
*/
default boolean apply(Settings current, Settings previous) {
if (hasChanged(current, previous)) {
T value = getValue(current, previous);
apply(value, current, previous);
return true;
}
return false;
}
/**
* Returns a callable runnable that calls {@link #apply(Object, Settings, Settings)} if the settings
* actually changed. This allows to defer the update to a later point in time while keeping type safety.
* If the value didn't change the returned runnable is a noop.
*/
default Runnable updater(Settings current, Settings previous) {
if (hasChanged(current, previous)) {
T value = getValue(current, previous);
return () -> { apply(value, current, previous);};
}
return () -> {};
}
}
/**
* Returns the {@link Setting} for the given key or <code>null</code> if the setting can not be found.
*/
public Setting get(String key) {
Setting<?> setting = keySettings.get(key);
if (setting == null) {
for (Map.Entry<String, Setting<?>> entry : complexMatchers.entrySet()) {
if (entry.getValue().match(key)) {
return entry.getValue();
}
}
} else {
return setting;
}
return null;
}
/**
* Returns <code>true</code> if the setting for the given key is dynamically updateable. Otherwise <code>false</code>.
*/
public boolean hasDynamicSetting(String key) {
final Setting setting = get(key);
return setting != null && setting.isDynamic();
}
/**
* Returns a settings object that contains all settings that are not
* already set in the given source. The diff contains either the default value for each
* setting or the settings value in the given default settings.
*/
public Settings diff(Settings source, Settings defaultSettings) {
Settings.Builder builder = Settings.builder();
for (Setting<?> setting : keySettings.values()) {
if (setting.exists(source) == false) {
builder.put(setting.getKey(), setting.getRaw(defaultSettings));
}
}
return builder.build();
}
}

View File

@ -0,0 +1,141 @@
/*
* 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.common.settings;
import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction;
import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.*;
import org.elasticsearch.cluster.service.InternalClusterService;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.ZenDiscovery;
import org.elasticsearch.discovery.zen.elect.ElectMasterService;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.search.SearchService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportService;
import java.util.*;
/**
* Encapsulates all valid cluster level settings.
*/
public final class ClusterSettings extends AbstractScopedSettings {
public ClusterSettings(Settings settings, Set<Setting<?>> settingsSet) {
super(settings, settingsSet, Setting.Scope.CLUSTER);
}
@Override
public synchronized Settings applySettings(Settings newSettings) {
Settings settings = super.applySettings(newSettings);
try {
for (Map.Entry<String, String> entry : settings.getAsMap().entrySet()) {
if (entry.getKey().startsWith("logger.")) {
String component = entry.getKey().substring("logger.".length());
if ("_root".equals(component)) {
ESLoggerFactory.getRootLogger().setLevel(entry.getValue());
} else {
ESLoggerFactory.getLogger(component).setLevel(entry.getValue());
}
}
}
} catch (Exception e) {
logger.warn("failed to refresh settings for [{}]", e, "logger");
}
return settings;
}
/**
* Returns <code>true</code> if the settings is a logger setting.
*/
public boolean isLoggerSetting(String key) {
return key.startsWith("logger.");
}
public static Set<Setting<?>> BUILT_IN_CLUSTER_SETTINGS = Collections.unmodifiableSet(new HashSet<>(Arrays.asList(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING,
AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING,
BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING,
BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING,
BalancedShardsAllocator.THRESHOLD_SETTING,
ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING,
ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING,
EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING,
EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING,
ZenDiscovery.REJOIN_ON_MASTER_GONE_SETTING,
FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP_SETTING,
FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING,
FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING,
IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING,
IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING,
IndicesTTLService.INDICES_TTL_INTERVAL_SETTING,
MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT_SETTING,
MetaData.SETTING_READ_ONLY_SETTING,
RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS_SETTING,
RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS_SETTING,
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING,
RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING,
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING,
RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING,
RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING,
ThreadPool.THREADPOOL_GROUP_SETTING,
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING,
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING,
DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING,
InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING,
SnapshotInProgressAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED_SETTING,
DestructiveOperations.REQUIRES_NAME_SETTING,
DiscoverySettings.PUBLISH_TIMEOUT_SETTING,
DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING,
DiscoverySettings.COMMIT_TIMEOUT_SETTING,
DiscoverySettings.NO_MASTER_BLOCK_SETTING,
HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING,
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING,
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING,
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING,
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING,
InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING,
SearchService.DEFAULT_SEARCH_TIMEOUT_SETTING,
ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING,
TransportService.TRACE_LOG_EXCLUDE_SETTING,
TransportService.TRACE_LOG_INCLUDE_SETTING,
TransportCloseIndexAction.CLUSTER_INDICES_CLOSE_ENABLE_SETTING,
ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING,
InternalClusterService.CLUSTER_SERVICE_RECONNECT_INTERVAL_SETTING,
HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING,
HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_TYPE_SETTING,
Transport.TRANSPORT_PROFILES_SETTING,
Transport.TRANSPORT_TCP_COMPRESS)));
}

View File

@ -0,0 +1,461 @@
/*
* 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.common.settings;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.MemorySizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.*;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.regex.Pattern;
/**
*/
public class Setting<T> extends ToXContentToBytes {
private final String key;
protected final Function<Settings, String> defaultValue;
private final Function<String, T> parser;
private final boolean dynamic;
private final Scope scope;
/**
* Creates a new Setting instance
* @param key the settings key for this setting.
* @param defaultValue a default value function that returns the default values string representation.
* @param parser a parser that parses the string rep into a complex datatype.
* @param dynamic true iff this setting can be dynamically updateable
* @param scope the scope of this setting
*/
public Setting(String key, Function<Settings, String> defaultValue, Function<String, T> parser, boolean dynamic, Scope scope) {
assert parser.apply(defaultValue.apply(Settings.EMPTY)) != null || this.isGroupSetting(): "parser returned null";
this.key = key;
this.defaultValue = defaultValue;
this.parser = parser;
this.dynamic = dynamic;
this.scope = scope;
}
/**
* Returns the settings key or a prefix if this setting is a group setting
* @see #isGroupSetting()
*/
public final String getKey() {
return key;
}
/**
* Returns <code>true</code> iff this setting is dynamically updateable, otherwise <code>false</code>
*/
public final boolean isDynamic() {
return dynamic;
}
/**
* Returns the settings scope
*/
public final Scope getScope() {
return scope;
}
/**
* Returns <code>true</code> iff this setting is a group setting. Group settings represent a set of settings
* rather than a single value. The key, see {@link #getKey()}, in contrast to non-group settings is a prefix like <tt>cluster.store.</tt>
* that matches all settings with this prefix.
*/
boolean isGroupSetting() {
return false;
}
boolean hasComplexMatcher() {
return isGroupSetting();
}
/**
* Returns the default values string representation for this setting.
* @param settings a settings object for settings that has a default value depending on another setting if available
*/
public final String getDefault(Settings settings) {
return defaultValue.apply(settings);
}
/**
* Returns <code>true</code> iff this setting is present in the given settings object. Otherwise <code>false</code>
*/
public final boolean exists(Settings settings) {
return settings.get(key) != null;
}
/**
* Returns the settings value. If the setting is not present in the given settings object the default value is returned
* instead.
*/
public T get(Settings settings) {
String value = getRaw(settings);
try {
return parser.apply(value);
} catch (ElasticsearchParseException ex) {
throw new IllegalArgumentException(ex.getMessage(), ex);
} catch (NumberFormatException ex) {
throw new IllegalArgumentException("Failed to parse value [" + value + "] for setting [" + getKey() + "]", ex);
} catch (IllegalArgumentException ex) {
throw ex;
} catch (Exception t) {
throw new IllegalArgumentException("Failed to parse value [" + value + "] for setting [" + getKey() + "]", t);
}
}
/**
* Returns the raw (string) settings value. If the setting is not present in the given settings object the default value is returned
* instead. This is useful if the value can't be parsed due to an invalid value to access the actual value.
*/
public String getRaw(Settings settings) {
return settings.get(key, defaultValue.apply(settings));
}
/**
* Returns <code>true</code> iff the given key matches the settings key or if this setting is a group setting if the
* given key is part of the settings group.
* @see #isGroupSetting()
*/
public boolean match(String toTest) {
return key.equals(toTest);
}
@Override
public final XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject();
builder.field("key", key);
builder.field("type", scope.name());
builder.field("dynamic", dynamic);
builder.field("is_group_setting", isGroupSetting());
builder.field("default", defaultValue.apply(Settings.EMPTY));
builder.endObject();
return builder;
}
/**
* The settings scope - settings can either be cluster settings or per index settings.
*/
public enum Scope {
CLUSTER,
INDEX;
}
final AbstractScopedSettings.SettingUpdater newUpdater(Consumer<T> consumer, ESLogger logger) {
return newUpdater(consumer, logger, (s) -> {});
}
AbstractScopedSettings.SettingUpdater newUpdater(Consumer<T> consumer, ESLogger logger, Consumer<T> validator) {
if (isDynamic()) {
return new Updater(consumer, logger, validator);
} else {
throw new IllegalStateException("setting [" + getKey() + "] is not dynamic");
}
}
/**
* this is used for settings that depend on each other... see {@link org.elasticsearch.common.settings.AbstractScopedSettings#addSettingsUpdateConsumer(Setting, Setting, BiConsumer)} and it's
* usage for details.
*/
static <A, B> AbstractScopedSettings.SettingUpdater<Tuple<A, B>> compoundUpdater(final BiConsumer<A,B> consumer, final Setting<A> aSettting, final Setting<B> bSetting, ESLogger logger) {
final AbstractScopedSettings.SettingUpdater<A> aSettingUpdater = aSettting.newUpdater(null, logger);
final AbstractScopedSettings.SettingUpdater<B> bSettingUpdater = bSetting.newUpdater(null, logger);
return new AbstractScopedSettings.SettingUpdater<Tuple<A, B>>() {
@Override
public boolean hasChanged(Settings current, Settings previous) {
return aSettingUpdater.hasChanged(current, previous) || bSettingUpdater.hasChanged(current, previous);
}
@Override
public Tuple<A, B> getValue(Settings current, Settings previous) {
return new Tuple<>(aSettingUpdater.getValue(current, previous), bSettingUpdater.getValue(current, previous));
}
@Override
public void apply(Tuple<A, B> value, Settings current, Settings previous) {
consumer.accept(value.v1(), value.v2());
}
@Override
public String toString() {
return "CompoundUpdater for: " + aSettingUpdater + " and " + bSettingUpdater;
}
};
}
private class Updater implements AbstractScopedSettings.SettingUpdater<T> {
private final Consumer<T> consumer;
private final ESLogger logger;
private final Consumer<T> accept;
public Updater(Consumer<T> consumer, ESLogger logger, Consumer<T> accept) {
this.consumer = consumer;
this.logger = logger;
this.accept = accept;
}
@Override
public String toString() {
return "Updater for: " + Setting.this.toString();
}
@Override
public boolean hasChanged(Settings current, Settings previous) {
final String newValue = getRaw(current);
final String value = getRaw(previous);
assert isGroupSetting() == false : "group settings must override this method";
assert value != null : "value was null but can't be unless default is null which is invalid";
return value.equals(newValue) == false;
}
@Override
public T getValue(Settings current, Settings previous) {
final String newValue = getRaw(current);
final String value = getRaw(previous);
T inst = get(current);
try {
accept.accept(inst);
} catch (Exception | AssertionError e) {
throw new IllegalArgumentException("illegal value can't update [" + key + "] from [" + value + "] to [" + newValue + "]", e);
}
return inst;
}
@Override
public void apply(T value, Settings current, Settings previous) {
logger.info("update [{}] from [{}] to [{}]", key, getRaw(previous), getRaw(current));
consumer.accept(value);
}
}
public Setting(String key, String defaultValue, Function<String, T> parser, boolean dynamic, Scope scope) {
this(key, (s) -> defaultValue, parser, dynamic, scope);
}
public static Setting<Float> floatSetting(String key, float defaultValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> Float.toString(defaultValue), Float::parseFloat, dynamic, scope);
}
public static Setting<Float> floatSetting(String key, float defaultValue, float minValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> Float.toString(defaultValue), (s) -> {
float value = Float.parseFloat(s);
if (value < minValue) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be >= " + minValue);
}
return value;
}, dynamic, scope);
}
public static Setting<Integer> intSetting(String key, int defaultValue, int minValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> Integer.toString(defaultValue), (s) -> parseInt(s, minValue, key), dynamic, scope);
}
public static int parseInt(String s, int minValue, String key) {
int value = Integer.parseInt(s);
if (value < minValue) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be >= " + minValue);
}
return value;
}
public static Setting<Integer> intSetting(String key, int defaultValue, boolean dynamic, Scope scope) {
return intSetting(key, defaultValue, Integer.MIN_VALUE, dynamic, scope);
}
public static Setting<Boolean> boolSetting(String key, boolean defaultValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> Boolean.toString(defaultValue), Booleans::parseBooleanExact, dynamic, scope);
}
public static Setting<ByteSizeValue> byteSizeSetting(String key, String percentage, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> percentage, (s) -> MemorySizeValue.parseBytesSizeValueOrHeapRatio(s, key), dynamic, scope);
}
public static Setting<ByteSizeValue> byteSizeSetting(String key, ByteSizeValue value, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> value.toString(), (s) -> ByteSizeValue.parseBytesSizeValue(s, key), dynamic, scope);
}
public static Setting<TimeValue> positiveTimeSetting(String key, TimeValue defaultValue, boolean dynamic, Scope scope) {
return timeSetting(key, defaultValue, TimeValue.timeValueMillis(0), dynamic, scope);
}
public static <T> Setting<List<T>> listSetting(String key, List<String> defaultStringValue, Function<String, T> singleValueParser, boolean dynamic, Scope scope) {
return listSetting(key, (s) -> defaultStringValue, singleValueParser, dynamic, scope);
}
public static <T> Setting<List<T>> listSetting(String key, Function<Settings, List<String>> defaultStringValue, Function<String, T> singleValueParser, boolean dynamic, Scope scope) {
Function<String, List<T>> parser = (s) -> {
try (XContentParser xContentParser = XContentType.JSON.xContent().createParser(s)){
XContentParser.Token token = xContentParser.nextToken();
if (token != XContentParser.Token.START_ARRAY) {
throw new IllegalArgumentException("expected START_ARRAY but got " + token);
}
ArrayList<T> list = new ArrayList<>();
while ((token = xContentParser.nextToken()) !=XContentParser.Token.END_ARRAY) {
if (token != XContentParser.Token.VALUE_STRING) {
throw new IllegalArgumentException("expected VALUE_STRING but got " + token);
}
list.add(singleValueParser.apply(xContentParser.text()));
}
return list;
} catch (IOException e) {
throw new IllegalArgumentException("failed to parse array", e);
}
};
return new Setting<List<T>>(key, (s) -> arrayToParsableString(defaultStringValue.apply(s).toArray(Strings.EMPTY_ARRAY)), parser, dynamic, scope) {
private final Pattern pattern = Pattern.compile(Pattern.quote(key)+"(\\.\\d+)?");
@Override
public String getRaw(Settings settings) {
String[] array = settings.getAsArray(key, null);
return array == null ? defaultValue.apply(settings) : arrayToParsableString(array);
}
public boolean match(String toTest) {
return pattern.matcher(toTest).matches();
}
@Override
boolean hasComplexMatcher() {
return true;
}
};
}
private static String arrayToParsableString(String[] array) {
try {
XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent());
builder.startArray();
for (String element : array) {
builder.value(element);
}
builder.endArray();
return builder.string();
} catch (IOException ex) {
throw new ElasticsearchException(ex);
}
}
public static Setting<Settings> groupSetting(String key, boolean dynamic, Scope scope) {
if (key.endsWith(".") == false) {
throw new IllegalArgumentException("key must end with a '.'");
}
return new Setting<Settings>(key, "", (s) -> null, dynamic, scope) {
@Override
public boolean isGroupSetting() {
return true;
}
@Override
public Settings get(Settings settings) {
return settings.getByPrefix(key);
}
@Override
public boolean match(String toTest) {
return Regex.simpleMatch(key + "*", toTest);
}
@Override
public AbstractScopedSettings.SettingUpdater<Settings> newUpdater(Consumer<Settings> consumer, ESLogger logger, Consumer<Settings> validator) {
if (isDynamic() == false) {
throw new IllegalStateException("setting [" + getKey() + "] is not dynamic");
}
final Setting<?> setting = this;
return new AbstractScopedSettings.SettingUpdater<Settings>() {
@Override
public boolean hasChanged(Settings current, Settings previous) {
Settings currentSettings = get(current);
Settings previousSettings = get(previous);
return currentSettings.equals(previousSettings) == false;
}
@Override
public Settings getValue(Settings current, Settings previous) {
Settings currentSettings = get(current);
Settings previousSettings = get(previous);
try {
validator.accept(currentSettings);
} catch (Exception | AssertionError e) {
throw new IllegalArgumentException("illegal value can't update [" + key + "] from [" + previousSettings.getAsMap() + "] to [" + currentSettings.getAsMap() + "]", e);
}
return currentSettings;
}
@Override
public void apply(Settings value, Settings current, Settings previous) {
consumer.accept(value);
}
@Override
public String toString() {
return "Updater for: " + setting.toString();
}
};
}
};
}
public static Setting<TimeValue> timeSetting(String key, Function<Settings, String> defaultValue, TimeValue minValue, boolean dynamic, Scope scope) {
return new Setting<>(key, defaultValue, (s) -> {
TimeValue timeValue = TimeValue.parseTimeValue(s, null, key);
if (timeValue.millis() < minValue.millis()) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be >= " + minValue);
}
return timeValue;
}, dynamic, scope);
}
public static Setting<TimeValue> timeSetting(String key, TimeValue defaultValue, TimeValue minValue, boolean dynamic, Scope scope) {
return timeSetting(key, (s) -> defaultValue.getStringRep(), minValue, dynamic, scope);
}
public static Setting<TimeValue> timeSetting(String key, TimeValue defaultValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> defaultValue.toString(), (s) -> TimeValue.parseTimeValue(s, defaultValue, key), dynamic, scope);
}
public static Setting<Double> doubleSetting(String key, double defaultValue, double minValue, boolean dynamic, Scope scope) {
return new Setting<>(key, (s) -> Double.toString(defaultValue), (s) -> {
final double d = Double.parseDouble(s);
if (d < minValue) {
throw new IllegalArgumentException("Failed to parse value [" + s + "] for setting [" + key + "] must be >= " + minValue);
}
return d;
}, dynamic, scope);
}
}

View File

@ -597,6 +597,8 @@ public final class Settings implements ToXContent {
return result.toArray(new String[result.size()]);
}
/**
* Returns group settings for the given setting prefix.
*/
@ -614,6 +616,9 @@ public final class Settings implements ToXContent {
if (settingPrefix.charAt(settingPrefix.length() - 1) != '.') {
settingPrefix = settingPrefix + ".";
}
return getGroupsInternal(settingPrefix, ignoreNonGrouped);
}
private Map<String, Settings> getGroupsInternal(String settingPrefix, boolean ignoreNonGrouped) throws SettingsException {
// we don't really care that it might happen twice
Map<String, Map<String, String>> map = new LinkedHashMap<>();
for (Object o : settings.keySet()) {
@ -643,6 +648,16 @@ public final class Settings implements ToXContent {
}
return Collections.unmodifiableMap(retVal);
}
/**
* Returns group settings for the given setting prefix.
*/
public Map<String, Settings> getAsGroups() throws SettingsException {
return getAsGroups(false);
}
public Map<String, Settings> getAsGroups(boolean ignoreNonGrouped) throws SettingsException {
return getGroupsInternal("", ignoreNonGrouped);
}
/**
* Returns a parsed version.
@ -706,7 +721,7 @@ public final class Settings implements ToXContent {
Builder builder = new Builder();
int numberOfSettings = in.readVInt();
for (int i = 0; i < numberOfSettings; i++) {
builder.put(in.readString(), in.readString());
builder.put(in.readString(), in.readOptionalString());
}
return builder.build();
}
@ -715,7 +730,7 @@ public final class Settings implements ToXContent {
out.writeVInt(settings.getAsMap().size());
for (Map.Entry<String, String> entry : settings.getAsMap().entrySet()) {
out.writeString(entry.getKey());
out.writeString(entry.getValue());
out.writeOptionalString(entry.getValue());
}
}
@ -818,6 +833,10 @@ public final class Settings implements ToXContent {
return this;
}
public Builder putNull(String key) {
return put(key, (String) null);
}
/**
* Sets a setting with the provided setting key and class as value.
*

View File

@ -21,6 +21,10 @@ package org.elasticsearch.common.settings;
import org.elasticsearch.common.inject.AbstractModule;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
/**
* A module that binds the provided settings to the {@link Settings} interface.
*
@ -30,15 +34,36 @@ public class SettingsModule extends AbstractModule {
private final Settings settings;
private final SettingsFilter settingsFilter;
private final Map<String, Setting<?>> clusterDynamicSettings = new HashMap<>();
public SettingsModule(Settings settings, SettingsFilter settingsFilter) {
this.settings = settings;
this.settingsFilter = settingsFilter;
for (Setting<?> setting : ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) {
registerSetting(setting);
}
}
@Override
protected void configure() {
bind(Settings.class).toInstance(settings);
bind(SettingsFilter.class).toInstance(settingsFilter);
final ClusterSettings clusterSettings = new ClusterSettings(settings, new HashSet<>(clusterDynamicSettings.values()));
bind(ClusterSettings.class).toInstance(clusterSettings);
}
public void registerSetting(Setting<?> setting) {
switch (setting.getScope()) {
case CLUSTER:
if (clusterDynamicSettings.containsKey(setting.getKey())) {
throw new IllegalArgumentException("Cannot register setting [" + setting.getKey() + "] twice");
}
clusterDynamicSettings.put(setting.getKey(), setting);
break;
case INDEX:
throw new UnsupportedOperationException("not yet implemented");
}
}
}

View File

@ -103,9 +103,9 @@ public abstract class XContentSettingsLoader implements SettingsLoader {
} else if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
// ignore this
serializeValue(settings, sb, path, parser, currentFieldName, true);
} else {
serializeValue(settings, sb, path, parser, currentFieldName);
serializeValue(settings, sb, path, parser, currentFieldName, false);
}
}
@ -126,31 +126,33 @@ public abstract class XContentSettingsLoader implements SettingsLoader {
} else if (token == XContentParser.Token.FIELD_NAME) {
fieldName = parser.currentName();
} else if (token == XContentParser.Token.VALUE_NULL) {
serializeValue(settings, sb, path, parser, fieldName + '.' + (counter++), true);
// ignore
} else {
serializeValue(settings, sb, path, parser, fieldName + '.' + (counter++));
serializeValue(settings, sb, path, parser, fieldName + '.' + (counter++), false);
}
}
}
private void serializeValue(Map<String, String> settings, StringBuilder sb, List<String> path, XContentParser parser, String fieldName) throws IOException {
private void serializeValue(Map<String, String> settings, StringBuilder sb, List<String> path, XContentParser parser, String fieldName, boolean isNull) throws IOException {
sb.setLength(0);
for (String pathEle : path) {
sb.append(pathEle).append('.');
}
sb.append(fieldName);
String key = sb.toString();
String currentValue = parser.text();
String previousValue = settings.put(key, currentValue);
if (previousValue != null) {
String currentValue = isNull ? null : parser.text();
if (settings.containsKey(key)) {
throw new ElasticsearchParseException(
"duplicate settings key [{}] found at line number [{}], column number [{}], previous value [{}], current value [{}]",
key,
parser.getTokenLocation().lineNumber,
parser.getTokenLocation().columnNumber,
previousValue,
settings.get(key),
currentValue
);
}
settings.put(key, currentValue);
}
}

View File

@ -229,6 +229,30 @@ public class TimeValue implements Streamable {
return Strings.format1Decimals(value, suffix);
}
public String getStringRep() {
if (duration < 0) {
return Long.toString(duration);
}
switch (timeUnit) {
case NANOSECONDS:
return Strings.format1Decimals(duration, "nanos");
case MICROSECONDS:
return Strings.format1Decimals(duration, "micros");
case MILLISECONDS:
return Strings.format1Decimals(duration, "ms");
case SECONDS:
return Strings.format1Decimals(duration, "s");
case MINUTES:
return Strings.format1Decimals(duration, "m");
case HOURS:
return Strings.format1Decimals(duration, "h");
case DAYS:
return Strings.format1Decimals(duration, "d");
default:
throw new IllegalArgumentException("unknown time unit: " + timeUnit.name());
}
}
public static TimeValue parseTimeValue(String sValue, TimeValue defaultValue, String settingName) {
settingName = Objects.requireNonNull(settingName);
assert settingName.startsWith("index.") == false || MetaDataIndexUpgradeService.INDEX_TIME_SETTINGS.contains(settingName) : settingName;

View File

@ -23,9 +23,10 @@ import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.rest.RestStatus;
import java.util.EnumSet;
@ -35,42 +36,40 @@ import java.util.EnumSet;
*/
public class DiscoverySettings extends AbstractComponent {
public final static int NO_MASTER_BLOCK_ID = 2;
public final static ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL);
public final static ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
/**
* sets the timeout for a complete publishing cycle, including both sending and committing. the master
* will continute to process the next cluster state update after this time has elapsed
**/
public static final String PUBLISH_TIMEOUT = "discovery.zen.publish_timeout";
public static final Setting<TimeValue> PUBLISH_TIMEOUT_SETTING = Setting.positiveTimeSetting("discovery.zen.publish_timeout", TimeValue.timeValueSeconds(30), true, Setting.Scope.CLUSTER);
/**
* sets the timeout for receiving enough acks for a specific cluster state and committing it. failing
* to receive responses within this window will cause the cluster state change to be rejected.
*/
public static final String COMMIT_TIMEOUT = "discovery.zen.commit_timeout";
public static final String NO_MASTER_BLOCK = "discovery.zen.no_master_block";
public static final String PUBLISH_DIFF_ENABLE = "discovery.zen.publish_diff.enable";
public static final TimeValue DEFAULT_PUBLISH_TIMEOUT = TimeValue.timeValueSeconds(30);
public static final TimeValue DEFAULT_COMMIT_TIMEOUT = TimeValue.timeValueSeconds(30);
public static final String DEFAULT_NO_MASTER_BLOCK = "write";
public final static int NO_MASTER_BLOCK_ID = 2;
public final static boolean DEFAULT_PUBLISH_DIFF_ENABLE = true;
public final static ClusterBlock NO_MASTER_BLOCK_ALL = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL);
public final static ClusterBlock NO_MASTER_BLOCK_WRITES = new ClusterBlock(NO_MASTER_BLOCK_ID, "no master", true, false, RestStatus.SERVICE_UNAVAILABLE, EnumSet.of(ClusterBlockLevel.WRITE, ClusterBlockLevel.METADATA_WRITE));
public static final Setting<TimeValue> COMMIT_TIMEOUT_SETTING = new Setting<>("discovery.zen.commit_timeout", (s) -> PUBLISH_TIMEOUT_SETTING.getRaw(s), (s) -> TimeValue.parseTimeValue(s, TimeValue.timeValueSeconds(30), "discovery.zen.commit_timeout"), true, Setting.Scope.CLUSTER);
public static final Setting<ClusterBlock> NO_MASTER_BLOCK_SETTING = new Setting<>("discovery.zen.no_master_block", "write", DiscoverySettings::parseNoMasterBlock, true, Setting.Scope.CLUSTER);
public static final Setting<Boolean> PUBLISH_DIFF_ENABLE_SETTING = Setting.boolSetting("discovery.zen.publish_diff.enable", true, true, Setting.Scope.CLUSTER);
private volatile ClusterBlock noMasterBlock;
private volatile TimeValue publishTimeout;
private volatile TimeValue commitTimeout;
private volatile boolean publishDiff;
@Inject
public DiscoverySettings(Settings settings, NodeSettingsService nodeSettingsService) {
public DiscoverySettings(Settings settings, ClusterSettings clusterSettings) {
super(settings);
nodeSettingsService.addListener(new ApplySettings());
this.noMasterBlock = parseNoMasterBlock(settings.get(NO_MASTER_BLOCK, DEFAULT_NO_MASTER_BLOCK));
this.publishTimeout = settings.getAsTime(PUBLISH_TIMEOUT, DEFAULT_PUBLISH_TIMEOUT);
this.commitTimeout = settings.getAsTime(COMMIT_TIMEOUT, new TimeValue(Math.min(DEFAULT_COMMIT_TIMEOUT.millis(), publishTimeout.millis())));
this.publishDiff = settings.getAsBoolean(PUBLISH_DIFF_ENABLE, DEFAULT_PUBLISH_DIFF_ENABLE);
clusterSettings.addSettingsUpdateConsumer(NO_MASTER_BLOCK_SETTING, this::setNoMasterBlock);
clusterSettings.addSettingsUpdateConsumer(PUBLISH_DIFF_ENABLE_SETTING, this::setPublishDiff);
clusterSettings.addSettingsUpdateConsumer(COMMIT_TIMEOUT_SETTING, this::setCommitTimeout);
clusterSettings.addSettingsUpdateConsumer(PUBLISH_TIMEOUT_SETTING, this::setPublishTimeout);
this.noMasterBlock = NO_MASTER_BLOCK_SETTING.get(settings);
this.publishTimeout = PUBLISH_TIMEOUT_SETTING.get(settings);
this.commitTimeout = COMMIT_TIMEOUT_SETTING.get(settings);
this.publishDiff = PUBLISH_DIFF_ENABLE_SETTING.get(settings);
}
/**
@ -88,47 +87,25 @@ public class DiscoverySettings extends AbstractComponent {
return noMasterBlock;
}
private void setNoMasterBlock(ClusterBlock noMasterBlock) {
this.noMasterBlock = noMasterBlock;
}
private void setPublishDiff(boolean publishDiff) {
this.publishDiff = publishDiff;
}
private void setPublishTimeout(TimeValue publishTimeout) {
this.publishTimeout = publishTimeout;
}
private void setCommitTimeout(TimeValue commitTimeout) {
this.commitTimeout = commitTimeout;
}
public boolean getPublishDiff() { return publishDiff;}
private class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
TimeValue newPublishTimeout = settings.getAsTime(PUBLISH_TIMEOUT, null);
if (newPublishTimeout != null) {
if (newPublishTimeout.millis() != publishTimeout.millis()) {
logger.info("updating [{}] from [{}] to [{}]", PUBLISH_TIMEOUT, publishTimeout, newPublishTimeout);
publishTimeout = newPublishTimeout;
if (settings.getAsTime(COMMIT_TIMEOUT, null) == null && commitTimeout.millis() > publishTimeout.millis()) {
logger.info("reducing default [{}] to [{}] due to publish timeout change", COMMIT_TIMEOUT, publishTimeout);
commitTimeout = publishTimeout;
}
}
}
TimeValue newCommitTimeout = settings.getAsTime(COMMIT_TIMEOUT, null);
if (newCommitTimeout != null) {
if (newCommitTimeout.millis() != commitTimeout.millis()) {
logger.info("updating [{}] from [{}] to [{}]", COMMIT_TIMEOUT, commitTimeout, newCommitTimeout);
commitTimeout = newCommitTimeout;
}
}
String newNoMasterBlockValue = settings.get(NO_MASTER_BLOCK);
if (newNoMasterBlockValue != null) {
ClusterBlock newNoMasterBlock = parseNoMasterBlock(newNoMasterBlockValue);
if (newNoMasterBlock != noMasterBlock) {
noMasterBlock = newNoMasterBlock;
}
}
Boolean newPublishDiff = settings.getAsBoolean(PUBLISH_DIFF_ENABLE, null);
if (newPublishDiff != null) {
if (newPublishDiff != publishDiff) {
logger.info("updating [{}] from [{}] to [{}]", PUBLISH_DIFF_ENABLE, publishDiff, newPublishDiff);
publishDiff = newPublishDiff;
}
}
}
}
private ClusterBlock parseNoMasterBlock(String value) {
private static ClusterBlock parseNoMasterBlock(String value) {
switch (value) {
case "all":
return NO_MASTER_BLOCK_ALL;

View File

@ -39,6 +39,8 @@ import org.elasticsearch.common.inject.internal.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.Discovery;
@ -55,7 +57,6 @@ import org.elasticsearch.discovery.zen.ping.ZenPingService;
import org.elasticsearch.discovery.zen.publish.PendingClusterStateStats;
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
@ -74,7 +75,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds;
*/
public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implements Discovery, PingContextProvider {
public final static String SETTING_REJOIN_ON_MASTER_GONE = "discovery.zen.rejoin_on_master_gone";
public final static Setting<Boolean> REJOIN_ON_MASTER_GONE_SETTING = Setting.boolSetting("discovery.zen.rejoin_on_master_gone", true, true, Setting.Scope.CLUSTER);
public final static String SETTING_PING_TIMEOUT = "discovery.zen.ping_timeout";
public final static String SETTING_JOIN_TIMEOUT = "discovery.zen.join_timeout";
public final static String SETTING_JOIN_RETRY_ATTEMPTS = "discovery.zen.join_retry_attempts";
@ -139,7 +140,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
@Inject
public ZenDiscovery(Settings settings, ClusterName clusterName, ThreadPool threadPool,
TransportService transportService, final ClusterService clusterService, NodeSettingsService nodeSettingsService,
TransportService transportService, final ClusterService clusterService, ClusterSettings clusterSettings,
ZenPingService pingService, ElectMasterService electMasterService,
DiscoverySettings discoverySettings) {
super(settings);
@ -160,7 +161,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
this.masterElectionFilterClientNodes = settings.getAsBoolean(SETTING_MASTER_ELECTION_FILTER_CLIENT, true);
this.masterElectionFilterDataNodes = settings.getAsBoolean(SETTING_MASTER_ELECTION_FILTER_DATA, false);
this.masterElectionWaitForJoinsTimeout = settings.getAsTime(SETTING_MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT, TimeValue.timeValueMillis(joinTimeout.millis() / 2));
this.rejoinOnMasterGone = settings.getAsBoolean(SETTING_REJOIN_ON_MASTER_GONE, true);
this.rejoinOnMasterGone = REJOIN_ON_MASTER_GONE_SETTING.get(settings);
if (this.joinRetryAttempts < 1) {
throw new IllegalArgumentException("'" + SETTING_JOIN_RETRY_ATTEMPTS + "' must be a positive number. got [" + SETTING_JOIN_RETRY_ATTEMPTS + "]");
@ -171,7 +172,14 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
logger.debug("using ping_timeout [{}], join.timeout [{}], master_election.filter_client [{}], master_election.filter_data [{}]", this.pingTimeout, joinTimeout, masterElectionFilterClientNodes, masterElectionFilterDataNodes);
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING, this::handleMinimumMasterNodesChanged, (value) -> {
final ClusterState clusterState = clusterService.state();
int masterNodes = clusterState.nodes().masterNodes().size();
if (value > masterNodes) {
throw new IllegalArgumentException("cannot set " + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " to more than the current master nodes count [" + masterNodes + "]");
}
});
clusterSettings.addSettingsUpdateConsumer(REJOIN_ON_MASTER_GONE_SETTING, this::setRejoingOnMasterGone);
this.masterFD = new MasterFaultDetection(settings, threadPool, transportService, clusterName, clusterService);
this.masterFD.addListener(new MasterNodeFailureListener());
@ -306,6 +314,10 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
return clusterJoinsCounter.get() > 0;
}
private void setRejoingOnMasterGone(boolean rejoin) {
this.rejoinOnMasterGone = rejoin;
}
/** end of {@link org.elasticsearch.discovery.zen.ping.PingContextProvider } implementation */
@ -1139,26 +1151,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
}
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
int minimumMasterNodes = settings.getAsInt(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES,
ZenDiscovery.this.electMaster.minimumMasterNodes());
if (minimumMasterNodes != ZenDiscovery.this.electMaster.minimumMasterNodes()) {
logger.info("updating {} from [{}] to [{}]", ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES,
ZenDiscovery.this.electMaster.minimumMasterNodes(), minimumMasterNodes);
handleMinimumMasterNodesChanged(minimumMasterNodes);
}
boolean rejoinOnMasterGone = settings.getAsBoolean(SETTING_REJOIN_ON_MASTER_GONE, ZenDiscovery.this.rejoinOnMasterGone);
if (rejoinOnMasterGone != ZenDiscovery.this.rejoinOnMasterGone) {
logger.info("updating {} from [{}] to [{}]", SETTING_REJOIN_ON_MASTER_GONE, ZenDiscovery.this.rejoinOnMasterGone, rejoinOnMasterGone);
ZenDiscovery.this.rejoinOnMasterGone = rejoinOnMasterGone;
}
}
}
/**
* All control of the join thread should happen under the cluster state update task thread.
* This is important to make sure that the background joining process is always in sync with any cluster state updates

View File

@ -22,11 +22,10 @@ package org.elasticsearch.discovery.zen.elect;
import com.carrotsearch.hppc.ObjectContainer;
import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.settings.Validator;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.CollectionUtils;
@ -41,23 +40,7 @@ import java.util.List;
*/
public class ElectMasterService extends AbstractComponent {
public static final String DISCOVERY_ZEN_MINIMUM_MASTER_NODES = "discovery.zen.minimum_master_nodes";
public static final Validator DISCOVERY_ZEN_MINIMUM_MASTER_NODES_VALIDATOR = new Validator() {
@Override
public String validate(String setting, String value, ClusterState clusterState) {
int intValue;
try {
intValue = Integer.parseInt(value);
} catch (NumberFormatException ex) {
return "cannot parse value [" + value + "] as an integer";
}
int masterNodes = clusterState.nodes().masterNodes().size();
if (intValue > masterNodes) {
return "cannot set " + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES + " to more than the current master nodes count [" + masterNodes + "]";
}
return null;
}
};
public static final Setting<Integer> DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING = Setting.intSetting("discovery.zen.minimum_master_nodes", -1, true, Setting.Scope.CLUSTER);
// This is the minimum version a master needs to be on, otherwise it gets ignored
// This is based on the minimum compatible version of the current version this node is on
@ -70,7 +53,7 @@ public class ElectMasterService extends AbstractComponent {
public ElectMasterService(Settings settings, Version version) {
super(settings);
this.minMasterVersion = version.minimumCompatibilityVersion();
this.minimumMasterNodes = settings.getAsInt(DISCOVERY_ZEN_MINIMUM_MASTER_NODES, -1);
this.minimumMasterNodes = DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(settings);
logger.debug("using minimum_master_nodes [{}]", minimumMasterNodes);
}

View File

@ -227,7 +227,7 @@ public class GatewayService extends AbstractLifecycleComponent<GatewayService> i
// automatically generate a UID for the metadata if we need to
metaDataBuilder.generateClusterUuidIfNeeded();
if (recoveredState.metaData().settings().getAsBoolean(MetaData.SETTING_READ_ONLY, false) || currentState.metaData().settings().getAsBoolean(MetaData.SETTING_READ_ONLY, false)) {
if (MetaData.SETTING_READ_ONLY_SETTING.get(recoveredState.metaData().settings()) || MetaData.SETTING_READ_ONLY_SETTING.get(currentState.metaData().settings())) {
blocks.addGlobalBlock(MetaData.CLUSTER_READ_ONLY_BLOCK);
}

View File

@ -21,36 +21,36 @@ package org.elasticsearch.index.store;
import org.apache.lucene.store.StoreRateLimiting;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
* IndexStoreConfig encapsulates node / cluster level configuration for index level {@link IndexStore} instances.
* For instance it maintains the node level rate limiter configuration: updates to the cluster that disable or enable
* {@value #INDICES_STORE_THROTTLE_TYPE} or {@value #INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC} are reflected immediately
* <tt>indices.store.throttle.type</tt> or <tt>indices.store.throttle.max_bytes_per_sec</tt> are reflected immediately
* on all referencing {@link IndexStore} instances
*/
public class IndexStoreConfig implements NodeSettingsService.Listener {
public class IndexStoreConfig{
/**
* Configures the node / cluster level throttle type. See {@link StoreRateLimiting.Type}.
*/
public static final String INDICES_STORE_THROTTLE_TYPE = "indices.store.throttle.type";
public static final Setting<StoreRateLimiting.Type> INDICES_STORE_THROTTLE_TYPE_SETTING = new Setting<>("indices.store.throttle.type", StoreRateLimiting.Type.NONE.name(),StoreRateLimiting.Type::fromString, true, Setting.Scope.CLUSTER);
/**
* Configures the node / cluster level throttle intensity. The default is <tt>10240 MB</tt>
*/
public static final String INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC = "indices.store.throttle.max_bytes_per_sec";
private volatile String rateLimitingType;
public static final Setting<ByteSizeValue> INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING = Setting.byteSizeSetting("indices.store.throttle.max_bytes_per_sec", new ByteSizeValue(0), true, Setting.Scope.CLUSTER);
private volatile StoreRateLimiting.Type rateLimitingType;
private volatile ByteSizeValue rateLimitingThrottle;
private final StoreRateLimiting rateLimiting = new StoreRateLimiting();
private final ESLogger logger;
public IndexStoreConfig(Settings settings) {
logger = Loggers.getLogger(IndexStoreConfig.class, settings);
// we don't limit by default (we default to CMS's auto throttle instead):
this.rateLimitingType = settings.get("indices.store.throttle.type", StoreRateLimiting.Type.NONE.name());
this.rateLimitingType = INDICES_STORE_THROTTLE_TYPE_SETTING.get(settings);
rateLimiting.setType(rateLimitingType);
this.rateLimitingThrottle = settings.getAsBytesSize("indices.store.throttle.max_bytes_per_sec", new ByteSizeValue(0));
this.rateLimitingThrottle = INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.get(settings);
rateLimiting.setMaxRate(rateLimitingThrottle);
logger.debug("using indices.store.throttle.type [{}], with index.store.throttle.max_bytes_per_sec [{}]", rateLimitingType, rateLimitingThrottle);
}
@ -62,22 +62,12 @@ public class IndexStoreConfig implements NodeSettingsService.Listener {
return rateLimiting;
}
@Override
public void onRefreshSettings(Settings settings) {
String rateLimitingType = settings.get(INDICES_STORE_THROTTLE_TYPE, this.rateLimitingType);
// try and parse the type
StoreRateLimiting.Type.fromString(rateLimitingType);
if (!rateLimitingType.equals(this.rateLimitingType)) {
logger.info("updating indices.store.throttle.type from [{}] to [{}]", this.rateLimitingType, rateLimitingType);
public void setRateLimitingType(StoreRateLimiting.Type rateLimitingType) {
this.rateLimitingType = rateLimitingType;
this.rateLimiting.setType(rateLimitingType);
rateLimiting.setType(rateLimitingType);
}
ByteSizeValue rateLimitingThrottle = settings.getAsBytesSize(INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC, this.rateLimitingThrottle);
if (!rateLimitingThrottle.equals(this.rateLimitingThrottle)) {
logger.info("updating indices.store.throttle.max_bytes_per_sec from [{}] to [{}], note, type is [{}]", this.rateLimitingThrottle, rateLimitingThrottle, this.rateLimitingType);
public void setRateLimitingThrottle(ByteSizeValue rateLimitingThrottle) {
this.rateLimitingThrottle = rateLimitingThrottle;
this.rateLimiting.setMaxRate(rateLimitingThrottle);
}
}
}

View File

@ -36,6 +36,7 @@ import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
@ -58,7 +59,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.mapper.MapperRegistry;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.PluginsService;
import java.io.IOException;
@ -100,7 +100,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
@Inject
public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv,
NodeSettingsService nodeSettingsService, AnalysisRegistry analysisRegistry,
ClusterSettings clusterSettings, AnalysisRegistry analysisRegistry,
IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver,
ClusterService clusterService, MapperRegistry mapperRegistry) {
super(settings);
@ -113,7 +113,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
this.clusterService = clusterService;
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.mapperRegistry = mapperRegistry;
nodeSettingsService.addListener(indexStoreConfig);
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_TYPE_SETTING, indexStoreConfig::setRateLimitingType);
clusterSettings.addSettingsUpdateConsumer(IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING, indexStoreConfig::setRateLimitingThrottle);
}
@Override

View File

@ -25,9 +25,10 @@ import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.breaker.NoopCircuitBreaker;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.util.ArrayList;
import java.util.List;
@ -45,25 +46,17 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
private final ConcurrentMap<String, CircuitBreaker> breakers = new ConcurrentHashMap();
// Old pre-1.4.0 backwards compatible settings
public static final String OLD_CIRCUIT_BREAKER_MAX_BYTES_SETTING = "indices.fielddata.breaker.limit";
public static final String OLD_CIRCUIT_BREAKER_OVERHEAD_SETTING = "indices.fielddata.breaker.overhead";
public static final Setting<ByteSizeValue> TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING = Setting.byteSizeSetting("indices.breaker.total.limit", "70%", true, Setting.Scope.CLUSTER);
public static final String TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING = "indices.breaker.total.limit";
public static final String DEFAULT_TOTAL_CIRCUIT_BREAKER_LIMIT = "70%";
public static final Setting<ByteSizeValue> FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING = Setting.byteSizeSetting("indices.breaker.fielddata.limit", "60%", true, Setting.Scope.CLUSTER);
public static final Setting<Double> FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING = Setting.doubleSetting("indices.breaker.fielddata.overhead", 1.03d, 0.0d, true, Setting.Scope.CLUSTER);
public static final Setting<CircuitBreaker.Type> FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING = new Setting<>("indices.breaker.fielddata.type", "memory", CircuitBreaker.Type::parseValue, false, Setting.Scope.CLUSTER);
public static final String FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING = "indices.breaker.fielddata.limit";
public static final String FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING = "indices.breaker.fielddata.overhead";
public static final String FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING = "indices.breaker.fielddata.type";
public static final String DEFAULT_FIELDDATA_BREAKER_LIMIT = "60%";
public static final double DEFAULT_FIELDDATA_OVERHEAD_CONSTANT = 1.03;
public static final Setting<ByteSizeValue> REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING = Setting.byteSizeSetting("indices.breaker.request.limit", "40%", true, Setting.Scope.CLUSTER);
public static final Setting<Double> REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING = Setting.doubleSetting("indices.breaker.request.overhead", 1.0d, 0.0d, true, Setting.Scope.CLUSTER);
public static final Setting<CircuitBreaker.Type> REQUEST_CIRCUIT_BREAKER_TYPE_SETTING = new Setting<>("indices.breaker.request.type", "memory", CircuitBreaker.Type::parseValue, false, Setting.Scope.CLUSTER);
public static final String REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING = "indices.breaker.request.limit";
public static final String REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING = "indices.breaker.request.overhead";
public static final String REQUEST_CIRCUIT_BREAKER_TYPE_SETTING = "indices.breaker.request.type";
public static final String DEFAULT_REQUEST_BREAKER_LIMIT = "40%";
public static final String DEFAULT_BREAKER_TYPE = "memory";
private volatile BreakerSettings parentSettings;
private volatile BreakerSettings fielddataSettings;
@ -73,41 +66,21 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
private final AtomicLong parentTripCount = new AtomicLong(0);
@Inject
public HierarchyCircuitBreakerService(Settings settings, NodeSettingsService nodeSettingsService) {
public HierarchyCircuitBreakerService(Settings settings, ClusterSettings clusterSettings) {
super(settings);
// This uses the old InternalCircuitBreakerService.CIRCUIT_BREAKER_MAX_BYTES_SETTING
// setting to keep backwards compatibility with 1.3, it can be safely
// removed when compatibility with 1.3 is no longer needed
String compatibilityFielddataLimitDefault = DEFAULT_FIELDDATA_BREAKER_LIMIT;
ByteSizeValue compatibilityFielddataLimit = settings.getAsMemory(OLD_CIRCUIT_BREAKER_MAX_BYTES_SETTING, null);
if (compatibilityFielddataLimit != null) {
compatibilityFielddataLimitDefault = compatibilityFielddataLimit.toString();
}
// This uses the old InternalCircuitBreakerService.CIRCUIT_BREAKER_OVERHEAD_SETTING
// setting to keep backwards compatibility with 1.3, it can be safely
// removed when compatibility with 1.3 is no longer needed
double compatibilityFielddataOverheadDefault = DEFAULT_FIELDDATA_OVERHEAD_CONSTANT;
Double compatibilityFielddataOverhead = settings.getAsDouble(OLD_CIRCUIT_BREAKER_OVERHEAD_SETTING, null);
if (compatibilityFielddataOverhead != null) {
compatibilityFielddataOverheadDefault = compatibilityFielddataOverhead;
}
this.fielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA,
settings.getAsMemory(FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, compatibilityFielddataLimitDefault).bytes(),
settings.getAsDouble(FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, compatibilityFielddataOverheadDefault),
CircuitBreaker.Type.parseValue(settings.get(FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING, DEFAULT_BREAKER_TYPE))
FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(),
FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
FIELDDATA_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
);
this.requestSettings = new BreakerSettings(CircuitBreaker.REQUEST,
settings.getAsMemory(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, DEFAULT_REQUEST_BREAKER_LIMIT).bytes(),
settings.getAsDouble(REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, 1.0),
CircuitBreaker.Type.parseValue(settings.get(REQUEST_CIRCUIT_BREAKER_TYPE_SETTING, DEFAULT_BREAKER_TYPE))
REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(),
REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING.get(settings),
REQUEST_CIRCUIT_BREAKER_TYPE_SETTING.get(settings)
);
this.parentSettings = new BreakerSettings(CircuitBreaker.PARENT,
settings.getAsMemory(TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, DEFAULT_TOTAL_CIRCUIT_BREAKER_LIMIT).bytes(), 1.0, CircuitBreaker.Type.PARENT);
this.parentSettings = new BreakerSettings(CircuitBreaker.PARENT, TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING.get(settings).bytes(), 1.0, CircuitBreaker.Type.PARENT);
if (logger.isTraceEnabled()) {
logger.trace("parent circuit breaker with settings {}", this.parentSettings);
}
@ -115,52 +88,38 @@ public class HierarchyCircuitBreakerService extends CircuitBreakerService {
registerBreaker(this.requestSettings);
registerBreaker(this.fielddataSettings);
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, this::setTotalCircuitBreakerLimit, this::validateTotalCircuitBreakerLimit);
clusterSettings.addSettingsUpdateConsumer(FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, this::setFieldDataBreakerLimit);
clusterSettings.addSettingsUpdateConsumer(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, this::setRequestBreakerLimit);
}
public class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
// Fielddata settings
ByteSizeValue newFielddataMax = settings.getAsMemory(FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, null);
Double newFielddataOverhead = settings.getAsDouble(FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, null);
if (newFielddataMax != null || newFielddataOverhead != null) {
long newFielddataLimitBytes = newFielddataMax == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getLimit() : newFielddataMax.bytes();
newFielddataOverhead = newFielddataOverhead == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getOverhead() : newFielddataOverhead;
BreakerSettings newFielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA, newFielddataLimitBytes, newFielddataOverhead,
HierarchyCircuitBreakerService.this.fielddataSettings.getType());
registerBreaker(newFielddataSettings);
HierarchyCircuitBreakerService.this.fielddataSettings = newFielddataSettings;
logger.info("Updated breaker settings fielddata: {}", newFielddataSettings);
}
// Request settings
ByteSizeValue newRequestMax = settings.getAsMemory(REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, null);
Double newRequestOverhead = settings.getAsDouble(REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, null);
if (newRequestMax != null || newRequestOverhead != null) {
long newRequestLimitBytes = newRequestMax == null ? HierarchyCircuitBreakerService.this.requestSettings.getLimit() : newRequestMax.bytes();
newRequestOverhead = newRequestOverhead == null ? HierarchyCircuitBreakerService.this.requestSettings.getOverhead() : newRequestOverhead;
BreakerSettings newRequestSettings = new BreakerSettings(CircuitBreaker.REQUEST, newRequestLimitBytes, newRequestOverhead,
private void setRequestBreakerLimit(ByteSizeValue newRequestMax, Double newRequestOverhead) {
BreakerSettings newRequestSettings = new BreakerSettings(CircuitBreaker.REQUEST, newRequestMax.bytes(), newRequestOverhead,
HierarchyCircuitBreakerService.this.requestSettings.getType());
registerBreaker(newRequestSettings);
HierarchyCircuitBreakerService.this.requestSettings = newRequestSettings;
logger.info("Updated breaker settings request: {}", newRequestSettings);
}
// Parent settings
long oldParentMax = HierarchyCircuitBreakerService.this.parentSettings.getLimit();
ByteSizeValue newParentMax = settings.getAsMemory(TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, null);
if (newParentMax != null && (newParentMax.bytes() != oldParentMax)) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, newParentMax.bytes(), 1.0, CircuitBreaker.Type.PARENT);
private void setFieldDataBreakerLimit(ByteSizeValue newFielddataMax, Double newFielddataOverhead) {
long newFielddataLimitBytes = newFielddataMax == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getLimit() : newFielddataMax.bytes();
newFielddataOverhead = newFielddataOverhead == null ? HierarchyCircuitBreakerService.this.fielddataSettings.getOverhead() : newFielddataOverhead;
BreakerSettings newFielddataSettings = new BreakerSettings(CircuitBreaker.FIELDDATA, newFielddataLimitBytes, newFielddataOverhead,
HierarchyCircuitBreakerService.this.fielddataSettings.getType());
registerBreaker(newFielddataSettings);
HierarchyCircuitBreakerService.this.fielddataSettings = newFielddataSettings;
logger.info("Updated breaker settings field data: {}", newFielddataSettings);
}
private boolean validateTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.bytes(), 1.0, CircuitBreaker.Type.PARENT);
validateSettings(new BreakerSettings[]{newParentSettings});
HierarchyCircuitBreakerService.this.parentSettings = newParentSettings;
logger.info("Updated breaker settings parent: {}", newParentSettings);
}
return true;
}
private void setTotalCircuitBreakerLimit(ByteSizeValue byteSizeValue) {
BreakerSettings newParentSettings = new BreakerSettings(CircuitBreaker.PARENT, byteSizeValue.bytes(), 1.0, CircuitBreaker.Type.PARENT);
this.parentSettings = newParentSettings;
}
/**

View File

@ -23,16 +23,16 @@ import org.apache.lucene.store.RateLimiter;
import org.apache.lucene.store.RateLimiter.SimpleRateLimiter;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.util.Objects;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
@ -40,34 +40,33 @@ import java.util.concurrent.TimeUnit;
*/
public class RecoverySettings extends AbstractComponent implements Closeable {
public static final String INDICES_RECOVERY_CONCURRENT_STREAMS = "indices.recovery.concurrent_streams";
public static final String INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS = "indices.recovery.concurrent_small_file_streams";
public static final String INDICES_RECOVERY_MAX_BYTES_PER_SEC = "indices.recovery.max_bytes_per_sec";
public static final Setting<Integer> INDICES_RECOVERY_CONCURRENT_STREAMS_SETTING = Setting.intSetting("indices.recovery.concurrent_streams", 3, true, Setting.Scope.CLUSTER);
public static final Setting<Integer> INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS_SETTING = Setting.intSetting("indices.recovery.concurrent_small_file_streams", 2, true, Setting.Scope.CLUSTER);
public static final Setting<ByteSizeValue> INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING = Setting.byteSizeSetting("indices.recovery.max_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB), true, Setting.Scope.CLUSTER);
/**
* how long to wait before retrying after issues cause by cluster state syncing between nodes
* i.e., local node is not yet known on remote node, remote shard not yet started etc.
*/
public static final String INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC = "indices.recovery.retry_delay_state_sync";
public static final Setting<TimeValue> INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING = Setting.positiveTimeSetting("indices.recovery.retry_delay_state_sync", TimeValue.timeValueMillis(500), true, Setting.Scope.CLUSTER);
/** how long to wait before retrying after network related issues */
public static final String INDICES_RECOVERY_RETRY_DELAY_NETWORK = "indices.recovery.retry_delay_network";
/**
* recoveries that don't show any activity for more then this interval will be failed.
* defaults to `indices.recovery.internal_action_long_timeout`
*/
public static final String INDICES_RECOVERY_ACTIVITY_TIMEOUT = "indices.recovery.recovery_activity_timeout";
public static final Setting<TimeValue> INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING = Setting.positiveTimeSetting("indices.recovery.retry_delay_network", TimeValue.timeValueSeconds(5), true, Setting.Scope.CLUSTER);
/** timeout value to use for requests made as part of the recovery process */
public static final String INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT = "indices.recovery.internal_action_timeout";
public static final Setting<TimeValue> INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING = Setting.positiveTimeSetting("indices.recovery.internal_action_timeout", TimeValue.timeValueMinutes(15), true, Setting.Scope.CLUSTER);
/**
* timeout value to use for requests made as part of the recovery process that are expected to take long time.
* defaults to twice `indices.recovery.internal_action_timeout`.
*/
public static final String INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT = "indices.recovery.internal_action_long_timeout";
public static final Setting<TimeValue> INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING = Setting.timeSetting("indices.recovery.internal_action_long_timeout", (s) -> TimeValue.timeValueMillis(INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING.get(s).millis() * 2).toString(), TimeValue.timeValueSeconds(0), true, Setting.Scope.CLUSTER);
/**
* recoveries that don't show any activity for more then this interval will be failed.
* defaults to `indices.recovery.internal_action_long_timeout`
*/
public static final Setting<TimeValue> INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING = Setting.timeSetting("indices.recovery.recovery_activity_timeout", (s) -> INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING.getRaw(s) , TimeValue.timeValueSeconds(0), true, Setting.Scope.CLUSTER);
public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb", "SMALL_FILE_CUTOFF_BYTES").bytes();
@ -89,31 +88,28 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE;
@Inject
public RecoverySettings(Settings settings, NodeSettingsService nodeSettingsService) {
public RecoverySettings(Settings settings, ClusterSettings clusterSettings) {
super(settings);
this.retryDelayStateSync = settings.getAsTime(INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, TimeValue.timeValueMillis(500));
this.retryDelayStateSync = INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.get(settings);
// doesn't have to be fast as nodes are reconnected every 10s by default (see InternalClusterService.ReconnectToNodes)
// and we want to give the master time to remove a faulty node
this.retryDelayNetwork = settings.getAsTime(INDICES_RECOVERY_RETRY_DELAY_NETWORK, TimeValue.timeValueSeconds(5));
this.retryDelayNetwork = INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING.get(settings);
this.internalActionTimeout = settings.getAsTime(INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, TimeValue.timeValueMinutes(15));
this.internalActionLongTimeout = settings.getAsTime(INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT, new TimeValue(internalActionTimeout.millis() * 2));
this.internalActionTimeout = INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING.get(settings);
this.internalActionLongTimeout = INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING.get(settings);
this.activityTimeout = settings.getAsTime(INDICES_RECOVERY_ACTIVITY_TIMEOUT,
// default to the internalActionLongTimeout used as timeouts on RecoverySource
internalActionLongTimeout
);
this.activityTimeout = INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING.get(settings);
this.concurrentStreams = settings.getAsInt(INDICES_RECOVERY_CONCURRENT_STREAMS, 3);
this.concurrentStreams = INDICES_RECOVERY_CONCURRENT_STREAMS_SETTING.get(settings);
this.concurrentStreamPool = EsExecutors.newScaling("recovery_stream", 0, concurrentStreams, 60, TimeUnit.SECONDS,
EsExecutors.daemonThreadFactory(settings, "[recovery_stream]"));
this.concurrentSmallFileStreams = settings.getAsInt(INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, 2);
this.concurrentSmallFileStreams = INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS_SETTING.get(settings);
this.concurrentSmallFileStreamPool = EsExecutors.newScaling("small_file_recovery_stream", 0, concurrentSmallFileStreams, 60,
TimeUnit.SECONDS, EsExecutors.daemonThreadFactory(settings, "[small_file_recovery_stream]"));
this.maxBytesPerSec = settings.getAsBytesSize(INDICES_RECOVERY_MAX_BYTES_PER_SEC, new ByteSizeValue(40, ByteSizeUnit.MB));
this.maxBytesPerSec = INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.get(settings);
if (maxBytesPerSec.bytes() <= 0) {
rateLimiter = null;
} else {
@ -123,7 +119,14 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
logger.debug("using max_bytes_per_sec[{}], concurrent_streams [{}]",
maxBytesPerSec, concurrentStreams);
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_CONCURRENT_STREAMS_SETTING, this::setConcurrentStreams);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS_SETTING, this::setConcurrentSmallFileStreams);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, this::setMaxBytesPerSec);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, this::setRetryDelayStateSync);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, this::setRetryDelayNetwork);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING, this::setInternalActionTimeout);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING, this::setInternalActionLongTimeout);
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, this::setActivityTimeout);
}
@Override
@ -173,51 +176,44 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
this.chunkSize = chunkSize;
}
private void setConcurrentStreams(int concurrentStreams) {
this.concurrentStreams = concurrentStreams;
concurrentStreamPool.setMaximumPoolSize(concurrentStreams);
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
ByteSizeValue maxSizePerSec = settings.getAsBytesSize(INDICES_RECOVERY_MAX_BYTES_PER_SEC, RecoverySettings.this.maxBytesPerSec);
if (!Objects.equals(maxSizePerSec, RecoverySettings.this.maxBytesPerSec)) {
logger.info("updating [{}] from [{}] to [{}]", INDICES_RECOVERY_MAX_BYTES_PER_SEC, RecoverySettings.this.maxBytesPerSec, maxSizePerSec);
RecoverySettings.this.maxBytesPerSec = maxSizePerSec;
if (maxSizePerSec.bytes() <= 0) {
public void setRetryDelayStateSync(TimeValue retryDelayStateSync) {
this.retryDelayStateSync = retryDelayStateSync;
}
public void setRetryDelayNetwork(TimeValue retryDelayNetwork) {
this.retryDelayNetwork = retryDelayNetwork;
}
public void setActivityTimeout(TimeValue activityTimeout) {
this.activityTimeout = activityTimeout;
}
public void setInternalActionTimeout(TimeValue internalActionTimeout) {
this.internalActionTimeout = internalActionTimeout;
}
public void setInternalActionLongTimeout(TimeValue internalActionLongTimeout) {
this.internalActionLongTimeout = internalActionLongTimeout;
}
private void setMaxBytesPerSec(ByteSizeValue maxBytesPerSec) {
this.maxBytesPerSec = maxBytesPerSec;
if (maxBytesPerSec.bytes() <= 0) {
rateLimiter = null;
} else if (rateLimiter != null) {
rateLimiter.setMBPerSec(maxSizePerSec.mbFrac());
rateLimiter.setMBPerSec(maxBytesPerSec.mbFrac());
} else {
rateLimiter = new SimpleRateLimiter(maxSizePerSec.mbFrac());
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.mbFrac());
}
}
int concurrentStreams = settings.getAsInt(INDICES_RECOVERY_CONCURRENT_STREAMS, RecoverySettings.this.concurrentStreams);
if (concurrentStreams != RecoverySettings.this.concurrentStreams) {
logger.info("updating [indices.recovery.concurrent_streams] from [{}] to [{}]", RecoverySettings.this.concurrentStreams, concurrentStreams);
RecoverySettings.this.concurrentStreams = concurrentStreams;
RecoverySettings.this.concurrentStreamPool.setMaximumPoolSize(concurrentStreams);
}
int concurrentSmallFileStreams = settings.getAsInt(INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, RecoverySettings.this.concurrentSmallFileStreams);
if (concurrentSmallFileStreams != RecoverySettings.this.concurrentSmallFileStreams) {
logger.info("updating [indices.recovery.concurrent_small_file_streams] from [{}] to [{}]", RecoverySettings.this.concurrentSmallFileStreams, concurrentSmallFileStreams);
RecoverySettings.this.concurrentSmallFileStreams = concurrentSmallFileStreams;
RecoverySettings.this.concurrentSmallFileStreamPool.setMaximumPoolSize(concurrentSmallFileStreams);
}
RecoverySettings.this.retryDelayNetwork = maybeUpdate(RecoverySettings.this.retryDelayNetwork, settings, INDICES_RECOVERY_RETRY_DELAY_NETWORK);
RecoverySettings.this.retryDelayStateSync = maybeUpdate(RecoverySettings.this.retryDelayStateSync, settings, INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC);
RecoverySettings.this.activityTimeout = maybeUpdate(RecoverySettings.this.activityTimeout, settings, INDICES_RECOVERY_ACTIVITY_TIMEOUT);
RecoverySettings.this.internalActionTimeout = maybeUpdate(RecoverySettings.this.internalActionTimeout, settings, INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT);
RecoverySettings.this.internalActionLongTimeout = maybeUpdate(RecoverySettings.this.internalActionLongTimeout, settings, INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT);
}
private TimeValue maybeUpdate(final TimeValue currentValue, final Settings settings, final String key) {
final TimeValue value = settings.getAsTime(key, currentValue);
if (value.equals(currentValue)) {
return currentValue;
}
logger.info("updating [] from [{}] to [{}]", key, currentValue, value);
return value;
}
private void setConcurrentSmallFileStreams(int concurrentSmallFileStreams) {
this.concurrentSmallFileStreams = concurrentSmallFileStreams;
concurrentSmallFileStreamPool.setMaximumPoolSize(concurrentSmallFileStreams);
}
}

View File

@ -36,6 +36,8 @@ import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
@ -49,7 +51,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.io.IOException;
import java.util.ArrayList;
@ -66,7 +67,7 @@ import java.util.concurrent.locks.ReentrantLock;
*/
public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLService> {
public static final String INDICES_TTL_INTERVAL = "indices.ttl.interval";
public static final Setting<TimeValue> INDICES_TTL_INTERVAL_SETTING = Setting.positiveTimeSetting("indices.ttl.interval", TimeValue.timeValueSeconds(60), true, Setting.Scope.CLUSTER);
public static final String INDEX_TTL_DISABLE_PURGE = "index.ttl.disable_purge";
private final ClusterService clusterService;
@ -77,16 +78,15 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
private PurgerThread purgerThread;
@Inject
public IndicesTTLService(Settings settings, ClusterService clusterService, IndicesService indicesService, NodeSettingsService nodeSettingsService, TransportBulkAction bulkAction) {
public IndicesTTLService(Settings settings, ClusterService clusterService, IndicesService indicesService, ClusterSettings clusterSettings, TransportBulkAction bulkAction) {
super(settings);
this.clusterService = clusterService;
this.indicesService = indicesService;
TimeValue interval = this.settings.getAsTime("indices.ttl.interval", TimeValue.timeValueSeconds(60));
TimeValue interval = INDICES_TTL_INTERVAL_SETTING.get(settings);
this.bulkAction = bulkAction;
this.bulkSize = this.settings.getAsInt("indices.ttl.bulk_size", 10000);
this.purgerThread = new PurgerThread(EsExecutors.threadName(settings, "[ttl_expire]"), interval);
nodeSettingsService.addListener(new ApplySettings());
clusterSettings.addSettingsUpdateConsumer(INDICES_TTL_INTERVAL_SETTING, this.purgerThread::resetInterval);
}
@Override
@ -310,20 +310,6 @@ public class IndicesTTLService extends AbstractLifecycleComponent<IndicesTTLServ
return bulkRequest;
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
final TimeValue currentInterval = IndicesTTLService.this.purgerThread.getInterval();
final TimeValue interval = settings.getAsTime(INDICES_TTL_INTERVAL, currentInterval);
if (!interval.equals(currentInterval)) {
logger.info("updating indices.ttl.interval from [{}] to [{}]",currentInterval, interval);
IndicesTTLService.this.purgerThread.resetInterval(interval);
}
}
}
private static final class Notifier {
private final ReentrantLock lock = new ReentrantLock();

View File

@ -44,6 +44,7 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.network.NetworkAddress;
import org.elasticsearch.common.network.NetworkModule;
import org.elasticsearch.common.network.NetworkService;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.common.settings.SettingsModule;
@ -75,7 +76,6 @@ import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.monitor.jvm.JvmInfo;
import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.percolator.PercolatorModule;
import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.plugins.Plugin;
@ -164,7 +164,6 @@ public class Node implements Releasable {
throw new IllegalStateException("Failed to created node environment", ex);
}
final NetworkService networkService = new NetworkService(settings);
final NodeSettingsService nodeSettingsService = new NodeSettingsService(settings);
final SettingsFilter settingsFilter = new SettingsFilter(settings);
final ThreadPool threadPool = new ThreadPool(settings);
boolean success = false;
@ -180,7 +179,7 @@ public class Node implements Releasable {
modules.add(new PluginsModule(pluginsService));
modules.add(new SettingsModule(this.settings, settingsFilter));
modules.add(new EnvironmentModule(environment));
modules.add(new NodeModule(this, nodeSettingsService, monitorService));
modules.add(new NodeModule(this, monitorService));
modules.add(new NetworkModule(networkService, settings, false));
modules.add(new ScriptModule(this.settings));
modules.add(new NodeEnvironmentModule(nodeEnvironment));
@ -204,7 +203,7 @@ public class Node implements Releasable {
injector = modules.createInjector();
client = injector.getInstance(Client.class);
threadPool.setNodeSettingsService(injector.getInstance(NodeSettingsService.class));
threadPool.setClusterSettings(injector.getInstance(ClusterSettings.class));
success = true;
} catch (IOException ex) {
throw new ElasticsearchException("failed to bind service", ex);

View File

@ -23,9 +23,7 @@ import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.node.Node;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.node.settings.NodeSettingsService;
/**
*
@ -33,16 +31,14 @@ import org.elasticsearch.node.settings.NodeSettingsService;
public class NodeModule extends AbstractModule {
private final Node node;
private final NodeSettingsService nodeSettingsService;
private final MonitorService monitorService;
// pkg private so tests can mock
Class<? extends PageCacheRecycler> pageCacheRecyclerImpl = PageCacheRecycler.class;
Class<? extends BigArrays> bigArraysImpl = BigArrays.class;
public NodeModule(Node node, NodeSettingsService nodeSettingsService, MonitorService monitorService) {
public NodeModule(Node node, MonitorService monitorService) {
this.node = node;
this.nodeSettingsService = nodeSettingsService;
this.monitorService = monitorService;
}
@ -60,7 +56,6 @@ public class NodeModule extends AbstractModule {
}
bind(Node.class).toInstance(node);
bind(NodeSettingsService.class).toInstance(nodeSettingsService);
bind(MonitorService.class).toInstance(monitorService);
bind(NodeService.class).asEagerSingleton();
}

View File

@ -1,122 +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.node.settings;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterStateListener;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.settings.Settings;
import java.util.Map;
import java.util.concurrent.CopyOnWriteArrayList;
/**
* A service that allows to register for node settings change that can come from cluster
* events holding new settings.
*/
public class NodeSettingsService extends AbstractComponent implements ClusterStateListener {
private static volatile Settings globalSettings = Settings.Builder.EMPTY_SETTINGS;
/**
* Returns the global (static) settings last updated by a node. Note, if you have multiple
* nodes on the same JVM, it will just return the latest one set...
*/
public static Settings getGlobalSettings() {
return globalSettings;
}
private volatile Settings lastSettingsApplied;
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
@Inject
public NodeSettingsService(Settings settings) {
super(settings);
globalSettings = settings;
}
// inject it as a member, so we won't get into possible cyclic problems
public void setClusterService(ClusterService clusterService) {
clusterService.add(this);
}
@Override
public void clusterChanged(ClusterChangedEvent event) {
// nothing to do until we actually recover from the gateway or any other block indicates we need to disable persistency
if (event.state().blocks().disableStatePersistence()) {
return;
}
if (!event.metaDataChanged()) {
// nothing changed in the metadata, no need to check
return;
}
if (lastSettingsApplied != null && event.state().metaData().settings().equals(lastSettingsApplied)) {
// nothing changed in the settings, ignore
return;
}
for (Listener listener : listeners) {
try {
listener.onRefreshSettings(event.state().metaData().settings());
} catch (Exception e) {
logger.warn("failed to refresh settings for [{}]", e, listener);
}
}
try {
for (Map.Entry<String, String> entry : event.state().metaData().settings().getAsMap().entrySet()) {
if (entry.getKey().startsWith("logger.")) {
String component = entry.getKey().substring("logger.".length());
if ("_root".equals(component)) {
ESLoggerFactory.getRootLogger().setLevel(entry.getValue());
} else {
ESLoggerFactory.getLogger(component).setLevel(entry.getValue());
}
}
}
} catch (Exception e) {
logger.warn("failed to refresh settings for [{}]", e, "logger");
}
lastSettingsApplied = event.state().metaData().settings();
globalSettings = lastSettingsApplied;
}
/**
* Only settings registered in {@link org.elasticsearch.cluster.ClusterModule} can be changed dynamically.
*/
public void addListener(Listener listener) {
this.listeners.add(listener);
}
public void removeListener(Listener listener) {
this.listeners.remove(listener);
}
public interface Listener {
void onRefreshSettings(Settings settings);
}
}

View File

@ -23,19 +23,27 @@ import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.rest.*;
import org.elasticsearch.rest.action.support.RestBuilderListener;
import java.io.IOException;
/**
*/
public class RestClusterGetSettingsAction extends BaseRestHandler {
private final ClusterSettings clusterSettings;
@Inject
public RestClusterGetSettingsAction(Settings settings, RestController controller, Client client) {
public RestClusterGetSettingsAction(Settings settings, RestController controller, Client client, ClusterSettings clusterSettings) {
super(settings, controller, client);
this.clusterSettings = clusterSettings;
controller.registerHandler(RestRequest.Method.GET, "/_cluster/settings", this);
}
@ -44,24 +52,34 @@ public class RestClusterGetSettingsAction extends BaseRestHandler {
ClusterStateRequest clusterStateRequest = Requests.clusterStateRequest()
.routingTable(false)
.nodes(false);
final boolean renderDefaults = request.paramAsBoolean("defaults", false);
clusterStateRequest.local(request.paramAsBoolean("local", clusterStateRequest.local()));
client.admin().cluster().state(clusterStateRequest, new RestBuilderListener<ClusterStateResponse>(channel) {
@Override
public RestResponse buildResponse(ClusterStateResponse response, XContentBuilder builder) throws Exception {
builder.startObject();
builder.startObject("persistent");
response.getState().metaData().persistentSettings().toXContent(builder, request);
builder.endObject();
builder.startObject("transient");
response.getState().metaData().transientSettings().toXContent(builder, request);
builder.endObject();
builder.endObject();
return new BytesRestResponse(RestStatus.OK, builder);
return new BytesRestResponse(RestStatus.OK, renderResponse(response.getState(), renderDefaults, builder, request));
}
});
}
private XContentBuilder renderResponse(ClusterState state, boolean renderDefaults, XContentBuilder builder, ToXContent.Params params) throws IOException {
builder.startObject();
builder.startObject("persistent");
state.metaData().persistentSettings().toXContent(builder, params);
builder.endObject();
builder.startObject("transient");
state.metaData().transientSettings().toXContent(builder, params);
builder.endObject();
if (renderDefaults) {
builder.startObject("defaults");
clusterSettings.diff(state.metaData().settings(), this.settings).toXContent(builder, params);
builder.endObject();
}
builder.endObject();
return builder;
}
}

View File

@ -88,6 +88,6 @@ public class RestUpdateSettingsAction extends BaseRestHandler {
}
updateSettingsRequest.settings(updateSettings);
client.admin().indices().updateSettings(updateSettingsRequest, new AcknowledgedRestListener<UpdateSettingsResponse>(channel));
client.admin().indices().updateSettings(updateSettingsRequest, new AcknowledgedRestListener<>(channel));
}
}

View File

@ -39,6 +39,8 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
@ -71,7 +73,6 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.IndicesWarmer.TerminationHandle;
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.script.ExecutableScript;
import org.elasticsearch.script.ScriptContext;
import org.elasticsearch.script.ScriptService;
@ -114,9 +115,10 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
public static final String NORMS_LOADING_KEY = "index.norms.loading";
public static final String DEFAULT_KEEPALIVE_KEY = "search.default_keep_alive";
public static final String KEEPALIVE_INTERVAL_KEY = "search.keep_alive_interval";
public static final String DEFAULT_SEARCH_TIMEOUT = "search.default_search_timeout";
public static final TimeValue NO_TIMEOUT = timeValueMillis(-1);
public static final Setting<TimeValue> DEFAULT_SEARCH_TIMEOUT_SETTING = Setting.timeSetting("search.default_search_timeout", NO_TIMEOUT, true, Setting.Scope.CLUSTER);
private final ThreadPool threadPool;
@ -155,7 +157,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
private final ParseFieldMatcher parseFieldMatcher;
@Inject
public SearchService(Settings settings, NodeSettingsService nodeSettingsService, ClusterService clusterService, IndicesService indicesService,IndicesWarmer indicesWarmer, ThreadPool threadPool,
public SearchService(Settings settings, ClusterSettings clusterSettings, ClusterService clusterService, IndicesService indicesService, IndicesWarmer indicesWarmer, ThreadPool threadPool,
ScriptService scriptService, PageCacheRecycler pageCacheRecycler, BigArrays bigArrays, DfsPhase dfsPhase, QueryPhase queryPhase, FetchPhase fetchPhase,
IndicesRequestCache indicesQueryCache) {
super(settings);
@ -189,19 +191,12 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
this.indicesWarmer.addListener(new FieldDataWarmer(indicesWarmer));
this.indicesWarmer.addListener(new SearchWarmer());
defaultSearchTimeout = settings.getAsTime(DEFAULT_SEARCH_TIMEOUT, NO_TIMEOUT);
nodeSettingsService.addListener(new SearchSettingsListener());
defaultSearchTimeout = DEFAULT_SEARCH_TIMEOUT_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(DEFAULT_SEARCH_TIMEOUT_SETTING, this::setDefaultSearchTimeout);
}
class SearchSettingsListener implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
final TimeValue maybeNewDefaultSearchTimeout = settings.getAsTime(SearchService.DEFAULT_SEARCH_TIMEOUT, SearchService.this.defaultSearchTimeout);
if (!maybeNewDefaultSearchTimeout.equals(SearchService.this.defaultSearchTimeout)) {
logger.info("updating [{}] from [{}] to [{}]", SearchService.DEFAULT_SEARCH_TIMEOUT, SearchService.this.defaultSearchTimeout, maybeNewDefaultSearchTimeout);
SearchService.this.defaultSearchTimeout = maybeNewDefaultSearchTimeout;
}
}
private void setDefaultSearchTimeout(TimeValue defaultSearchTimeout) {
this.defaultSearchTimeout = defaultSearchTimeout;
}
@Override

View File

@ -33,8 +33,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
import org.elasticsearch.cluster.settings.DynamicSettings;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -118,18 +117,19 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
private final MetaDataCreateIndexService createIndexService;
private final DynamicSettings dynamicSettings;
private final ClusterSettings dynamicSettings;
private final MetaDataIndexUpgradeService metaDataIndexUpgradeService;
private final CopyOnWriteArrayList<ActionListener<RestoreCompletionResponse>> listeners = new CopyOnWriteArrayList<>();
private final BlockingQueue<UpdateIndexShardRestoreStatusRequest> updatedSnapshotStateQueue = ConcurrentCollections.newBlockingQueue();
private final ClusterSettings clusterSettings;
@Inject
public RestoreService(Settings settings, ClusterService clusterService, RepositoriesService repositoriesService, TransportService transportService,
AllocationService allocationService, MetaDataCreateIndexService createIndexService, @ClusterDynamicSettings DynamicSettings dynamicSettings,
MetaDataIndexUpgradeService metaDataIndexUpgradeService) {
AllocationService allocationService, MetaDataCreateIndexService createIndexService, ClusterSettings dynamicSettings,
MetaDataIndexUpgradeService metaDataIndexUpgradeService, ClusterSettings clusterSettings) {
super(settings);
this.clusterService = clusterService;
this.repositoriesService = repositoriesService;
@ -140,6 +140,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
this.metaDataIndexUpgradeService = metaDataIndexUpgradeService;
transportService.registerRequestHandler(UPDATE_RESTORE_ACTION_NAME, UpdateIndexShardRestoreStatusRequest::new, ThreadPool.Names.SAME, new UpdateRestoreStateRequestHandler());
clusterService.add(this);
this.clusterSettings = clusterSettings;
}
/**
@ -389,24 +390,9 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
private void restoreGlobalStateIfRequested(MetaData.Builder mdBuilder) {
if (request.includeGlobalState()) {
if (metaData.persistentSettings() != null) {
boolean changed = false;
Settings.Builder persistentSettings = Settings.settingsBuilder().put();
for (Map.Entry<String, String> entry : metaData.persistentSettings().getAsMap().entrySet()) {
if (dynamicSettings.isDynamicOrLoggingSetting(entry.getKey())) {
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue(), clusterService.state());
if (error == null) {
persistentSettings.put(entry.getKey(), entry.getValue());
changed = true;
} else {
logger.warn("ignoring persistent setting [{}], [{}]", entry.getKey(), error);
}
} else {
logger.warn("ignoring persistent setting [{}], not dynamically updateable", entry.getKey());
}
}
if (changed) {
mdBuilder.persistentSettings(persistentSettings.build());
}
Settings settings = metaData.persistentSettings();
clusterSettings.dryRun(settings);
mdBuilder.persistentSettings(settings);
}
if (metaData.templates() != null) {
// TODO: Should all existing templates be deleted first?

View File

@ -20,13 +20,13 @@
package org.elasticsearch.threadpool;
import org.apache.lucene.util.Counter;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.settings.Validator;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsException;
import org.elasticsearch.common.unit.SizeValue;
@ -38,14 +38,11 @@ import org.elasticsearch.common.util.concurrent.XRejectedExecutionHandler;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.node.settings.NodeSettingsService;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.*;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.concurrent.atomic.AtomicBoolean;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -172,7 +169,7 @@ public class ThreadPool extends AbstractComponent {
}
}
public static final String THREADPOOL_GROUP = "threadpool.";
public static final Setting<Settings> THREADPOOL_GROUP_SETTING = Setting.groupSetting("threadpool.", true, Setting.Scope.CLUSTER);
private volatile Map<String, ExecutorHolder> executors;
@ -184,7 +181,7 @@ public class ThreadPool extends AbstractComponent {
private final EstimatedTimeThread estimatedTimeThread;
private boolean settingsListenerIsSet = false;
private final AtomicBoolean settingsListenerIsSet = new AtomicBoolean(false);
static final Executor DIRECT_EXECUTOR = command -> command.run();
@ -197,7 +194,8 @@ public class ThreadPool extends AbstractComponent {
assert settings.get("name") != null : "ThreadPool's settings should contain a name";
Map<String, Settings> groupSettings = getThreadPoolSettingsGroup(settings);
Map<String, Settings> groupSettings = THREADPOOL_GROUP_SETTING.get(settings).getAsGroups();
validate(groupSettings);
int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
int halfProcMaxAt5 = Math.min(((availableProcessors + 1) / 2), 5);
@ -252,18 +250,12 @@ public class ThreadPool extends AbstractComponent {
this.estimatedTimeThread.start();
}
private Map<String, Settings> getThreadPoolSettingsGroup(Settings settings) {
Map<String, Settings> groupSettings = settings.getGroups(THREADPOOL_GROUP);
validate(groupSettings);
return groupSettings;
}
public void setNodeSettingsService(NodeSettingsService nodeSettingsService) {
if(settingsListenerIsSet) {
public void setClusterSettings(ClusterSettings clusterSettings) {
if(settingsListenerIsSet.compareAndSet(false, true)) {
clusterSettings.addSettingsUpdateConsumer(THREADPOOL_GROUP_SETTING, this::updateSettings, (s) -> validate(s.getAsGroups()));
} else {
throw new IllegalStateException("the node settings listener was set more then once");
}
nodeSettingsService.addListener(new ApplySettings());
settingsListenerIsSet = true;
}
public long estimatedTimeInMillis() {
@ -526,8 +518,8 @@ public class ThreadPool extends AbstractComponent {
throw new IllegalArgumentException("No type found [" + type + "], for [" + name + "]");
}
public void updateSettings(Settings settings) {
Map<String, Settings> groupSettings = getThreadPoolSettingsGroup(settings);
private void updateSettings(Settings settings) {
Map<String, Settings> groupSettings = settings.getAsGroups();
if (groupSettings.isEmpty()) {
return;
}
@ -583,7 +575,7 @@ public class ThreadPool extends AbstractComponent {
ThreadPoolType correctThreadPoolType = THREAD_POOL_TYPES.get(key);
// TODO: the type equality check can be removed after #3760/#6732 are addressed
if (type != null && !correctThreadPoolType.getType().equals(type)) {
throw new IllegalArgumentException("setting " + THREADPOOL_GROUP + key + ".type to " + type + " is not permitted; must be " + correctThreadPoolType.getType());
throw new IllegalArgumentException("setting " + THREADPOOL_GROUP_SETTING.getKey() + key + ".type to " + type + " is not permitted; must be " + correctThreadPoolType.getType());
}
}
}
@ -866,13 +858,6 @@ public class ThreadPool extends AbstractComponent {
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
updateSettings(settings);
}
}
/**
* Returns <code>true</code> if the given service was terminated successfully. If the termination timed out,
* the service is <code>null</code> this method will return <code>false</code>.
@ -911,38 +896,4 @@ public class ThreadPool extends AbstractComponent {
}
return false;
}
public static ThreadPoolTypeSettingsValidator THREAD_POOL_TYPE_SETTINGS_VALIDATOR = new ThreadPoolTypeSettingsValidator();
private static class ThreadPoolTypeSettingsValidator implements Validator {
@Override
public String validate(String setting, String value, ClusterState clusterState) {
// TODO: the type equality validation can be removed after #3760/#6732 are addressed
Matcher matcher = Pattern.compile("threadpool\\.(.*)\\.type").matcher(setting);
if (!matcher.matches()) {
return null;
} else {
String threadPool = matcher.group(1);
ThreadPool.ThreadPoolType defaultThreadPoolType = ThreadPool.THREAD_POOL_TYPES.get(threadPool);
ThreadPool.ThreadPoolType threadPoolType;
try {
threadPoolType = ThreadPool.ThreadPoolType.fromType(value);
} catch (IllegalArgumentException e) {
return e.getMessage();
}
if (defaultThreadPoolType.equals(threadPoolType)) {
return null;
} else {
return String.format(
Locale.ROOT,
"thread pool type for [%s] can only be updated to [%s] but was [%s]",
threadPool,
defaultThreadPoolType.getType(),
threadPoolType.getType()
);
}
}
}
}
}

View File

@ -21,6 +21,8 @@ package org.elasticsearch.transport;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
@ -34,9 +36,8 @@ import java.util.Map;
public interface Transport extends LifecycleComponent<Transport> {
public static class TransportSettings {
public static final String TRANSPORT_TCP_COMPRESS = "transport.tcp.compress";
}
Setting<Settings> TRANSPORT_PROFILES_SETTING = Setting.groupSetting("transport.profiles.", true, Setting.Scope.CLUSTER);
Setting<Boolean> TRANSPORT_TCP_COMPRESS = Setting.boolSetting("transport.tcp.compress", false, false, Setting.Scope.CLUSTER);
void transportServiceAdapter(TransportServiceAdapter service);

View File

@ -29,6 +29,8 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.metrics.MeanMetric;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress;
@ -37,20 +39,15 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.*;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.function.Supplier;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
@ -88,14 +85,14 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
// tracer log
public static final String SETTING_TRACE_LOG_INCLUDE = "transport.tracer.include";
public static final String SETTING_TRACE_LOG_EXCLUDE = "transport.tracer.exclude";
public static final Setting<List<String>> TRACE_LOG_INCLUDE_SETTING = Setting.listSetting("transport.tracer.include", Collections.emptyList(), Function.identity(), true, Setting.Scope.CLUSTER);
public static final Setting<List<String>> TRACE_LOG_EXCLUDE_SETTING = Setting.listSetting("transport.tracer.exclude", Arrays.asList("internal:discovery/zen/fd*", TransportLivenessAction.NAME), Function.identity(), true, Setting.Scope.CLUSTER);
private final ESLogger tracerLog;
volatile String[] tracerLogInclude;
volatile String[] tracelLogExclude;
private final ApplySettings settingsListener = new ApplySettings();
/** if set will call requests sent to this id to shortcut and executed locally */
volatile DiscoveryNode localNode = null;
@ -109,8 +106,8 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
super(settings);
this.transport = transport;
this.threadPool = threadPool;
this.tracerLogInclude = settings.getAsArray(SETTING_TRACE_LOG_INCLUDE, Strings.EMPTY_ARRAY, true);
this.tracelLogExclude = settings.getAsArray(SETTING_TRACE_LOG_EXCLUDE, new String[]{"internal:discovery/zen/fd*", TransportLivenessAction.NAME}, true);
setTracerLogInclude(TRACE_LOG_INCLUDE_SETTING.get(settings));
setTracerLogExclude(TRACE_LOG_EXCLUDE_SETTING.get(settings));
tracerLog = Loggers.getLogger(logger, ".tracer");
adapter = createAdapter();
}
@ -134,34 +131,18 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
// These need to be optional as they don't exist in the context of a transport client
@Inject(optional = true)
public void setDynamicSettings(NodeSettingsService nodeSettingsService) {
nodeSettingsService.addListener(settingsListener);
public void setDynamicSettings(ClusterSettings clusterSettings) {
clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_INCLUDE_SETTING, this::setTracerLogInclude);
clusterSettings.addSettingsUpdateConsumer(TRACE_LOG_EXCLUDE_SETTING, this::setTracerLogExclude);
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
String[] newTracerLogInclude = settings.getAsArray(SETTING_TRACE_LOG_INCLUDE, TransportService.this.tracerLogInclude, true);
String[] newTracerLogExclude = settings.getAsArray(SETTING_TRACE_LOG_EXCLUDE, TransportService.this.tracelLogExclude, true);
if (newTracerLogInclude == TransportService.this.tracerLogInclude && newTracerLogExclude == TransportService.this.tracelLogExclude) {
return;
}
if (Arrays.equals(newTracerLogInclude, TransportService.this.tracerLogInclude) &&
Arrays.equals(newTracerLogExclude, TransportService.this.tracelLogExclude)) {
return;
}
TransportService.this.tracerLogInclude = newTracerLogInclude;
TransportService.this.tracelLogExclude = newTracerLogExclude;
logger.info("tracer log updated to use include: {}, exclude: {}", newTracerLogInclude, newTracerLogExclude);
}
void setTracerLogInclude(List<String> tracerLogInclude) {
this.tracerLogInclude = tracerLogInclude.toArray(Strings.EMPTY_ARRAY);
}
// used for testing
public void applySettings(Settings settings) {
settingsListener.onRefreshSettings(settings);
void setTracerLogExclude(List<String> tracelLogExclude) {
this.tracelLogExclude = tracelLogExclude.toArray(Strings.EMPTY_ARRAY);
}
@Override
protected void doStart() {
adapter.rxMetric.clear();

View File

@ -225,7 +225,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
this.connectTimeout = this.settings.getAsTime("transport.netty.connect_timeout", settings.getAsTime("transport.tcp.connect_timeout", settings.getAsTime(TCP_CONNECT_TIMEOUT, TCP_DEFAULT_CONNECT_TIMEOUT)));
this.maxCumulationBufferCapacity = this.settings.getAsBytesSize("transport.netty.max_cumulation_buffer_capacity", null);
this.maxCompositeBufferComponents = this.settings.getAsInt("transport.netty.max_composite_buffer_components", -1);
this.compress = settings.getAsBoolean(TransportSettings.TRANSPORT_TCP_COMPRESS, false);
this.compress = Transport.TRANSPORT_TCP_COMPRESS.get(settings);
this.connectionsPerNodeRecovery = this.settings.getAsInt("transport.netty.connections_per_node.recovery", settings.getAsInt(CONNECTIONS_PER_NODE_RECOVERY, 2));
this.connectionsPerNodeBulk = this.settings.getAsInt("transport.netty.connections_per_node.bulk", settings.getAsInt(CONNECTIONS_PER_NODE_BULK, 3));
@ -295,7 +295,7 @@ public class NettyTransport extends AbstractLifecycleComponent<Transport> implem
this.serverOpenChannels = openChannels;
// extract default profile first and create standard bootstrap
Map<String, Settings> profiles = settings.getGroups("transport.profiles", true);
Map<String, Settings> profiles = TRANSPORT_PROFILES_SETTING.get(settings()).getAsGroups(true);
if (!profiles.containsKey(DEFAULT_PROFILE)) {
profiles = new HashMap<>(profiles);
profiles.put(DEFAULT_PROFILE, Settings.EMPTY);

View File

@ -0,0 +1,126 @@
/*
* 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.action.admin.cluster.settings;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import java.util.concurrent.atomic.AtomicReference;
public class SettingsUpdaterTests extends ESTestCase {
public void testUpdateSetting() {
AtomicReference<Float> index = new AtomicReference<>();
AtomicReference<Float> shard = new AtomicReference<>();
ClusterState.Builder builder = ClusterState.builder(new ClusterName("foo"));
ClusterSettings settingsService = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, index::set);
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, shard::set);
SettingsUpdater updater = new SettingsUpdater(settingsService);
MetaData.Builder metaData = MetaData.builder()
.persistentSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 1.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 2.5).build())
.transientSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 3.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 4.5).build());
ClusterState build = builder.metaData(metaData).build();
ClusterState clusterState = updater.updateSettings(build, Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 0.5).build(),
Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 0.4).build());
assertNotSame(clusterState, build);
assertEquals(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.get(clusterState.metaData().persistentSettings()), 0.4, 0.1);
assertEquals(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.get(clusterState.metaData().persistentSettings()), 2.5, 0.1);
assertEquals(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.get(clusterState.metaData().transientSettings()), 0.5, 0.1);
assertEquals(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.get(clusterState.metaData().transientSettings()), 4.5, 0.1);
clusterState = updater.updateSettings(clusterState, Settings.builder().putNull("cluster.routing.*").build(),
Settings.EMPTY);
assertEquals(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.get(clusterState.metaData().persistentSettings()), 0.4, 0.1);
assertEquals(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.get(clusterState.metaData().persistentSettings()), 2.5, 0.1);
assertFalse(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.exists(clusterState.metaData().transientSettings()));
assertFalse(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.exists(clusterState.metaData().transientSettings()));
clusterState = updater.updateSettings(clusterState,
Settings.EMPTY, Settings.builder().putNull("cluster.routing.*").put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 10.0).build());
assertEquals(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.get(clusterState.metaData().persistentSettings()), 10.0, 0.1);
assertFalse(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.exists(clusterState.metaData().persistentSettings()));
assertFalse(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.exists(clusterState.metaData().transientSettings()));
assertFalse(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.exists(clusterState.metaData().transientSettings()));
assertNull("updater only does a dryRun", index.get());
assertNull("updater only does a dryRun", shard.get());
}
public void testAllOrNothing() {
ClusterState.Builder builder = ClusterState.builder(new ClusterName("foo"));
ClusterSettings settingsService = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
AtomicReference<Float> index = new AtomicReference<>();
AtomicReference<Float> shard = new AtomicReference<>();
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, index::set);
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, shard::set);
SettingsUpdater updater = new SettingsUpdater(settingsService);
MetaData.Builder metaData = MetaData.builder()
.persistentSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 1.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 2.5).build())
.transientSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 3.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 4.5).build());
ClusterState build = builder.metaData(metaData).build();
try {
updater.updateSettings(build, Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), "not a float").build(),
Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), "not a float").put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 1.0f).build());
fail("all or nothing");
} catch (IllegalArgumentException ex) {
logger.info("", ex);
assertEquals("Failed to parse value [not a float] for setting [cluster.routing.allocation.balance.index]", ex.getMessage());
}
assertNull("updater only does a dryRun", index.get());
assertNull("updater only does a dryRun", shard.get());
}
public void testClusterBlock() {
ClusterState.Builder builder = ClusterState.builder(new ClusterName("foo"));
ClusterSettings settingsService = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
AtomicReference<Float> index = new AtomicReference<>();
AtomicReference<Float> shard = new AtomicReference<>();
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING, index::set);
settingsService.addSettingsUpdateConsumer(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, shard::set);
SettingsUpdater updater = new SettingsUpdater(settingsService);
MetaData.Builder metaData = MetaData.builder()
.persistentSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 1.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 2.5).build())
.transientSettings(Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 3.5)
.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 4.5).build());
ClusterState build = builder.metaData(metaData).build();
ClusterState clusterState = updater.updateSettings(build, Settings.builder().put(MetaData.SETTING_READ_ONLY_SETTING.getKey(), true).build(),
Settings.builder().put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 1.6).put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 1.0f).build());
assertEquals(clusterState.blocks().global().size(), 1);
assertEquals(clusterState.blocks().global().iterator().next(), MetaData.CLUSTER_READ_ONLY_BLOCK);
clusterState = updater.updateSettings(build, Settings.EMPTY,
Settings.builder().put(MetaData.SETTING_READ_ONLY_SETTING.getKey(), false).build());
assertEquals(clusterState.blocks().global().size(), 0);
}
}

View File

@ -65,7 +65,7 @@ public class UpgradeIT extends ESBackcompatTestCase {
public void testUpgrade() throws Exception {
// allow the cluster to rebalance quickly - 2 concurrent rebalance are default we can do higher
Settings.Builder builder = Settings.builder();
builder.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 100);
builder.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), 100);
client().admin().cluster().prepareUpdateSettings().setPersistentSettings(builder).get();
int numIndexes = randomIntBetween(2, 4);
@ -117,13 +117,13 @@ public class UpgradeIT extends ESBackcompatTestCase {
ensureGreen();
// disable allocation entirely until all nodes are upgraded
builder = Settings.builder();
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.NONE);
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), EnableAllocationDecider.Allocation.NONE);
client().admin().cluster().prepareUpdateSettings().setTransientSettings(builder).get();
backwardsCluster().upgradeAllNodes();
builder = Settings.builder();
// disable rebalanceing entirely for the time being otherwise we might get relocations / rebalance from nodes with old segments
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE);
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, EnableAllocationDecider.Allocation.ALL);
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE);
builder.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), EnableAllocationDecider.Allocation.ALL);
client().admin().cluster().prepareUpdateSettings().setTransientSettings(builder).get();
ensureGreen();
logger.info("--> Nodes upgrade complete");

View File

@ -45,8 +45,8 @@ public class IndexingMasterFailoverIT extends ESIntegTestCase {
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s") // for hitting simulated network failures quickly
.put(FaultDetection.SETTING_PING_RETRIES, "1") // for hitting simulated network failures quickly
.put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out
.put(DiscoverySettings.PUBLISH_TIMEOUT, "1s") // <-- for hitting simulated network failures quickly
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2)
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)
.build();
internalCluster().startMasterOnlyNodesAsync(3, sharedSettings).get();

View File

@ -107,7 +107,7 @@ public class OldIndexBackwardsCompatibilityIT extends ESIntegTestCase {
public Settings nodeSettings(int ord) {
return Settings.builder()
.put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) // disable merging so no segments will be upgraded
.put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, 30) // increase recovery speed for small files
.put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS_SETTING.getKey(), 30) // increase recovery speed for small files
.build();
}

View File

@ -181,7 +181,7 @@ public class RestoreBackwardsCompatIT extends AbstractSnapshotIntegTestCase {
logger.info("--> check settings");
ClusterState clusterState = client().admin().cluster().prepareState().get().getState();
assertThat(clusterState.metaData().persistentSettings().get(FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP + "version_attr"), equalTo(version));
assertThat(clusterState.metaData().persistentSettings().get(FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + "version_attr"), equalTo(version));
logger.info("--> check templates");
IndexTemplateMetaData template = clusterState.getMetaData().templates().get("template_" + version.toLowerCase(Locale.ROOT));

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.Store;
@ -126,7 +127,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
// manual collection or upon cluster forming.
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT, "1s")
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT_SETTING.getKey(), "1s")
.build();
}
@ -137,9 +138,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
}
public void testClusterInfoServiceCollectsInformation() throws Exception {
internalCluster().startNodesAsync(2,
Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "200ms").build())
.get();
internalCluster().startNodesAsync(2).get();
assertAcked(prepareCreate("test").setSettings(settingsBuilder()
.put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, 0)
.put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE).build()));
@ -147,6 +146,8 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
InternalTestCluster internalTestCluster = internalCluster();
// Get the cluster info service on the master node
final InternalClusterInfoService infoService = (InternalClusterInfoService) internalTestCluster.getInstance(ClusterInfoService.class, internalTestCluster.getMasterName());
infoService.setUpdateFrequency(TimeValue.timeValueMillis(200));
infoService.onMaster();
ClusterInfo info = infoService.refresh();
assertNotNull("info should not be null", info);
ImmutableOpenMap<String, DiskUsage> leastUsages = info.getNodeLeastAvailableDiskUsages();
@ -188,7 +189,7 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
public void testClusterInfoServiceInformationClearOnError() throws InterruptedException, ExecutionException {
internalCluster().startNodesAsync(2,
// manually control publishing
Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "60m").build())
Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL_SETTING.getKey(), "60m").build())
.get();
prepareCreate("test").setSettings(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).get();
ensureGreen("test");

View File

@ -31,11 +31,10 @@ import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllo
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
import org.elasticsearch.cluster.settings.DynamicSettings;
import org.elasticsearch.cluster.settings.Validator;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.*;
import org.elasticsearch.index.settings.IndexDynamicSettings;
public class ClusterModuleTests extends ModuleTestCase {
@ -73,18 +72,20 @@ public class ClusterModuleTests extends ModuleTestCase {
}
public void testRegisterClusterDynamicSettingDuplicate() {
ClusterModule module = new ClusterModule(Settings.EMPTY);
final SettingsFilter settingsFilter = new SettingsFilter(Settings.EMPTY);
SettingsModule module = new SettingsModule(Settings.EMPTY, settingsFilter);
try {
module.registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
module.registerSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Cannot register setting [" + EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE + "] twice");
assertEquals(e.getMessage(), "Cannot register setting [" + EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey() + "] twice");
}
}
public void testRegisterClusterDynamicSetting() {
ClusterModule module = new ClusterModule(Settings.EMPTY);
module.registerClusterDynamicSetting("foo.bar", Validator.EMPTY);
assertInstanceBindingWithAnnotation(module, DynamicSettings.class, dynamicSettings -> dynamicSettings.hasDynamicSetting("foo.bar"), ClusterDynamicSettings.class);
final SettingsFilter settingsFilter = new SettingsFilter(Settings.EMPTY);
SettingsModule module = new SettingsModule(Settings.EMPTY, settingsFilter);
module.registerSetting(Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER));
assertInstanceBinding(module, ClusterSettings.class, service -> service.hasDynamicSetting("foo.bar"));
}
public void testRegisterIndexDynamicSettingDuplicate() {

View File

@ -971,7 +971,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
public void testLongClusterStateUpdateLogging() throws Exception {
Settings settings = settingsBuilder()
.put("discovery.type", "local")
.put(InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, "10s")
.put(InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.getKey(), "10s")
.build();
internalCluster().startNode(settings);
ClusterService clusterService1 = internalCluster().getInstance(ClusterService.class);
@ -1007,7 +1007,7 @@ public class ClusterServiceIT extends ESIntegTestCase {
processedFirstTask.await(1, TimeUnit.SECONDS);
assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put(InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, "10ms")));
.put(InternalClusterService.CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.getKey(), "10ms")));
clusterService1.submitStateUpdateTask("test2", new ClusterStateUpdateTask() {
@Override

View File

@ -280,7 +280,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
assertNoMasterBlockOnAllNodes();
logger.info("--> bringing another node up");
internalCluster().startNode(settingsBuilder().put(settings).put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2).build());
internalCluster().startNode(settingsBuilder().put(settings).put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2).build());
clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNodes("2").get();
assertThat(clusterHealthResponse.isTimedOut(), equalTo(false));
}
@ -317,7 +317,7 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
// set an initial value which is at least quorum to avoid split brains during initial startup
int initialMinMasterNodes = randomIntBetween(nodeCount / 2 + 1, nodeCount);
settings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, initialMinMasterNodes);
settings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), initialMinMasterNodes);
logger.info("--> starting [{}] nodes. min_master_nodes set to [{}]", nodeCount, initialMinMasterNodes);
@ -328,19 +328,21 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
int updateCount = randomIntBetween(1, nodeCount);
logger.info("--> updating [{}] to [{}]", ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, updateCount);
logger.info("--> updating [{}] to [{}]", ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), updateCount);
assertAcked(client().admin().cluster().prepareUpdateSettings()
.setPersistentSettings(settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, updateCount)));
.setPersistentSettings(settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), updateCount)));
logger.info("--> verifying no node left and master is up");
assertFalse(client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(nodeCount)).get().isTimedOut());
updateCount = nodeCount + randomIntBetween(1, 2000);
logger.info("--> trying to updating [{}] to [{}]", ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, updateCount);
assertThat(client().admin().cluster().prepareUpdateSettings()
.setPersistentSettings(settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, updateCount))
.get().getPersistentSettings().getAsMap().keySet(),
empty());
logger.info("--> trying to updating [{}] to [{}]", ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), updateCount);
try {
client().admin().cluster().prepareUpdateSettings()
.setPersistentSettings(settingsBuilder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), updateCount));
} catch (IllegalArgumentException ex) {
assertEquals(ex.getMessage(), "cannot set discovery.zen.minimum_master_nodes to more than the current master nodes count [" +updateCount+ "]");
}
logger.info("--> verifying no node left and master is up");
assertFalse(client().admin().cluster().prepareHealth().setWaitForNodes(Integer.toString(nodeCount)).get().isTimedOut());
@ -351,8 +353,8 @@ public class MinimumMasterNodesIT extends ESIntegTestCase {
.put("discovery.type", "zen")
.put(FaultDetection.SETTING_PING_TIMEOUT, "1h") // disable it
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2)
.put(DiscoverySettings.COMMIT_TIMEOUT, "100ms") // speed things up
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)
.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "100ms") // speed things up
.build();
internalCluster().startNodesAsync(3, settings).get();
ensureGreen(); // ensure cluster state is recovered before we disrupt things

View File

@ -67,7 +67,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
.put("discovery.zen.minimum_master_nodes", 2)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put(DiscoverySettings.NO_MASTER_BLOCK, "all")
.put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all")
.build();
TimeValue timeout = TimeValue.timeValueMillis(200);
@ -219,7 +219,7 @@ public class NoMasterNodeIT extends ESIntegTestCase {
.put("discovery.zen.minimum_master_nodes", 2)
.put(ZenDiscovery.SETTING_PING_TIMEOUT, "200ms")
.put("discovery.initial_state_timeout", "500ms")
.put(DiscoverySettings.NO_MASTER_BLOCK, "write")
.put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "write")
.build();
internalCluster().startNode(settings);

View File

@ -50,8 +50,8 @@ public class AckClusterUpdateSettingsIT extends ESIntegTestCase {
.put(super.nodeSettings(nodeOrdinal))
//make sure that enough concurrent reroutes can happen at the same time
//we have a minimum of 2 nodes, and a maximum of 10 shards, thus 5 should be enough
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, 5)
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 10)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), 5)
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), 10)
.build();
}
@ -69,7 +69,7 @@ public class AckClusterUpdateSettingsIT extends ESIntegTestCase {
private void removePublishTimeout() {
//to test that the acknowledgement mechanism is working we better disable the wait for publish
//otherwise the operation is most likely acknowledged even if it doesn't support ack
assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "0")));
assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "0")));
}
public void testClusterUpdateSettingsAcknowledgement() {

View File

@ -67,7 +67,7 @@ public class AckIT extends ESIntegTestCase {
//to test that the acknowledgement mechanism is working we better disable the wait for publish
//otherwise the operation is most likely acknowledged even if it doesn't support ack
return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put(DiscoverySettings.PUBLISH_TIMEOUT, 0).build();
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), 0).build();
}
public void testUpdateSettingsAcknowledgement() {

View File

@ -106,9 +106,9 @@ public class AwarenessAllocationIT extends ESIntegTestCase {
public void testAwarenessZones() throws Exception {
Settings commonSettings = Settings.settingsBuilder()
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP + "zone.values", "a,b")
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES, "zone")
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 3)
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP_SETTING.getKey() + "zone.values", "a,b")
.put(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTE_SETTING.getKey(), "zone")
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 3)
.put(ZenDiscovery.SETTING_JOIN_TIMEOUT, "10s")
.build();

View File

@ -56,7 +56,7 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_ME
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_READ;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_WRITE;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBlocked;
@ -71,15 +71,15 @@ public class ClusterRerouteIT extends ESIntegTestCase {
public void testRerouteWithCommands_disableAllocationSettings() throws Exception {
Settings commonSettings = settingsBuilder()
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none")
.build();
rerouteWithCommands(commonSettings);
}
public void testRerouteWithCommands_enableAllocationSettings() throws Exception {
Settings commonSettings = settingsBuilder()
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.NONE.name())
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE.name())
.build();
rerouteWithCommands(commonSettings);
}
@ -147,15 +147,15 @@ public class ClusterRerouteIT extends ESIntegTestCase {
public void testRerouteWithAllocateLocalGateway_disableAllocationSettings() throws Exception {
Settings commonSettings = settingsBuilder()
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none")
.build();
rerouteWithAllocateLocalGateway(commonSettings);
}
public void testRerouteWithAllocateLocalGateway_enableAllocationSettings() throws Exception {
Settings commonSettings = settingsBuilder()
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.NONE.name())
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE.name())
.build();
rerouteWithAllocateLocalGateway(commonSettings);
}
@ -279,7 +279,7 @@ public class ClusterRerouteIT extends ESIntegTestCase {
logger.info("--> disable allocation");
Settings newSettings = settingsBuilder()
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.NONE.name())
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE.name())
.build();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(newSettings).execute().actionGet();

View File

@ -50,7 +50,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
public void testAddNodesAndIndices() {
Settings.Builder settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
AllocationService service = createAllocationService(settings.build());
ClusterState clusterState = initCluster(service, 1, 3, 3, 1);
@ -93,7 +93,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
public void testMinimalRelocations() {
Settings.Builder settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString())
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString())
.put("cluster.routing.allocation.node_concurrent_recoveries", 2);
AllocationService service = createAllocationService(settings.build());
@ -161,7 +161,7 @@ public class AddIncrementallyTests extends ESAllocationTestCase {
public void testMinimalRelocationsNoLimit() {
Settings.Builder settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString())
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString())
.put("cluster.routing.allocation.node_concurrent_recoveries", 100)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 100);
AllocationService service = createAllocationService(settings.build());

View File

@ -98,8 +98,8 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
public void testAllocateCommand() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none")
.build());
logger.info("--> building initial routing table");
@ -186,8 +186,8 @@ public class AllocationCommandsTests extends ESAllocationTestCase {
public void testCancelCommand() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")
.put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none")
.build());
logger.info("--> building initial routing table");

View File

@ -39,8 +39,8 @@ public class AllocationPriorityTests extends ESAllocationTestCase {
public void testPrioritizedIndicesAllocatedFirst() {
AllocationService allocation = createAllocationService(settingsBuilder().
put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CONCURRENT_RECOVERIES, 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, 1).build());
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING.getKey(), 1)
.put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), 1).build());
final String highPriorityName;
final String lowPriorityName;
final int priorityFirst;

View File

@ -55,7 +55,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testMoveShardOnceNewNodeWithAttributeAdded1() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -123,7 +123,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testMoveShardOnceNewNodeWithAttributeAdded2() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -193,7 +193,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.put("cluster.routing.allocation.balance.index", 0.0f)
@ -293,7 +293,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -387,7 +387,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testMoveShardOnceNewNodeWithAttributeAdded5() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -465,7 +465,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testMoveShardOnceNewNodeWithAttributeAdded6() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -545,7 +545,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testFullAwareness1() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.force.rack_id.values", "1,2")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -612,7 +612,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testFullAwareness2() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.force.rack_id.values", "1,2")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
.build());
@ -681,7 +681,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.put("cluster.routing.allocation.awareness.force.rack_id.values", "1,2")
.put("cluster.routing.allocation.awareness.attributes", "rack_id")
@ -767,7 +767,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
.put("cluster.routing.allocation.awareness.attributes", "zone")
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());
@ -828,7 +828,7 @@ public class AwarenessAllocationTests extends ESAllocationTestCase {
public void testUnassignedShardsWithUnbalancedZones() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "zone")
.build());

View File

@ -37,10 +37,10 @@ import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllo
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.test.gateway.NoopGatewayAllocator;
import org.hamcrest.Matchers;
@ -65,10 +65,10 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
final float balanceTreshold = 1.0f;
Settings.Builder settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
settings.put(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, indexBalance);
settings.put(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, replicaBalance);
settings.put(BalancedShardsAllocator.SETTING_THRESHOLD, balanceTreshold);
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
settings.put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), indexBalance);
settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance);
settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceTreshold);
AllocationService strategy = createAllocationService(settings.build());
@ -90,10 +90,10 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
final float balanceTreshold = 1.0f;
Settings.Builder settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
settings.put(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, indexBalance);
settings.put(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, replicaBalance);
settings.put(BalancedShardsAllocator.SETTING_THRESHOLD, balanceTreshold);
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
settings.put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), indexBalance);
settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance);
settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceTreshold);
AllocationService strategy = createAllocationService(settings.build());
@ -279,36 +279,30 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
public void testPersistedSettings() {
Settings.Builder settings = settingsBuilder();
settings.put(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, 0.2);
settings.put(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, 0.3);
settings.put(BalancedShardsAllocator.SETTING_THRESHOLD, 2.0);
final NodeSettingsService.Listener[] listeners = new NodeSettingsService.Listener[1];
NodeSettingsService service = new NodeSettingsService(settingsBuilder().build()) {
@Override
public void addListener(Listener listener) {
assertNull("addListener was called twice while only one time was expected", listeners[0]);
listeners[0] = listener;
}
};
settings.put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 0.2);
settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 0.3);
settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), 2.0);
ClusterSettings service = new ClusterSettings(settingsBuilder().build(), ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
BalancedShardsAllocator allocator = new BalancedShardsAllocator(settings.build(), service);
assertThat(allocator.getIndexBalance(), Matchers.equalTo(0.2f));
assertThat(allocator.getShardBalance(), Matchers.equalTo(0.3f));
assertThat(allocator.getThreshold(), Matchers.equalTo(2.0f));
settings = settingsBuilder();
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
listeners[0].onRefreshSettings(settings.build());
settings.put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 0.2);
settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 0.3);
settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), 2.0);
settings.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString());
service.applySettings(settings.build());
assertThat(allocator.getIndexBalance(), Matchers.equalTo(0.2f));
assertThat(allocator.getShardBalance(), Matchers.equalTo(0.3f));
assertThat(allocator.getThreshold(), Matchers.equalTo(2.0f));
settings = settingsBuilder();
settings.put(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, 0.5);
settings.put(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, 0.1);
settings.put(BalancedShardsAllocator.SETTING_THRESHOLD, 3.0);
listeners[0].onRefreshSettings(settings.build());
settings.put(BalancedShardsAllocator.INDEX_BALANCE_FACTOR_SETTING.getKey(), 0.5);
settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), 0.1);
settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), 3.0);
service.applySettings(settings.build());
assertThat(allocator.getIndexBalance(), Matchers.equalTo(0.5f));
assertThat(allocator.getShardBalance(), Matchers.equalTo(0.1f));
assertThat(allocator.getThreshold(), Matchers.equalTo(3.0f));
@ -317,7 +311,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
public void testNoRebalanceOnPrimaryOverload() {
Settings.Builder settings = settingsBuilder();
AllocationService strategy = new AllocationService(settings.build(), randomAllocationDeciders(settings.build(),
new NodeSettingsService(Settings.Builder.EMPTY_SETTINGS), getRandom()), new ShardsAllocators(settings.build(),
new ClusterSettings(Settings.Builder.EMPTY_SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), getRandom()), new ShardsAllocators(settings.build(),
NoopGatewayAllocator.INSTANCE, new ShardsAllocator() {
@Override

View File

@ -46,7 +46,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
private final ESLogger logger = Loggers.getLogger(ClusterRebalanceRoutingTests.class);
public void testAlways() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build());
MetaData metaData = MetaData.builder()
@ -132,7 +132,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
public void testClusterPrimariesActive1() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.INDICES_PRIMARIES_ACTIVE.toString()).build());
MetaData metaData = MetaData.builder()
@ -236,7 +236,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
public void testClusterPrimariesActive2() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.INDICES_PRIMARIES_ACTIVE.toString()).build());
MetaData metaData = MetaData.builder()
@ -320,7 +320,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
public void testClusterAllActive1() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.INDICES_ALL_ACTIVE.toString()).build());
MetaData metaData = MetaData.builder()
@ -443,7 +443,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
public void testClusterAllActive2() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.INDICES_ALL_ACTIVE.toString()).build());
MetaData metaData = MetaData.builder()
@ -527,7 +527,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
}
public void testClusterAllActive3() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.INDICES_ALL_ACTIVE.toString()).build());
MetaData metaData = MetaData.builder()
@ -737,7 +737,7 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase {
public void testRebalanceWhileShardFetching() {
final AtomicBoolean hasFetches = new AtomicBoolean(true);
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build(), new NoopGatewayAllocator() {
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {

View File

@ -46,7 +46,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
public void testSimpleDeadNodeOnStartedPrimaryShard() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("--> building initial routing table");
@ -97,7 +97,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
public void testDeadNodeWhileRelocatingOnToNode() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("--> building initial routing table");
@ -171,7 +171,7 @@ public class DeadNodesAllocationTests extends ESAllocationTestCase {
public void testDeadNodeWhileRelocatingOnFromNode() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("--> building initial routing table");

View File

@ -41,7 +41,7 @@ public class FailedNodeRoutingTests extends ESAllocationTestCase {
private final ESLogger logger = Loggers.getLogger(FailedNodeRoutingTests.class);
public void testSimpleFailedNodeTest() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build());
MetaData metaData = MetaData.builder()

View File

@ -57,7 +57,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testFailedShardPrimaryRelocatingToAndFrom() {
AllocationService allocation = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("--> building initial routing table");
@ -145,7 +145,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testFailPrimaryStartedCheckReplicaElected() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("Building initial routing table");
@ -226,7 +226,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testFirstAllocationFailureSingleNode() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("Building initial routing table");
@ -282,7 +282,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testSingleShardMultipleAllocationFailures() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("Building initial routing table");
@ -338,7 +338,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testFirstAllocationFailureTwoNodes() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("Building initial routing table");
@ -398,7 +398,7 @@ public class FailedShardsRoutingTests extends ESAllocationTestCase {
public void testRebalanceFailure() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
logger.info("Building initial routing table");

View File

@ -48,7 +48,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");
@ -178,7 +178,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");
@ -340,7 +340,7 @@ public class IndexBalanceTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");

View File

@ -53,7 +53,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
public void testDoNotAllocateFromPrimary() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());
@ -167,7 +167,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
public void testRandom() {
AllocationService service = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());
@ -216,7 +216,7 @@ public class NodeVersionAllocationDeciderTests extends ESAllocationTestCase {
public void testRollingRestart() {
AllocationService service = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());

View File

@ -57,7 +57,7 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(),
new ClusterInfoService() {

View File

@ -47,7 +47,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");
@ -119,7 +119,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");
@ -211,7 +211,7 @@ public class RoutingNodesIntegrityTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 1)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 3)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1).build());
logger.info("Building initial routing table");

View File

@ -41,7 +41,7 @@ public class ShardVersioningTests extends ESAllocationTestCase {
private final ESLogger logger = Loggers.getLogger(ShardVersioningTests.class);
public void testSimple() {
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
AllocationService strategy = createAllocationService(settingsBuilder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(),
ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build());
MetaData metaData = MetaData.builder()

View File

@ -90,7 +90,7 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
public void testClusterLevelShardsLimitAllocate() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE, 1)
.put(ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), 1)
.build());
logger.info("Building initial routing table");
@ -126,7 +126,7 @@ public class ShardsLimitAllocationTests extends ESAllocationTestCase {
// Bump the cluster total shards to 2
strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE, 2)
.put(ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.getKey(), 2)
.build());
logger.info("Do another reroute, make sure shards are now allocated");

View File

@ -211,7 +211,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
public void testMultiIndexEvenDistribution() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());
@ -323,7 +323,7 @@ public class SingleShardNoReplicasRoutingTests extends ESAllocationTestCase {
public void testMultiIndexUnevenNodes() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build());

View File

@ -50,7 +50,7 @@ public class TenShardsOneReplicaRoutingTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 10)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.put("cluster.routing.allocation.balance.index", 0.0f)
.put("cluster.routing.allocation.balance.replica", 1.0f)

View File

@ -60,9 +60,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testDiskThreshold() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node1", new DiskUsage("node1", "node1", "/dev/null", 100, 10)); // 90% used
@ -96,7 +96,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
};
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -170,9 +170,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Set the high threshold to 70 instead of 80
// node2 now should not have new shards allocated to it, but shards can remain
diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.7).build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.7).build();
deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(
@ -181,7 +181,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -201,9 +201,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Set the high threshold to 60 instead of 70
// node2 now should not have new shards allocated to it, and shards cannot remain
diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 0.5)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.6).build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.5)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.6).build();
deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(
@ -212,7 +212,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -254,9 +254,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testDiskThresholdWithAbsoluteSizes() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "30b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "9b").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "30b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "9b").build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 10)); // 90% used
@ -292,7 +292,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -349,7 +349,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
};
strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -405,9 +405,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Set the high threshold to 70 instead of 80
// node2 now should not have new shards allocated to it, but shards can remain
diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "40b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "30b").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "40b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "30b").build();
deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(
@ -416,7 +416,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -436,9 +436,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// Set the high threshold to 60 instead of 70
// node2 now should not have new shards allocated to it, and shards cannot remain
diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "50b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "40b").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "50b")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "40b").build();
deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList(
@ -447,7 +447,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -522,9 +522,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testDiskThresholdWithShardSizes() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "71%").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "71%").build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 31)); // 69% used
@ -556,7 +556,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -589,9 +589,9 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testUnknownDiskUsage() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.85).build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.85).build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node2", new DiskUsage("node2", "node2", "/dev/null", 100, 50)); // 50% used
@ -624,7 +624,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -688,10 +688,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testShardRelocationsTakenIntoAccount() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), 0.7)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), 0.8).build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 40)); // 60% used
@ -727,7 +727,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
@ -794,10 +794,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testCanRemainWithShardRelocatingAway() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "70%").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build();
// We have an index with 2 primary shards each taking 40 bytes. Each node has 100 bytes available
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
@ -889,7 +889,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
)));
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
// Ensure that the reroute call doesn't alter the routing table, since the first primary is relocating away
@ -906,10 +906,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
public void testForSingleDataNode() {
Settings diskSettings = settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "70%").build();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), true)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "60%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%").build();
ImmutableOpenMap.Builder<String, DiskUsage> usagesBuilder = ImmutableOpenMap.builder();
usagesBuilder.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 100)); // 0% used
@ -989,7 +989,7 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", -1)
.build(), deciders, makeShardsAllocators(), cis);
RoutingAllocation.Result result = strategy.reroute(clusterState, "reroute");

View File

@ -28,12 +28,12 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
@ -45,7 +45,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
*/
public class DiskThresholdDeciderUnitTests extends ESTestCase {
public void testDynamicSettings() {
NodeSettingsService nss = new NodeSettingsService(Settings.EMPTY);
ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ClusterInfoService cis = EmptyClusterInfoService.INSTANCE;
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
@ -59,18 +59,15 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
assertTrue(decider.isEnabled());
assertTrue(decider.isIncludeRelocations());
DiskThresholdDecider.ApplySettings applySettings = decider.newApplySettings();
Settings newSettings = Settings.builder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, false)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, false)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "70%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, "500mb")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, "30s")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.getKey(), false)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.getKey(), false)
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), "70%")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), "500mb")
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "30s")
.build();
applySettings.onRefreshSettings(newSettings);
nss.applySettings(newSettings);
assertThat("high threshold bytes should be unset",
decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
assertThat("high threshold percentage should be changed",
@ -86,7 +83,7 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
}
public void testCanAllocateUsesMaxAvailableSpace() {
NodeSettingsService nss = new NodeSettingsService(Settings.EMPTY);
ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ClusterInfoService cis = EmptyClusterInfoService.INSTANCE;
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
@ -127,7 +124,7 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
}
public void testCanRemainUsesLeastAvailableSpace() {
NodeSettingsService nss = new NodeSettingsService(Settings.EMPTY);
ClusterSettings nss = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
ClusterInfoService cis = EmptyClusterInfoService.INSTANCE;
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
ImmutableOpenMap.Builder<ShardRouting, String> shardRoutingMap = ImmutableOpenMap.builder();

View File

@ -37,7 +37,7 @@ public class EnableAllocationDeciderIT extends ESIntegTestCase {
public void testEnableRebalance() throws InterruptedException {
final String firstNode = internalCluster().startNode();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE)).get();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE)).get();
// we test with 2 shards since otherwise it's pretty fragile if there are difference in the num or shards such that
// all shards are relocated to the second node which is not what we want here. It's solely a test for the settings to take effect
final int numShards = 2;
@ -64,7 +64,7 @@ public class EnableAllocationDeciderIT extends ESIntegTestCase {
assertThat("index: [test] expected to be rebalanced on both nodes", test.size(), equalTo(2));
// flip the cluster wide setting such that we can also balance for index test_1 eventually we should have one shard of each index on each node
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE, randomBoolean() ? EnableAllocationDecider.Rebalance.PRIMARIES : EnableAllocationDecider.Rebalance.ALL)).get();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), randomBoolean() ? EnableAllocationDecider.Rebalance.PRIMARIES : EnableAllocationDecider.Rebalance.ALL)).get();
logger.info("--> balance index [test_1]");
client().admin().cluster().prepareReroute().get();
ensureGreen("test_1");

View File

@ -22,7 +22,6 @@ package org.elasticsearch.cluster.routing.allocation.decider;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
@ -32,10 +31,10 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Allocation;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Rebalance;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESAllocationTestCase;
import java.util.EnumSet;
@ -44,8 +43,8 @@ import java.util.List;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING;
import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo;
@ -58,7 +57,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
public void testClusterEnableNone() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.NONE.name())
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.NONE.name())
.build());
logger.info("Building initial routing table");
@ -86,7 +85,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
public void testClusterEnableOnlyPrimaries() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE, Allocation.PRIMARIES.name())
.put(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), Allocation.PRIMARIES.name())
.build());
logger.info("Building initial routing table");
@ -159,11 +158,11 @@ public class EnableAllocationTests extends ESAllocationTestCase {
final boolean useClusterSetting = randomBoolean();
final Rebalance allowedOnes = RandomPicks.randomFrom(getRandom(), EnumSet.of(Rebalance.PRIMARIES, Rebalance.REPLICAS, Rebalance.ALL));
Settings build = settingsBuilder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE, useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 3)
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), 3)
.build();
NodeSettingsService nodeSettingsService = new NodeSettingsService(build);
AllocationService strategy = createAllocationService(build, nodeSettingsService, getRandom());
ClusterSettings clusterSettings = new ClusterSettings(build, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
AllocationService strategy = createAllocationService(build, clusterSettings, getRandom());
Settings indexSettings = useClusterSetting ? Settings.EMPTY : settingsBuilder().put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Rebalance.NONE).build();
logger.info("Building initial routing table");
@ -213,7 +212,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
if (useClusterSetting) {
prevState = clusterState;
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(settingsBuilder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE, allowedOnes)
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), allowedOnes)
.build())).build();
} else {
prevState = clusterState;
@ -224,7 +223,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
.build();
}
nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
clusterSettings.applySettings(clusterState.metaData().settings());
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat("expected 6 shards to be started 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(6));
@ -261,11 +260,11 @@ public class EnableAllocationTests extends ESAllocationTestCase {
public void testEnableClusterBalanceNoReplicas() {
final boolean useClusterSetting = randomBoolean();
Settings build = settingsBuilder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE, useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, 3)
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), useClusterSetting ? Rebalance.NONE: RandomPicks.randomFrom(getRandom(), Rebalance.values())) // index settings override cluster settings
.put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), 3)
.build();
NodeSettingsService nodeSettingsService = new NodeSettingsService(build);
AllocationService strategy = createAllocationService(build, nodeSettingsService, getRandom());
ClusterSettings clusterSettings = new ClusterSettings(build, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
AllocationService strategy = createAllocationService(build, clusterSettings, getRandom());
Settings indexSettings = useClusterSetting ? Settings.EMPTY : settingsBuilder().put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Rebalance.NONE).build();
logger.info("Building initial routing table");
@ -307,7 +306,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
if (useClusterSetting) {
prevState = clusterState;
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).transientSettings(settingsBuilder()
.put(CLUSTER_ROUTING_REBALANCE_ENABLE, randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL)
.put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL)
.build())).build();
} else {
prevState = clusterState;
@ -315,7 +314,7 @@ public class EnableAllocationTests extends ESAllocationTestCase {
clusterState = ClusterState.builder(clusterState).metaData(MetaData.builder(metaData).removeAllIndices()
.put(IndexMetaData.builder(meta).settings(settingsBuilder().put(meta.getSettings()).put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, randomBoolean() ? Rebalance.PRIMARIES : Rebalance.ALL).build()))).build();
}
nodeSettingsService.clusterChanged(new ClusterChangedEvent("foo", clusterState, prevState));
clusterSettings.applySettings(clusterState.metaData().settings());
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat("expected 4 primaries to be started and 2 to relocate useClusterSettings: " + useClusterSetting, clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4));

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.InternalClusterInfoService;
import org.elasticsearch.cluster.MockInternalClusterInfoService;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
@ -46,22 +47,12 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
public class MockDiskUsagesIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
// Update more frequently
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "1s")
.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
// Use the mock internal cluster info service, which has fake-able disk usages
return pluginList(MockInternalClusterInfoService.TestPlugin.class);
}
//@TestLogging("org.elasticsearch.cluster:TRACE,org.elasticsearch.cluster.routing.allocation.decider:TRACE")
public void testRerouteOccursOnDiskPassingHighWatermark() throws Exception {
List<String> nodes = internalCluster().startNodesAsync(3).get();
@ -77,15 +68,16 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
// Start with all nodes at 50% usage
final MockInternalClusterInfoService cis = (MockInternalClusterInfoService)
internalCluster().getInstance(ClusterInfoService.class, internalCluster().getMasterName());
cis.setUpdateFrequency(TimeValue.timeValueMillis(200));
cis.onMaster();
cis.setN1Usage(nodes.get(0), new DiskUsage(nodes.get(0), "n1", "/dev/null", 100, 50));
cis.setN2Usage(nodes.get(1), new DiskUsage(nodes.get(1), "n2", "/dev/null", 100, 50));
cis.setN3Usage(nodes.get(2), new DiskUsage(nodes.get(2), "n3", "/dev/null", 100, 50));
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, randomFrom("20b", "80%"))
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, randomFrom("10b", "90%"))
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, "1ms")).get();
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), randomFrom("20b", "80%"))
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), randomFrom("10b", "90%"))
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.getKey(), "1ms")).get();
// Create an index with 10 shards so we can check allocation for it
prepareCreate("test").setSettings(settingsBuilder()
.put("number_of_shards", 10)

View File

@ -19,6 +19,7 @@
package org.elasticsearch.cluster.settings;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequestBuilder;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.cluster.ClusterName;
@ -32,8 +33,8 @@ import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.hamcrest.Matchers;
import static org.elasticsearch.common.inject.matcher.Matchers.not;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.ESIntegTestCase.Scope.TEST;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -48,22 +49,142 @@ public class ClusterSettingsIT extends ESIntegTestCase {
public void testClusterNonExistingSettingsUpdate() {
String key1 = "no_idea_what_you_are_talking_about";
int value1 = 10;
ClusterUpdateSettingsResponse response = client().admin().cluster()
try {
client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(key1, value1).build())
.get();
fail("bogus value");
} catch (IllegalArgumentException ex) {
assertEquals(ex.getMessage(), "transient setting [no_idea_what_you_are_talking_about], not dynamically updateable");
}
}
public void testDeleteIsAppliedFirst() {
DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
assertEquals(discoverySettings.getPublishTimeout(), DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY));
assertTrue(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY));
ClusterUpdateSettingsResponse response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder()
.put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false)
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s").build())
.get();
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().entrySet(), Matchers.emptyIterable());
assertEquals(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), "1s");
assertTrue(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY));
assertFalse(response.getTransientSettings().getAsBoolean(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), null));
response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().putNull((randomBoolean() ? "discovery.zen.*" : "*")).put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "2s"))
.get();
assertEquals(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), "2s");
assertNull(response.getTransientSettings().getAsBoolean(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), null));
}
public void testResetClusterSetting() {
DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
ClusterUpdateSettingsResponse response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s").build())
.get();
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), equalTo("1s"));
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().putNull(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()))
.get();
assertAcked(response);
assertNull(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()));
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder()
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s")
.put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false).build())
.get();
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), equalTo("1s"));
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
assertFalse(discoverySettings.getPublishDiff());
response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().putNull((randomBoolean() ? "discovery.zen.*" : "*")))
.get();
assertNull(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()));
assertNull(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey()));
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
// now persistent
response = client().admin().cluster()
.prepareUpdateSettings()
.setPersistentSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s").build())
.get();
assertAcked(response);
assertThat(response.getPersistentSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), equalTo("1s"));
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
response = client().admin().cluster()
.prepareUpdateSettings()
.setPersistentSettings(Settings.builder().putNull((DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey())))
.get();
assertAcked(response);
assertNull(response.getPersistentSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()));
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
response = client().admin().cluster()
.prepareUpdateSettings()
.setPersistentSettings(Settings.builder()
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s")
.put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false).build())
.get();
assertAcked(response);
assertThat(response.getPersistentSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), equalTo("1s"));
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
assertFalse(discoverySettings.getPublishDiff());
response = client().admin().cluster()
.prepareUpdateSettings()
.setPersistentSettings(Settings.builder().putNull((randomBoolean() ? "discovery.zen.*" : "*")))
.get();
assertNull(response.getPersistentSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()));
assertNull(response.getPersistentSettings().getAsMap().get(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey()));
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
assertThat(discoverySettings.getPublishDiff(), equalTo(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.get(Settings.EMPTY)));
}
public void testClusterSettingsUpdateResponse() {
String key1 = IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC;
String key1 = IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC_SETTING.getKey();
int value1 = 10;
String key2 = EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE;
boolean value2 = false;
String key2 = EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey();
String value2 = EnableAllocationDecider.Allocation.NONE.name();
Settings transientSettings1 = Settings.builder().put(key1, value1, ByteSizeUnit.BYTES).build();
Settings persistentSettings1 = Settings.builder().put(key2, value2).build();
@ -114,43 +235,59 @@ public class ClusterSettingsIT extends ESIntegTestCase {
assertThat(response3.getPersistentSettings().get(key2), notNullValue());
}
public void testCanUpdateTracerSettings() {
ClusterUpdateSettingsResponse clusterUpdateSettingsResponse = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().putArray("transport.tracer.include", "internal:index/shard/recovery/*",
"internal:gateway/local*"))
.get();
assertArrayEquals(clusterUpdateSettingsResponse.getTransientSettings().getAsArray("transport.tracer.include"), new String[] {"internal:index/shard/recovery/*",
"internal:gateway/local*"});
}
public void testUpdateDiscoveryPublishTimeout() {
DiscoverySettings discoverySettings = internalCluster().getInstance(DiscoverySettings.class);
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.DEFAULT_PUBLISH_TIMEOUT));
assertThat(discoverySettings.getPublishTimeout(), equalTo(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.get(Settings.EMPTY)));
ClusterUpdateSettingsResponse response = client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "1s").build())
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s").build())
.get();
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT), equalTo("1s"));
assertThat(response.getTransientSettings().getAsMap().get(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey()), equalTo("1s"));
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
response = client().admin().cluster()
try {
client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, "whatever").build())
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "whatever").build())
.get();
fail("bogus value");
} catch (IllegalArgumentException ex) {
assertEquals(ex.getMessage(), "Failed to parse setting [discovery.zen.commit_timeout] with value [whatever] as a time value: unit is missing or unrecognized");
}
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().entrySet(), Matchers.emptyIterable());
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
response = client().admin().cluster()
try {
client().admin().cluster()
.prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT, -1).build())
.setTransientSettings(Settings.builder().put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), -1).build())
.get();
fail("bogus value");
} catch (IllegalArgumentException ex) {
assertEquals(ex.getMessage(), "Failed to parse value [-1] for setting [discovery.zen.publish_timeout] must be >= 0s");
}
assertAcked(response);
assertThat(response.getTransientSettings().getAsMap().entrySet(), Matchers.emptyIterable());
assertThat(discoverySettings.getPublishTimeout().seconds(), equalTo(1l));
}
public void testClusterUpdateSettingsWithBlocks() {
String key1 = "cluster.routing.allocation.enable";
Settings transientSettings = Settings.builder().put(key1, false).build();
Settings transientSettings = Settings.builder().put(key1, EnableAllocationDecider.Allocation.NONE.name()).build();
String key2 = "cluster.routing.allocation.node_concurrent_recoveries";
Settings persistentSettings = Settings.builder().put(key2, "5").build();
@ -165,7 +302,7 @@ public class ClusterSettingsIT extends ESIntegTestCase {
assertBlocked(request, MetaData.CLUSTER_READ_ONLY_BLOCK);
// But it's possible to update the settings to update the "cluster.blocks.read_only" setting
Settings settings = settingsBuilder().put(MetaData.SETTING_READ_ONLY, false).build();
Settings settings = settingsBuilder().put(MetaData.SETTING_READ_ONLY_SETTING.getKey(), false).build();
assertAcked(client().admin().cluster().prepareUpdateSettings().setTransientSettings(settings).get());
} finally {

View File

@ -225,7 +225,7 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
public void testAttributePreferenceRouting() {
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.awareness.attributes", "rack_id,zone")
.build());
@ -280,7 +280,7 @@ public class RoutingIteratorTests extends ESAllocationTestCase {
public void testNodeSelectorRouting(){
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.build());
MetaData metaData = MetaData.builder()

View File

@ -19,12 +19,12 @@
package org.elasticsearch.common.breaker;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.indices.breaker.BreakerSettings;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESTestCase;
import java.util.concurrent.atomic.AtomicBoolean;
@ -87,7 +87,7 @@ public class MemoryCircuitBreakerTests extends ESTestCase {
final AtomicReference<Throwable> lastException = new AtomicReference<>(null);
final AtomicReference<ChildMemoryCircuitBreaker> breakerRef = new AtomicReference<>(null);
final CircuitBreakerService service = new HierarchyCircuitBreakerService(Settings.EMPTY, new NodeSettingsService(Settings.EMPTY)) {
final CircuitBreakerService service = new HierarchyCircuitBreakerService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)) {
@Override
public CircuitBreaker getBreaker(String name) {
@ -147,7 +147,7 @@ public class MemoryCircuitBreakerTests extends ESTestCase {
final AtomicInteger parentTripped = new AtomicInteger(0);
final AtomicReference<ChildMemoryCircuitBreaker> breakerRef = new AtomicReference<>(null);
final CircuitBreakerService service = new HierarchyCircuitBreakerService(Settings.EMPTY, new NodeSettingsService(Settings.EMPTY)) {
final CircuitBreakerService service = new HierarchyCircuitBreakerService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)) {
@Override
public CircuitBreaker getBreaker(String name) {

View File

@ -0,0 +1,168 @@
/*
* 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.common.settings;
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
public class ScopedSettingsTests extends ESTestCase {
public void testAddConsumer() {
Setting<Integer> testSetting = Setting.intSetting("foo.bar", 1, true, Setting.Scope.CLUSTER);
Setting<Integer> testSetting2 = Setting.intSetting("foo.bar.baz", 1, true, Setting.Scope.CLUSTER);
AbstractScopedSettings service = new ClusterSettings(Settings.EMPTY, Collections.singleton(testSetting));
AtomicInteger consumer = new AtomicInteger();
service.addSettingsUpdateConsumer(testSetting, consumer::set);
AtomicInteger consumer2 = new AtomicInteger();
try {
service.addSettingsUpdateConsumer(testSetting2, consumer2::set);
fail("setting not registered");
} catch (IllegalArgumentException ex) {
assertEquals("Setting is not registered for key [foo.bar.baz]", ex.getMessage());
}
try {
service.addSettingsUpdateConsumer(testSetting, testSetting2, (a, b) -> {consumer.set(a); consumer2.set(b);});
fail("setting not registered");
} catch (IllegalArgumentException ex) {
assertEquals("Setting is not registered for key [foo.bar.baz]", ex.getMessage());
}
assertEquals(0, consumer.get());
assertEquals(0, consumer2.get());
service.applySettings(Settings.builder().put("foo.bar", 2).put("foo.bar.baz", 15).build());
assertEquals(2, consumer.get());
assertEquals(0, consumer2.get());
}
public void testApply() {
Setting<Integer> testSetting = Setting.intSetting("foo.bar", 1, true, Setting.Scope.CLUSTER);
Setting<Integer> testSetting2 = Setting.intSetting("foo.bar.baz", 1, true, Setting.Scope.CLUSTER);
AbstractScopedSettings service = new ClusterSettings(Settings.EMPTY, new HashSet<>(Arrays.asList(testSetting, testSetting2)));
AtomicInteger consumer = new AtomicInteger();
service.addSettingsUpdateConsumer(testSetting, consumer::set);
AtomicInteger consumer2 = new AtomicInteger();
service.addSettingsUpdateConsumer(testSetting2, consumer2::set, (s) -> assertTrue(s > 0));
AtomicInteger aC = new AtomicInteger();
AtomicInteger bC = new AtomicInteger();
service.addSettingsUpdateConsumer(testSetting, testSetting2, (a, b) -> {aC.set(a); bC.set(b);});
assertEquals(0, consumer.get());
assertEquals(0, consumer2.get());
assertEquals(0, aC.get());
assertEquals(0, bC.get());
try {
service.applySettings(Settings.builder().put("foo.bar", 2).put("foo.bar.baz", -15).build());
fail("invalid value");
} catch (IllegalArgumentException ex) {
assertEquals("illegal value can't update [foo.bar.baz] from [1] to [-15]", ex.getMessage());
}
assertEquals(0, consumer.get());
assertEquals(0, consumer2.get());
assertEquals(0, aC.get());
assertEquals(0, bC.get());
try {
service.dryRun(Settings.builder().put("foo.bar", 2).put("foo.bar.baz", -15).build());
fail("invalid value");
} catch (IllegalArgumentException ex) {
assertEquals("illegal value can't update [foo.bar.baz] from [1] to [-15]", ex.getMessage());
}
assertEquals(0, consumer.get());
assertEquals(0, consumer2.get());
assertEquals(0, aC.get());
assertEquals(0, bC.get());
service.dryRun(Settings.builder().put("foo.bar", 2).put("foo.bar.baz", 15).build());
assertEquals(0, consumer.get());
assertEquals(0, consumer2.get());
assertEquals(0, aC.get());
assertEquals(0, bC.get());
service.applySettings(Settings.builder().put("foo.bar", 2).put("foo.bar.baz", 15).build());
assertEquals(2, consumer.get());
assertEquals(15, consumer2.get());
assertEquals(2, aC.get());
assertEquals(15, bC.get());
}
public void testGet() {
ClusterSettings settings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
// group setting - complex matcher
Setting setting = settings.get("cluster.routing.allocation.require.value");
assertEquals(setting, FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP_SETTING);
setting = settings.get("cluster.routing.allocation.total_shards_per_node");
assertEquals(setting, ShardsLimitAllocationDecider.CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING);
// array settings - complex matcher
assertNotNull(settings.get("transport.tracer.include." + randomIntBetween(1, 100)));
assertSame(TransportService.TRACE_LOG_INCLUDE_SETTING, settings.get("transport.tracer.include." + randomIntBetween(1, 100)));
// array settings - complex matcher - only accepts numbers
assertNull(settings.get("transport.tracer.include.FOO"));
}
public void testIsDynamic(){
ClusterSettings settings = new ClusterSettings(Settings.EMPTY, new HashSet<>(Arrays.asList(Setting.intSetting("foo.bar", 1, true, Setting.Scope.CLUSTER), Setting.intSetting("foo.bar.baz", 1, false, Setting.Scope.CLUSTER))));
assertFalse(settings.hasDynamicSetting("foo.bar.baz"));
assertTrue(settings.hasDynamicSetting("foo.bar"));
assertNotNull(settings.get("foo.bar.baz"));
settings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
assertTrue(settings.hasDynamicSetting("transport.tracer.include." + randomIntBetween(1, 100)));
assertFalse(settings.hasDynamicSetting("transport.tracer.include.BOOM"));
assertTrue(settings.hasDynamicSetting("cluster.routing.allocation.require.value"));
}
public void testDiff() throws IOException {
Setting<Integer> foobarbaz = Setting.intSetting("foo.bar.baz", 1, false, Setting.Scope.CLUSTER);
Setting<Integer> foobar = Setting.intSetting("foo.bar", 1, true, Setting.Scope.CLUSTER);
ClusterSettings settings = new ClusterSettings(Settings.EMPTY, new HashSet<>(Arrays.asList(foobar, foobarbaz)));
Settings diff = settings.diff(Settings.builder().put("foo.bar", 5).build(), Settings.EMPTY);
assertEquals(diff.getAsMap().size(), 1);
assertEquals(diff.getAsInt("foo.bar.baz", null), Integer.valueOf(1));
diff = settings.diff(Settings.builder().put("foo.bar", 5).build(), Settings.builder().put("foo.bar.baz", 17).build());
assertEquals(diff.getAsMap().size(), 1);
assertEquals(diff.getAsInt("foo.bar.baz", null), Integer.valueOf(17));
}
public void testUpdateTracer() {
ClusterSettings settings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
AtomicReference<List<String>> ref = new AtomicReference<>();
settings.addSettingsUpdateConsumer(TransportService.TRACE_LOG_INCLUDE_SETTING, ref::set);
settings.applySettings(Settings.builder().putArray("transport.tracer.include", "internal:index/shard/recovery/*", "internal:gateway/local*").build());
assertNotNull(ref.get().size());
assertEquals(ref.get().size(), 2);
assertTrue(ref.get().contains("internal:index/shard/recovery/*"));
assertTrue(ref.get().contains("internal:gateway/local*"));
}
}

View File

@ -0,0 +1,323 @@
/*
* 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.common.settings;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicReference;
public class SettingTests extends ESTestCase {
public void testGet() {
Setting<Boolean> booleanSetting = Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER);
assertFalse(booleanSetting.get(Settings.EMPTY));
assertFalse(booleanSetting.get(Settings.builder().put("foo.bar", false).build()));
assertTrue(booleanSetting.get(Settings.builder().put("foo.bar", true).build()));
}
public void testByteSize() {
Setting<ByteSizeValue> byteSizeValueSetting = Setting.byteSizeSetting("a.byte.size", new ByteSizeValue(1024), true, Setting.Scope.CLUSTER);
assertFalse(byteSizeValueSetting.isGroupSetting());
ByteSizeValue byteSizeValue = byteSizeValueSetting.get(Settings.EMPTY);
assertEquals(byteSizeValue.bytes(), 1024);
AtomicReference<ByteSizeValue> value = new AtomicReference<>(null);
ClusterSettings.SettingUpdater settingUpdater = byteSizeValueSetting.newUpdater(value::set, logger);
try {
settingUpdater.apply(Settings.builder().put("a.byte.size", 12).build(), Settings.EMPTY);
fail("no unit");
} catch (IllegalArgumentException ex) {
assertEquals("failed to parse setting [a.byte.size] with value [12] as a size in bytes: unit is missing or unrecognized", ex.getMessage());
}
assertTrue(settingUpdater.apply(Settings.builder().put("a.byte.size", "12b").build(), Settings.EMPTY));
assertEquals(new ByteSizeValue(12), value.get());
}
public void testSimpleUpdate() {
Setting<Boolean> booleanSetting = Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER);
AtomicReference<Boolean> atomicBoolean = new AtomicReference<>(null);
ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(atomicBoolean::set, logger);
Settings build = Settings.builder().put("foo.bar", false).build();
settingUpdater.apply(build, Settings.EMPTY);
assertNull(atomicBoolean.get());
build = Settings.builder().put("foo.bar", true).build();
settingUpdater.apply(build, Settings.EMPTY);
assertTrue(atomicBoolean.get());
// try update bogus value
build = Settings.builder().put("foo.bar", "I am not a boolean").build();
try {
settingUpdater.apply(build, Settings.EMPTY);
fail("not a boolean");
} catch (IllegalArgumentException ex) {
assertEquals("Failed to parse value [I am not a boolean] cannot be parsed to boolean [ true/1/on/yes OR false/0/off/no ]", ex.getMessage());
}
}
public void testUpdateNotDynamic() {
Setting<Boolean> booleanSetting = Setting.boolSetting("foo.bar", false, false, Setting.Scope.CLUSTER);
assertFalse(booleanSetting.isGroupSetting());
AtomicReference<Boolean> atomicBoolean = new AtomicReference<>(null);
try {
booleanSetting.newUpdater(atomicBoolean::set, logger);
fail("not dynamic");
} catch (IllegalStateException ex) {
assertEquals("setting [foo.bar] is not dynamic", ex.getMessage());
}
}
public void testUpdaterIsIsolated() {
Setting<Boolean> booleanSetting = Setting.boolSetting("foo.bar", false, true, Setting.Scope.CLUSTER);
AtomicReference<Boolean> ab1 = new AtomicReference<>(null);
AtomicReference<Boolean> ab2 = new AtomicReference<>(null);
ClusterSettings.SettingUpdater settingUpdater = booleanSetting.newUpdater(ab1::set, logger);
ClusterSettings.SettingUpdater settingUpdater2 = booleanSetting.newUpdater(ab2::set, logger);
settingUpdater.apply(Settings.builder().put("foo.bar", true).build(), Settings.EMPTY);
assertTrue(ab1.get());
assertNull(ab2.get());
}
public void testDefault() {
TimeValue defautlValue = TimeValue.timeValueMillis(randomIntBetween(0, 1000000));
Setting<TimeValue> setting = Setting.positiveTimeSetting("my.time.value", defautlValue, randomBoolean(), Setting.Scope.CLUSTER);
assertFalse(setting.isGroupSetting());
String aDefault = setting.getDefault(Settings.EMPTY);
assertEquals(defautlValue.millis() + "ms", aDefault);
assertEquals(defautlValue.millis(), setting.get(Settings.EMPTY).millis());
Setting<String> secondaryDefault = new Setting<>("foo.bar", (s) -> s.get("old.foo.bar", "some_default"), (s) -> s, randomBoolean(), Setting.Scope.CLUSTER);
assertEquals("some_default", secondaryDefault.get(Settings.EMPTY));
assertEquals("42", secondaryDefault.get(Settings.builder().put("old.foo.bar", 42).build()));
}
public void testComplexType() {
AtomicReference<ComplexType> ref = new AtomicReference<>(null);
Setting<ComplexType> setting = new Setting<>("foo.bar", (s) -> "", (s) -> new ComplexType(s), true, Setting.Scope.CLUSTER);
assertFalse(setting.isGroupSetting());
ref.set(setting.get(Settings.EMPTY));
ComplexType type = ref.get();
ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger);
assertFalse(settingUpdater.apply(Settings.EMPTY, Settings.EMPTY));
assertSame("no update - type has not changed", type, ref.get());
// change from default
assertTrue(settingUpdater.apply(Settings.builder().put("foo.bar", "2").build(), Settings.EMPTY));
assertNotSame("update - type has changed", type, ref.get());
assertEquals("2", ref.get().foo);
// change back to default...
assertTrue(settingUpdater.apply(Settings.EMPTY, Settings.builder().put("foo.bar", "2").build()));
assertNotSame("update - type has changed", type, ref.get());
assertEquals("", ref.get().foo);
}
public void testType() {
Setting<Integer> integerSetting = Setting.intSetting("foo.int.bar", 1, true, Setting.Scope.CLUSTER);
assertEquals(integerSetting.getScope(), Setting.Scope.CLUSTER);
integerSetting = Setting.intSetting("foo.int.bar", 1, true, Setting.Scope.INDEX);
assertEquals(integerSetting.getScope(), Setting.Scope.INDEX);
}
public void testGroups() {
AtomicReference<Settings> ref = new AtomicReference<>(null);
Setting<Settings> setting = Setting.groupSetting("foo.bar.", true, Setting.Scope.CLUSTER);
assertTrue(setting.isGroupSetting());
ClusterSettings.SettingUpdater settingUpdater = setting.newUpdater(ref::set, logger);
Settings currentInput = Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").put("foo.bar.3.value", "3").build();
Settings previousInput = Settings.EMPTY;
assertTrue(settingUpdater.apply(currentInput, previousInput));
assertNotNull(ref.get());
Settings settings = ref.get();
Map<String, Settings> asMap = settings.getAsGroups();
assertEquals(3, asMap.size());
assertEquals(asMap.get("1").get("value"), "1");
assertEquals(asMap.get("2").get("value"), "2");
assertEquals(asMap.get("3").get("value"), "3");
previousInput = currentInput;
currentInput = Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").put("foo.bar.3.value", "3").build();
Settings current = ref.get();
assertFalse(settingUpdater.apply(currentInput, previousInput));
assertSame(current, ref.get());
previousInput = currentInput;
currentInput = Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").build();
// now update and check that we got it
assertTrue(settingUpdater.apply(currentInput, previousInput));
assertNotSame(current, ref.get());
asMap = ref.get().getAsGroups();
assertEquals(2, asMap.size());
assertEquals(asMap.get("1").get("value"), "1");
assertEquals(asMap.get("2").get("value"), "2");
previousInput = currentInput;
currentInput = Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "4").build();
// now update and check that we got it
assertTrue(settingUpdater.apply(currentInput, previousInput));
assertNotSame(current, ref.get());
asMap = ref.get().getAsGroups();
assertEquals(2, asMap.size());
assertEquals(asMap.get("1").get("value"), "1");
assertEquals(asMap.get("2").get("value"), "4");
assertTrue(setting.match("foo.bar.baz"));
assertFalse(setting.match("foo.baz.bar"));
ClusterSettings.SettingUpdater predicateSettingUpdater = setting.newUpdater(ref::set, logger,(s) -> assertFalse(true));
try {
predicateSettingUpdater.apply(Settings.builder().put("foo.bar.1.value", "1").put("foo.bar.2.value", "2").build(), Settings.EMPTY);
fail("not accepted");
} catch (IllegalArgumentException ex) {
assertEquals(ex.getMessage(), "illegal value can't update [foo.bar.] from [{}] to [{1.value=1, 2.value=2}]");
}
}
public static class ComplexType {
final String foo;
public ComplexType(String foo) {
this.foo = foo;
}
}
public static class Composite {
private Integer b;
private Integer a;
public void set(Integer a, Integer b) {
this.a = a;
this.b = b;
}
}
public void testComposite() {
Composite c = new Composite();
Setting<Integer> a = Setting.intSetting("foo.int.bar.a", 1, true, Setting.Scope.CLUSTER);
Setting<Integer> b = Setting.intSetting("foo.int.bar.b", 1, true, Setting.Scope.CLUSTER);
ClusterSettings.SettingUpdater<Tuple<Integer, Integer>> settingUpdater = Setting.compoundUpdater(c::set, a, b, logger);
assertFalse(settingUpdater.apply(Settings.EMPTY, Settings.EMPTY));
assertNull(c.a);
assertNull(c.b);
Settings build = Settings.builder().put("foo.int.bar.a", 2).build();
assertTrue(settingUpdater.apply(build, Settings.EMPTY));
assertEquals(2, c.a.intValue());
assertEquals(1, c.b.intValue());
Integer aValue = c.a;
assertFalse(settingUpdater.apply(build, build));
assertSame(aValue, c.a);
Settings previous = build;
build = Settings.builder().put("foo.int.bar.a", 2).put("foo.int.bar.b", 5).build();
assertTrue(settingUpdater.apply(build, previous));
assertEquals(2, c.a.intValue());
assertEquals(5, c.b.intValue());
// reset to default
assertTrue(settingUpdater.apply(Settings.EMPTY, build));
assertEquals(1, c.a.intValue());
assertEquals(1, c.b.intValue());
}
public void testListSettings() {
Setting<List<String>> listSetting = Setting.listSetting("foo.bar", Arrays.asList("foo,bar"), (s) -> s.toString(), true, Setting.Scope.CLUSTER);
List<String> value = listSetting.get(Settings.EMPTY);
assertEquals(1, value.size());
assertEquals("foo,bar", value.get(0));
List<String> input = Arrays.asList("test", "test1, test2", "test", ",,,,");
Settings.Builder builder = Settings.builder().putArray("foo.bar", input.toArray(new String[0]));
value = listSetting.get(builder.build());
assertEquals(input.size(), value.size());
assertArrayEquals(value.toArray(new String[0]), input.toArray(new String[0]));
// try to parse this really annoying format
builder = Settings.builder();
for (int i = 0; i < input.size(); i++) {
builder.put("foo.bar." + i, input.get(i));
}
value = listSetting.get(builder.build());
assertEquals(input.size(), value.size());
assertArrayEquals(value.toArray(new String[0]), input.toArray(new String[0]));
AtomicReference<List<String>> ref = new AtomicReference<>();
AbstractScopedSettings.SettingUpdater settingUpdater = listSetting.newUpdater(ref::set, logger);
assertTrue(settingUpdater.hasChanged(builder.build(), Settings.EMPTY));
settingUpdater.apply(builder.build(), Settings.EMPTY);
assertEquals(input.size(), ref.get().size());
assertArrayEquals(ref.get().toArray(new String[0]), input.toArray(new String[0]));
settingUpdater.apply(Settings.builder().putArray("foo.bar", "123").build(), builder.build());
assertEquals(1, ref.get().size());
assertArrayEquals(ref.get().toArray(new String[0]), new String[] {"123"});
settingUpdater.apply(Settings.builder().put("foo.bar", "1,2,3").build(), Settings.builder().putArray("foo.bar", "123").build());
assertEquals(3, ref.get().size());
assertArrayEquals(ref.get().toArray(new String[0]), new String[] {"1", "2", "3"});
settingUpdater.apply(Settings.EMPTY, Settings.builder().put("foo.bar", "1,2,3").build());
assertEquals(1, ref.get().size());
assertEquals("foo,bar", ref.get().get(0));
Setting<List<Integer>> otherSettings = Setting.listSetting("foo.bar", Collections.emptyList(), Integer::parseInt, true, Setting.Scope.CLUSTER);
List<Integer> defaultValue = otherSettings.get(Settings.EMPTY);
assertEquals(0, defaultValue.size());
List<Integer> intValues = otherSettings.get(Settings.builder().put("foo.bar", "0,1,2,3").build());
assertEquals(4, intValues.size());
for (int i = 0; i < intValues.size(); i++) {
assertEquals(i, intValues.get(i).intValue());
}
}
public void testListSettingAcceptsNumberSyntax() {
Setting<List<String>> listSetting = Setting.listSetting("foo.bar", Arrays.asList("foo,bar"), (s) -> s.toString(), true, Setting.Scope.CLUSTER);
List<String> input = Arrays.asList("test", "test1, test2", "test", ",,,,");
Settings.Builder builder = Settings.builder().putArray("foo.bar", input.toArray(new String[0]));
// try to parse this really annoying format
for (String key : builder.internalMap().keySet()) {
assertTrue("key: " + key + " doesn't match", listSetting.match(key));
}
builder = Settings.builder().put("foo.bar", "1,2,3");
for (String key : builder.internalMap().keySet()) {
assertTrue("key: " + key + " doesn't match", listSetting.match(key));
}
assertFalse(listSetting.match("foo_bar"));
assertFalse(listSetting.match("foo_bar.1"));
assertTrue(listSetting.match("foo.bar"));
assertTrue(listSetting.match("foo.bar." + randomIntBetween(0,10000)));
}
}

View File

@ -162,4 +162,14 @@ public class TimeValueTests extends ESTestCase {
assertThat(e.getMessage(), containsString("Failed to parse"));
}
}
public void testToStringRep() {
assertEquals("-1", new TimeValue(-1).getStringRep());
assertEquals("10ms", new TimeValue(10, TimeUnit.MILLISECONDS).getStringRep());
assertEquals("1533ms", new TimeValue(1533, TimeUnit.MILLISECONDS).getStringRep());
assertEquals("90s", new TimeValue(90, TimeUnit.SECONDS).getStringRep());
assertEquals("90m", new TimeValue(90, TimeUnit.MINUTES).getStringRep());
assertEquals("36h", new TimeValue(36, TimeUnit.HOURS).getStringRep());
assertEquals("1000d", new TimeValue(1000, TimeUnit.DAYS).getStringRep());
}
}

View File

@ -21,13 +21,13 @@ package org.elasticsearch.common.util;
import org.apache.lucene.util.BytesRef;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.breaker.CircuitBreaker;
import org.elasticsearch.common.breaker.CircuitBreakingException;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.junit.Before;
@ -336,9 +336,9 @@ public class BigArraysTests extends ESSingleNodeTestCase {
for (String type : Arrays.asList("Byte", "Int", "Long", "Float", "Double", "Object")) {
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
Settings.builder()
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, size - 1, ByteSizeUnit.BYTES)
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), size - 1, ByteSizeUnit.BYTES)
.build(),
new NodeSettingsService(Settings.EMPTY));
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
try {
@ -356,9 +356,9 @@ public class BigArraysTests extends ESSingleNodeTestCase {
final long maxSize = randomIntBetween(1 << 10, 1 << 22);
HierarchyCircuitBreakerService hcbs = new HierarchyCircuitBreakerService(
Settings.builder()
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, maxSize, ByteSizeUnit.BYTES)
.put(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), maxSize, ByteSizeUnit.BYTES)
.build(),
new NodeSettingsService(Settings.EMPTY));
new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
BigArrays bigArrays = new BigArrays(null, hcbs).withCircuitBreaking();
Method create = BigArrays.class.getMethod("new" + type + "Array", long.class);
final int size = scaledRandomIntBetween(1, 20);

View File

@ -132,7 +132,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
.put(FaultDetection.SETTING_PING_TIMEOUT, "1s") // for hitting simulated network failures quickly
.put(FaultDetection.SETTING_PING_RETRIES, "1") // for hitting simulated network failures quickly
.put("discovery.zen.join_timeout", "10s") // still long to induce failures but to long so test won't time out
.put(DiscoverySettings.PUBLISH_TIMEOUT, "1s") // <-- for hitting simulated network failures quickly
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "1s") // <-- for hitting simulated network failures quickly
.put("http.enabled", false) // just to make test quicker
.put("gateway.local.list_timeout", "10s") // still long to induce failures but to long so test won't time out
.build();
@ -150,7 +150,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
// TODO: Rarely use default settings form some of these
Settings nodeSettings = Settings.builder()
.put(DEFAULT_SETTINGS)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, minimumMasterNode)
.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minimumMasterNode)
.build();
if (discoveryConfig == null) {
@ -217,7 +217,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
logger.info("--> reducing min master nodes to 2");
assertAcked(client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2)).get());
.setTransientSettings(Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)).get());
String master = internalCluster().getMasterName();
String nonMaster = null;
@ -293,9 +293,9 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
// Wait until the master node sees al 3 nodes again.
ensureStableCluster(3, new TimeValue(DISRUPTION_HEALING_OVERHEAD.millis() + networkPartition.expectedTimeToHeal().millis()));
logger.info("Verify no master block with {} set to {}", DiscoverySettings.NO_MASTER_BLOCK, "all");
logger.info("Verify no master block with {} set to {}", DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all");
client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(DiscoverySettings.NO_MASTER_BLOCK, "all"))
.setTransientSettings(Settings.builder().put(DiscoverySettings.NO_MASTER_BLOCK_SETTING.getKey(), "all"))
.get();
networkPartition.startDisrupting();
@ -863,7 +863,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
internalCluster().startNodesAsync(3,
Settings.builder()
.put(DiscoveryService.SETTING_INITIAL_STATE_TIMEOUT, "1ms")
.put(DiscoverySettings.PUBLISH_TIMEOUT, "3s")
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "3s")
.build()).get();
logger.info("applying disruption while cluster is forming ...");

View File

@ -30,6 +30,7 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.LocalTransportAddress;
@ -38,7 +39,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.BaseFuture;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.membership.MembershipAction;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cluster.TestClusterService;
import org.elasticsearch.test.junit.annotations.TestLogging;
@ -67,7 +67,7 @@ public class NodeJoinControllerTests extends ESTestCase {
// make sure we have a master
clusterService.setState(ClusterState.builder(clusterService.state()).nodes(DiscoveryNodes.builder(initialNodes).masterNodeId(localNode.id())));
nodeJoinController = new NodeJoinController(clusterService, new NoopRoutingService(Settings.EMPTY),
new DiscoverySettings(Settings.EMPTY, new NodeSettingsService(Settings.EMPTY)), Settings.EMPTY);
new DiscoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), Settings.EMPTY);
}
public void testSimpleJoinAccumulation() throws InterruptedException, ExecutionException {

View File

@ -84,7 +84,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
assertThat(zenDiscovery.isRejoinOnMasterGone(), is(true));
client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder().put(ZenDiscovery.SETTING_REJOIN_ON_MASTER_GONE, false))
.setTransientSettings(Settings.builder().put(ZenDiscovery.REJOIN_ON_MASTER_GONE_SETTING.getKey(), false))
.get();
assertThat(zenDiscovery.isRejoinOnMasterGone(), is(false));

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.common.settings.ClusterSettings;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Randomness;
import org.elasticsearch.common.collect.ImmutableOpenMap;
@ -39,7 +40,6 @@ import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.DiscoveryNodesProvider;
import org.elasticsearch.node.service.NodeService;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService;
@ -138,7 +138,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
public MockNode createMockNode(String name, Settings settings, Version version, @Nullable ClusterStateListener listener) throws Exception {
settings = Settings.builder()
.put("name", name)
.put(TransportService.SETTING_TRACE_LOG_INCLUDE, "", TransportService.SETTING_TRACE_LOG_EXCLUDE, "NOTHING")
.put(TransportService.TRACE_LOG_INCLUDE_SETTING.getKey(), "", TransportService.TRACE_LOG_EXCLUDE_SETTING.getKey(), "NOTHING")
.put(settings)
.build();
@ -219,7 +219,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
protected MockPublishAction buildPublishClusterStateAction(Settings settings, MockTransportService transportService, DiscoveryNodesProvider nodesProvider,
PublishClusterStateAction.NewPendingClusterStateListener listener) {
DiscoverySettings discoverySettings = new DiscoverySettings(settings, new NodeSettingsService(settings));
DiscoverySettings discoverySettings = new DiscoverySettings(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
return new MockPublishAction(settings, transportService, nodesProvider, listener, discoverySettings, ClusterName.DEFAULT);
}
@ -327,7 +327,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
}
public void testDisablingDiffPublishing() throws Exception {
Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, false).build();
Settings noDiffPublishingSettings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), false).build();
MockNode nodeA = createMockNode("nodeA", noDiffPublishingSettings, Version.CURRENT, new ClusterStateListener() {
@Override
@ -366,7 +366,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
public void testSimultaneousClusterStatePublishing() throws Exception {
int numberOfNodes = randomIntBetween(2, 10);
int numberOfIterations = scaledRandomIntBetween(5, 50);
Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE, randomBoolean()).build();
Settings settings = Settings.builder().put(DiscoverySettings.PUBLISH_DIFF_ENABLE_SETTING.getKey(), randomBoolean()).build();
MockNode master = createMockNode("node0", settings, Version.CURRENT, new ClusterStateListener() {
@Override
public void clusterChanged(ClusterChangedEvent event) {
@ -492,8 +492,8 @@ public class PublishClusterStateActionTests extends ESTestCase {
final boolean expectingToCommit = randomBoolean();
Settings.Builder settings = Settings.builder();
// make sure we have a reasonable timeout if we expect to timeout, o.w. one that will make the test "hang"
settings.put(DiscoverySettings.COMMIT_TIMEOUT, expectingToCommit == false && timeOutNodes > 0 ? "100ms" : "1h")
.put(DiscoverySettings.PUBLISH_TIMEOUT, "5ms"); // test is about committing
settings.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), expectingToCommit == false && timeOutNodes > 0 ? "100ms" : "1h")
.put(DiscoverySettings.PUBLISH_TIMEOUT_SETTING.getKey(), "5ms"); // test is about committing
MockNode master = createMockNode("master", settings.build());
@ -677,7 +677,7 @@ public class PublishClusterStateActionTests extends ESTestCase {
*/
public void testTimeoutOrCommit() throws Exception {
Settings settings = Settings.builder()
.put(DiscoverySettings.COMMIT_TIMEOUT, "1ms").build(); // short but so we will sometime commit sometime timeout
.put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "1ms").build(); // short but so we will sometime commit sometime timeout
MockNode master = createMockNode("master", settings);
MockNode node = createMockNode("node", settings);

View File

@ -57,7 +57,7 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
//ridiculous settings to make sure we don't run into uninitialized because fo default
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 100)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", 100)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 100)
.build());
@ -111,7 +111,7 @@ public class GatewayMetaStateTests extends ESAllocationTestCase {
//ridiculous settings to make sure we don't run into uninitialized because fo default
AllocationService strategy = createAllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 100)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), "always")
.put("cluster.routing.allocation.cluster_concurrent_rebalance", 100)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 100)
.build());

View File

@ -51,7 +51,7 @@ public class QuorumGatewayIT extends ESIntegTestCase {
logger.info("--> starting 3 nodes");
// we are shutting down nodes - make sure we don't have 2 clusters if we test network
internalCluster().startNodesAsync(3,
Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, 2).build()).get();
Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2).build()).get();
createIndex("test");

View File

@ -82,9 +82,9 @@ public class RecoveryBackwardsCompatibilityIT extends ESBackcompatTestCase {
SearchResponse countResponse = client().prepareSearch().setSize(0).get();
assertHitCount(countResponse, numDocs);
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "none")).execute().actionGet();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "none")).execute().actionGet();
backwardsCluster().upgradeAllNodes();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, "all")).execute().actionGet();
client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.settingsBuilder().put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), "all")).execute().actionGet();
ensureGreen();
countResponse = client().prepareSearch().setSize(0).get();

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