Merge pull request #12916 from rjernst/module_culling
Flatten ClusterModule and add more tests
This commit is contained in:
commit
9974b79c8a
|
@ -183,7 +183,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
|
|||
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());
|
||||
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());
|
||||
|
@ -200,7 +200,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct
|
|||
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());
|
||||
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());
|
||||
|
|
|
@ -19,59 +19,301 @@
|
|||
|
||||
package org.elasticsearch.cluster;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import org.elasticsearch.action.support.DestructiveOperations;
|
||||
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
|
||||
import org.elasticsearch.cluster.action.index.NodeIndexDeletedAction;
|
||||
import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction;
|
||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||
import org.elasticsearch.cluster.metadata.*;
|
||||
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;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataIndexTemplateService;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataMappingService;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataService;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodeService;
|
||||
import org.elasticsearch.cluster.routing.OperationRouting;
|
||||
import org.elasticsearch.cluster.routing.RoutingService;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationModule;
|
||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.RebalanceOnlyWhenActiveAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
|
||||
import org.elasticsearch.cluster.service.InternalClusterService;
|
||||
import org.elasticsearch.cluster.settings.ClusterDynamicSettingsModule;
|
||||
import org.elasticsearch.common.Classes;
|
||||
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
|
||||
import org.elasticsearch.cluster.settings.DynamicSettings;
|
||||
import org.elasticsearch.cluster.settings.Validator;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.Module;
|
||||
import org.elasticsearch.common.inject.SpawnModules;
|
||||
import org.elasticsearch.common.inject.multibindings.Multibinder;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.settings.IndexDynamicSettingsModule;
|
||||
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;
|
||||
import org.elasticsearch.index.indexing.IndexingSlowLog;
|
||||
import org.elasticsearch.index.search.stats.SearchSlowLog;
|
||||
import org.elasticsearch.index.settings.IndexDynamicSettings;
|
||||
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.translog.TranslogConfig;
|
||||
import org.elasticsearch.index.translog.TranslogService;
|
||||
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.store.IndicesStore;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*
|
||||
* Configures classes and services that affect the entire cluster.
|
||||
*/
|
||||
public class ClusterModule extends AbstractModule implements SpawnModules {
|
||||
public class ClusterModule extends AbstractModule {
|
||||
|
||||
public static final String EVEN_SHARD_COUNT_ALLOCATOR = "even_shard";
|
||||
public static final String BALANCED_ALLOCATOR = "balanced"; // default
|
||||
public static final String SHARDS_ALLOCATOR_TYPE_KEY = "cluster.routing.allocation.type";
|
||||
public static final List<Class<? extends AllocationDecider>> DEFAULT_ALLOCATION_DECIDERS =
|
||||
Collections.unmodifiableList(Arrays.asList(
|
||||
SameShardAllocationDecider.class,
|
||||
FilterAllocationDecider.class,
|
||||
ReplicaAfterPrimaryActiveAllocationDecider.class,
|
||||
ThrottlingAllocationDecider.class,
|
||||
RebalanceOnlyWhenActiveAllocationDecider.class,
|
||||
ClusterRebalanceAllocationDecider.class,
|
||||
ConcurrentRebalanceAllocationDecider.class,
|
||||
EnableAllocationDecider.class, // new enable allocation logic should proceed old disable allocation logic
|
||||
DisableAllocationDecider.class,
|
||||
AwarenessAllocationDecider.class,
|
||||
ShardsLimitAllocationDecider.class,
|
||||
NodeVersionAllocationDecider.class,
|
||||
DiskThresholdDecider.class,
|
||||
SnapshotInProgressAllocationDecider.class));
|
||||
|
||||
private final Settings settings;
|
||||
public static final String CLUSTER_SERVICE_IMPL = "cluster.info.service.type";
|
||||
|
||||
private Set<Class<? extends IndexTemplateFilter>> indexTemplateFilters = new HashSet<>();
|
||||
private final DynamicSettings.Builder clusterDynamicSettings = new DynamicSettings.Builder();
|
||||
private final DynamicSettings.Builder indexDynamicSettings = new DynamicSettings.Builder();
|
||||
private final ExtensionPoint.TypeExtensionPoint<ShardsAllocator> shardsAllocators = new ExtensionPoint.TypeExtensionPoint<>("shards_allocator", ShardsAllocator.class);
|
||||
private final ExtensionPoint.SetExtensionPoint<AllocationDecider> allocationDeciders = new ExtensionPoint.SetExtensionPoint<>("allocation_decider", AllocationDecider.class, AllocationDeciders.class);
|
||||
private final ExtensionPoint.SetExtensionPoint<IndexTemplateFilter> indexTemplateFilters = new ExtensionPoint.SetExtensionPoint<>("index_template_filter", IndexTemplateFilter.class);
|
||||
|
||||
// pkg private so tests can mock
|
||||
Class<? extends ClusterInfoService> clusterInfoServiceImpl = InternalClusterInfoService.class;
|
||||
|
||||
public ClusterModule(Settings settings) {
|
||||
this.settings = settings;
|
||||
|
||||
registerBuiltinClusterSettings();
|
||||
registerBuiltinIndexSettings();
|
||||
|
||||
for (Class<? extends AllocationDecider> decider : ClusterModule.DEFAULT_ALLOCATION_DECIDERS) {
|
||||
registerAllocationDecider(decider);
|
||||
}
|
||||
registerShardsAllocator(ClusterModule.BALANCED_ALLOCATOR, BalancedShardsAllocator.class);
|
||||
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(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, Validator.EMPTY);
|
||||
registerClusterDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION, Validator.EMPTY);
|
||||
registerClusterDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, 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(IndicesStore.INDICES_STORE_THROTTLE_TYPE, Validator.EMPTY);
|
||||
registerClusterDynamicSetting(IndicesStore.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_FILE_CHUNK_SIZE, Validator.BYTES_SIZE);
|
||||
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_OPS, Validator.INTEGER);
|
||||
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, Validator.BYTES_SIZE);
|
||||
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_COMPRESS, 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(RecoverySettings.INDICES_RECOVERY_MAX_SIZE_PER_SEC, Validator.BYTES_SIZE);
|
||||
registerClusterDynamicSetting(ThreadPool.THREADPOOL_GROUP + "*", Validator.EMPTY);
|
||||
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(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);
|
||||
}
|
||||
|
||||
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_MERGE_COUNT, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(MergeSchedulerConfig.AUTO_THROTTLE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_REQUIRE_GROUP + "*", Validator.EMPTY);
|
||||
registerIndexDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "*", Validator.EMPTY);
|
||||
registerIndexDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_EXCLUDE_GROUP + "*", Validator.EMPTY);
|
||||
registerIndexDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, Validator.NON_NEGATIVE_INTEGER);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_READ_ONLY, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_BLOCKS_READ, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_BLOCKS_WRITE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_BLOCKS_METADATA, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexMetaData.SETTING_PRIORITY, Validator.NON_NEGATIVE_INTEGER);
|
||||
registerIndexDynamicSetting(IndicesTTLService.INDEX_TTL_DISABLE_PURGE, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexShard.INDEX_REFRESH_INTERVAL, Validator.TIME);
|
||||
registerIndexDynamicSetting(PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN);
|
||||
registerIndexDynamicSetting(EngineConfig.INDEX_GC_DELETES_SETTING, Validator.TIME);
|
||||
registerIndexDynamicSetting(IndexShard.INDEX_FLUSH_ON_CLOSE, Validator.BOOLEAN);
|
||||
registerIndexDynamicSetting(EngineConfig.INDEX_VERSION_MAP_SIZE, Validator.BYTES_SIZE_OR_PERCENTAGE);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN, Validator.TIME);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO, Validator.TIME);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG, Validator.TIME);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE, Validator.TIME);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE, Validator.TIME);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_REFORMAT, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE, Validator.INTEGER);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED, Validator.DOUBLE);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT, Validator.BYTES_SIZE);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE, Validator.INTEGER_GTE_2);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT, Validator.INTEGER_GTE_2);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT, Validator.BYTES_SIZE);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER, Validator.DOUBLE_GTE_2);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT, Validator.NON_NEGATIVE_DOUBLE);
|
||||
registerIndexDynamicSetting(MergePolicyConfig.INDEX_COMPOUND_FORMAT, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_INTERVAL, Validator.TIME);
|
||||
registerIndexDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, Validator.INTEGER);
|
||||
registerIndexDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, Validator.BYTES_SIZE);
|
||||
registerIndexDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_PERIOD, Validator.TIME);
|
||||
registerIndexDynamicSetting(TranslogService.INDEX_TRANSLOG_DISABLE_FLUSH, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(TranslogConfig.INDEX_TRANSLOG_DURABILITY, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndicesWarmer.INDEX_WARMER_ENABLED, Validator.EMPTY);
|
||||
registerIndexDynamicSetting(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED, Validator.BOOLEAN);
|
||||
registerIndexDynamicSetting(IndicesRequestCache.DEPRECATED_INDEX_CACHE_REQUEST_ENABLED, Validator.BOOLEAN);
|
||||
registerIndexDynamicSetting(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, Validator.TIME);
|
||||
}
|
||||
|
||||
public void registerIndexDynamicSetting(String setting, Validator validator) {
|
||||
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);
|
||||
}
|
||||
|
||||
public void registerShardsAllocator(String name, Class<? extends ShardsAllocator> clazz) {
|
||||
shardsAllocators.registerExtension(name, clazz);
|
||||
}
|
||||
|
||||
public void registerIndexTemplateFilter(Class<? extends IndexTemplateFilter> indexTemplateFilter) {
|
||||
indexTemplateFilters.add(indexTemplateFilter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterable<? extends Module> spawnModules() {
|
||||
return ImmutableList.of(new AllocationModule(settings),
|
||||
new ClusterDynamicSettingsModule(),
|
||||
new IndexDynamicSettingsModule());
|
||||
indexTemplateFilters.registerExtension(indexTemplateFilter);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(DynamicSettings.class).annotatedWith(ClusterDynamicSettings.class).toInstance(clusterDynamicSettings.build());
|
||||
bind(DynamicSettings.class).annotatedWith(IndexDynamicSettings.class).toInstance(indexDynamicSettings.build());
|
||||
|
||||
// bind ShardsAllocator
|
||||
String shardsAllocatorType = shardsAllocators.bindType(binder(), settings, ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, ClusterModule.BALANCED_ALLOCATOR);
|
||||
if (shardsAllocatorType.equals(ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR)) {
|
||||
final ESLogger logger = Loggers.getLogger(getClass(), settings);
|
||||
logger.warn("{} allocator has been removed in 2.0 using {} instead", ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR, ClusterModule.BALANCED_ALLOCATOR);
|
||||
}
|
||||
allocationDeciders.bind(binder());
|
||||
indexTemplateFilters.bind(binder());
|
||||
|
||||
bind(ClusterInfoService.class).to(clusterInfoServiceImpl).asEagerSingleton();
|
||||
bind(GatewayAllocator.class).asEagerSingleton();
|
||||
bind(AllocationService.class).asEagerSingleton();
|
||||
bind(DiscoveryNodeService.class).asEagerSingleton();
|
||||
bind(ClusterService.class).to(InternalClusterService.class).asEagerSingleton();
|
||||
bind(OperationRouting.class).asEagerSingleton();
|
||||
|
@ -84,18 +326,10 @@ public class ClusterModule extends AbstractModule implements SpawnModules {
|
|||
bind(MetaDataUpdateSettingsService.class).asEagerSingleton();
|
||||
bind(MetaDataIndexTemplateService.class).asEagerSingleton();
|
||||
bind(IndexNameExpressionResolver.class).asEagerSingleton();
|
||||
|
||||
bind(RoutingService.class).asEagerSingleton();
|
||||
|
||||
bind(ShardStateAction.class).asEagerSingleton();
|
||||
bind(NodeIndexDeletedAction.class).asEagerSingleton();
|
||||
bind(NodeMappingRefreshAction.class).asEagerSingleton();
|
||||
bind(MappingUpdatedAction.class).asEagerSingleton();
|
||||
bind(ClusterInfoService.class).to(clusterInfoServiceImpl).asEagerSingleton();
|
||||
|
||||
Multibinder<IndexTemplateFilter> mbinder = Multibinder.newSetBinder(binder(), IndexTemplateFilter.class);
|
||||
for (Class<? extends IndexTemplateFilter> indexTemplateFilter : indexTemplateFilters) {
|
||||
mbinder.addBinding().to(indexTemplateFilter);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -191,7 +191,7 @@ public class MetaDataUpdateSettingsService extends AbstractComponent implements
|
|||
if (!dynamicSettings.hasDynamicSetting(setting.getKey())) {
|
||||
removedSettings.add(setting.getKey());
|
||||
} else {
|
||||
String error = dynamicSettings.validateDynamicSetting(setting.getKey(), setting.getValue());
|
||||
String error = dynamicSettings.validateDynamicSetting(setting.getKey(), setting.getValue(), clusterService.state());
|
||||
if (error != null) {
|
||||
errors.add("[" + setting.getKey() + "] - " + error);
|
||||
}
|
||||
|
|
|
@ -1,128 +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.routing.allocation;
|
||||
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.AwarenessAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.RebalanceOnlyWhenActiveAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.SnapshotInProgressAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.multibindings.Multibinder;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.logging.Loggers;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.ExtensionPoint;
|
||||
import org.elasticsearch.gateway.GatewayAllocator;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* A module to setup classes related to shard allocation.
|
||||
*
|
||||
* There are two basic concepts for allocation.
|
||||
* <ul>
|
||||
* <li>An {@link AllocationDecider} decides *when* an allocation should be attempted.</li>
|
||||
* <li>A {@link ShardsAllocator} determins *how* an allocation takes place</li>
|
||||
* </ul>
|
||||
*/
|
||||
public class AllocationModule extends AbstractModule {
|
||||
|
||||
public static final String EVEN_SHARD_COUNT_ALLOCATOR = "even_shard";
|
||||
public static final String BALANCED_ALLOCATOR = "balanced"; // default
|
||||
public static final String SHARDS_ALLOCATOR_TYPE_KEY = "cluster.routing.allocation.type";
|
||||
|
||||
public static final List<Class<? extends AllocationDecider>> DEFAULT_ALLOCATION_DECIDERS =
|
||||
Collections.unmodifiableList(Arrays.asList(
|
||||
SameShardAllocationDecider.class,
|
||||
FilterAllocationDecider.class,
|
||||
ReplicaAfterPrimaryActiveAllocationDecider.class,
|
||||
ThrottlingAllocationDecider.class,
|
||||
RebalanceOnlyWhenActiveAllocationDecider.class,
|
||||
ClusterRebalanceAllocationDecider.class,
|
||||
ConcurrentRebalanceAllocationDecider.class,
|
||||
EnableAllocationDecider.class, // new enable allocation logic should proceed old disable allocation logic
|
||||
DisableAllocationDecider.class,
|
||||
AwarenessAllocationDecider.class,
|
||||
ShardsLimitAllocationDecider.class,
|
||||
NodeVersionAllocationDecider.class,
|
||||
DiskThresholdDecider.class,
|
||||
SnapshotInProgressAllocationDecider.class));
|
||||
|
||||
|
||||
private final Settings settings;
|
||||
private final ExtensionPoint.TypeExtensionPoint<ShardsAllocator> shardsAllocators = new ExtensionPoint.TypeExtensionPoint<>("shards_allocator", ShardsAllocator.class);
|
||||
private final ExtensionPoint.SetExtensionPoint<AllocationDecider> allocationDeciders = new ExtensionPoint.SetExtensionPoint<>("allocation_decider", AllocationDecider.class, AllocationDeciders.class);
|
||||
|
||||
public AllocationModule(Settings settings) {
|
||||
this.settings = settings;
|
||||
for (Class<? extends AllocationDecider> decider : DEFAULT_ALLOCATION_DECIDERS) {
|
||||
allocationDeciders.registerExtension(decider);
|
||||
}
|
||||
shardsAllocators.registerExtension(BALANCED_ALLOCATOR, BalancedShardsAllocator.class);
|
||||
shardsAllocators.registerExtension(EVEN_SHARD_COUNT_ALLOCATOR, BalancedShardsAllocator.class);
|
||||
}
|
||||
|
||||
/** Register a custom allocation decider */
|
||||
public void registerAllocationDecider(Class<? extends AllocationDecider> allocationDecider) {
|
||||
allocationDeciders.registerExtension(allocationDecider);
|
||||
}
|
||||
|
||||
/** Register a custom shard allocator with the given name */
|
||||
public void registerShardAllocator(String name, Class<? extends ShardsAllocator> clazz) {
|
||||
shardsAllocators.registerExtension(name, clazz);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
// bind ShardsAllocator
|
||||
String shardsAllocatorType = shardsAllocators.bindType(binder(), settings, AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, AllocationModule.BALANCED_ALLOCATOR);
|
||||
if (shardsAllocatorType.equals(EVEN_SHARD_COUNT_ALLOCATOR)) {
|
||||
final ESLogger logger = Loggers.getLogger(getClass(), settings);
|
||||
logger.warn("{} allocator has been removed in 2.0 using {} instead", AllocationModule.EVEN_SHARD_COUNT_ALLOCATOR, AllocationModule.BALANCED_ALLOCATOR);
|
||||
}
|
||||
// bind AllocationDeciders
|
||||
allocationDeciders.bind(binder());
|
||||
|
||||
bind(GatewayAllocator.class).asEagerSingleton();
|
||||
bind(AllocationService.class).asEagerSingleton();
|
||||
}
|
||||
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
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;
|
||||
|
@ -52,7 +53,7 @@ public class ClusterRebalanceAllocationDecider extends AllocationDecider {
|
|||
public static final String CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE = "cluster.routing.allocation.allow_rebalance";
|
||||
public static final Validator ALLOCATION_ALLOW_REBALANCE_VALIDATOR = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
ClusterRebalanceType.parseString(value);
|
||||
return null;
|
||||
|
|
|
@ -1,124 +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.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.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.util.Providers;
|
||||
import org.elasticsearch.discovery.DiscoverySettings;
|
||||
import org.elasticsearch.discovery.zen.ZenDiscovery;
|
||||
import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.indices.store.IndicesStore;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class ClusterDynamicSettingsModule extends AbstractModule {
|
||||
|
||||
private final DynamicSettings clusterDynamicSettings;
|
||||
|
||||
public ClusterDynamicSettingsModule() {
|
||||
clusterDynamicSettings = new DynamicSettings();
|
||||
clusterDynamicSettings.addDynamicSetting(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_ATTRIBUTES);
|
||||
clusterDynamicSettings.addDynamicSetting(AwarenessAllocationDecider.CLUSTER_ROUTING_ALLOCATION_AWARENESS_FORCE_GROUP + "*");
|
||||
clusterDynamicSettings.addDynamicSetting(BalancedShardsAllocator.SETTING_INDEX_BALANCE_FACTOR, Validator.FLOAT);
|
||||
clusterDynamicSettings.addDynamicSetting(BalancedShardsAllocator.SETTING_SHARD_BALANCE_FACTOR, Validator.FLOAT);
|
||||
clusterDynamicSettings.addDynamicSetting(BalancedShardsAllocator.SETTING_THRESHOLD, Validator.NON_NEGATIVE_FLOAT);
|
||||
clusterDynamicSettings.addDynamicSetting(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE,
|
||||
ClusterRebalanceAllocationDecider.ALLOCATION_ALLOW_REBALANCE_VALIDATOR);
|
||||
clusterDynamicSettings.addDynamicSetting(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE, Validator.INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE);
|
||||
clusterDynamicSettings.addDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE);
|
||||
clusterDynamicSettings.addDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION);
|
||||
clusterDynamicSettings.addDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_ALLOCATION);
|
||||
clusterDynamicSettings.addDynamicSetting(DisableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION);
|
||||
clusterDynamicSettings.addDynamicSetting(ZenDiscovery.SETTING_REJOIN_ON_MASTER_GONE, Validator.BOOLEAN);
|
||||
clusterDynamicSettings.addDynamicSetting(DiscoverySettings.NO_MASTER_BLOCK);
|
||||
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_INCLUDE_GROUP + "*");
|
||||
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP + "*");
|
||||
clusterDynamicSettings.addDynamicSetting(FilterAllocationDecider.CLUSTER_ROUTING_REQUIRE_GROUP + "*");
|
||||
clusterDynamicSettings.addDynamicSetting(IndicesStore.INDICES_STORE_THROTTLE_TYPE);
|
||||
clusterDynamicSettings.addDynamicSetting(IndicesStore.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(IndicesTTLService.INDICES_TTL_INTERVAL, Validator.TIME);
|
||||
clusterDynamicSettings.addDynamicSetting(MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT, Validator.TIME);
|
||||
clusterDynamicSettings.addDynamicSetting(MetaData.SETTING_READ_ONLY);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, Validator.BYTES_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_OPS, Validator.INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, Validator.BYTES_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_COMPRESS);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, Validator.POSITIVE_INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, Validator.POSITIVE_INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(RecoverySettings.INDICES_RECOVERY_MAX_SIZE_PER_SEC, Validator.BYTES_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(ThreadPool.THREADPOOL_GROUP + "*");
|
||||
clusterDynamicSettings.addDynamicSetting(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES, Validator.INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES, Validator.INTEGER);
|
||||
clusterDynamicSettings.addDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK);
|
||||
clusterDynamicSettings.addDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK);
|
||||
clusterDynamicSettings.addDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED, Validator.BOOLEAN);
|
||||
clusterDynamicSettings.addDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS, Validator.BOOLEAN);
|
||||
clusterDynamicSettings.addDynamicSetting(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(SnapshotInProgressAllocationDecider.CLUSTER_ROUTING_ALLOCATION_SNAPSHOT_RELOCATION_ENABLED);
|
||||
clusterDynamicSettings.addDynamicSetting(DestructiveOperations.REQUIRES_NAME);
|
||||
clusterDynamicSettings.addDynamicSetting(DiscoverySettings.PUBLISH_TIMEOUT, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(DiscoverySettings.PUBLISH_DIFF_ENABLE, Validator.BOOLEAN);
|
||||
clusterDynamicSettings.addDynamicSetting(HierarchyCircuitBreakerService.TOTAL_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(HierarchyCircuitBreakerService.FIELDDATA_CIRCUIT_BREAKER_OVERHEAD_SETTING, Validator.NON_NEGATIVE_DOUBLE);
|
||||
clusterDynamicSettings.addDynamicSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_LIMIT_SETTING, Validator.MEMORY_SIZE);
|
||||
clusterDynamicSettings.addDynamicSetting(HierarchyCircuitBreakerService.REQUEST_CIRCUIT_BREAKER_OVERHEAD_SETTING, Validator.NON_NEGATIVE_DOUBLE);
|
||||
clusterDynamicSettings.addDynamicSetting(InternalClusterService.SETTING_CLUSTER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD, Validator.TIME_NON_NEGATIVE);
|
||||
clusterDynamicSettings.addDynamicSetting(SearchService.DEFAULT_SEARCH_TIMEOUT, Validator.TIMEOUT);
|
||||
}
|
||||
|
||||
public void addDynamicSettings(String... settings) {
|
||||
clusterDynamicSettings.addDynamicSettings(settings);
|
||||
}
|
||||
|
||||
public void addDynamicSetting(String setting, Validator validator) {
|
||||
clusterDynamicSettings.addDynamicSetting(setting, validator);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(DynamicSettings.class).annotatedWith(ClusterDynamicSettings.class).toInstance(clusterDynamicSettings);
|
||||
|
||||
// Bind to null provider just in case somebody will forget to supply @ClusterDynamicSetting or @IndexDynamicSetting annotations
|
||||
// This will cause any attempt to inject a unannotated DynamicSettings to fail with Guice error, instead of silently
|
||||
// injecting an empty copy of dynamic settings
|
||||
bind(DynamicSettings.class).toProvider(Providers.<DynamicSettings>of(null));
|
||||
}
|
||||
}
|
|
@ -19,18 +19,39 @@
|
|||
|
||||
package org.elasticsearch.cluster.settings;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.regex.Regex;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A container for setting names and validation methods for those settings.
|
||||
*/
|
||||
public class DynamicSettings {
|
||||
|
||||
private ImmutableMap<String, Validator> dynamicSettings = ImmutableMap.of();
|
||||
private final Map<String, Validator> dynamicSettings;
|
||||
|
||||
public static class Builder {
|
||||
private Map<String, Validator> settings = new HashMap<>();
|
||||
|
||||
public void addSetting(String setting, Validator validator) {
|
||||
Validator old = settings.put(setting, validator);
|
||||
if (old != null) {
|
||||
throw new IllegalArgumentException("Cannot register setting [" + setting + "] twice");
|
||||
}
|
||||
}
|
||||
|
||||
public DynamicSettings build() {
|
||||
return new DynamicSettings(settings);
|
||||
}
|
||||
}
|
||||
|
||||
private DynamicSettings(Map<String, Validator> settings) {
|
||||
this.dynamicSettings = Collections.unmodifiableMap(settings);
|
||||
}
|
||||
|
||||
public boolean isDynamicOrLoggingSetting(String key) {
|
||||
return hasDynamicSetting(key) || key.startsWith("logger.");
|
||||
|
@ -45,32 +66,12 @@ public class DynamicSettings {
|
|||
return false;
|
||||
}
|
||||
|
||||
public String validateDynamicSetting(String dynamicSetting, String value) {
|
||||
public String validateDynamicSetting(String dynamicSetting, String value, ClusterState clusterState) {
|
||||
for (Map.Entry<String, Validator> setting : dynamicSettings.entrySet()) {
|
||||
if (Regex.simpleMatch(setting.getKey(), dynamicSetting)) {
|
||||
return setting.getValue().validate(dynamicSetting, value);
|
||||
return setting.getValue().validate(dynamicSetting, value, clusterState);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
public synchronized void addDynamicSetting(String setting, Validator validator) {
|
||||
MapBuilder<String, Validator> updatedSettings = MapBuilder.newMapBuilder(dynamicSettings);
|
||||
updatedSettings.put(setting, validator);
|
||||
dynamicSettings = updatedSettings.immutableMap();
|
||||
}
|
||||
|
||||
public synchronized void addDynamicSetting(String setting) {
|
||||
addDynamicSetting(setting, Validator.EMPTY);
|
||||
}
|
||||
|
||||
|
||||
public synchronized void addDynamicSettings(String... settings) {
|
||||
MapBuilder<String, Validator> updatedSettings = MapBuilder.newMapBuilder(dynamicSettings);
|
||||
for (String setting : settings) {
|
||||
updatedSettings.put(setting, Validator.EMPTY);
|
||||
}
|
||||
dynamicSettings = updatedSettings.immutableMap();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.cluster.settings;
|
||||
|
||||
import org.elasticsearch.ElasticsearchParseException;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.common.Booleans;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
|
@ -32,18 +33,18 @@ import static org.elasticsearch.common.unit.MemorySizeValue.parseBytesSizeValueO
|
|||
*/
|
||||
public interface Validator {
|
||||
|
||||
String validate(String setting, String value);
|
||||
String validate(String setting, String value, ClusterState clusterState);
|
||||
|
||||
public static final Validator EMPTY = new Validator() {
|
||||
Validator EMPTY = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
return null;
|
||||
}
|
||||
};
|
||||
|
||||
public static final Validator TIME = new Validator() {
|
||||
Validator TIME = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value must not be null");
|
||||
}
|
||||
|
@ -57,9 +58,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator TIMEOUT = new Validator() {
|
||||
Validator TIMEOUT = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value must not be null");
|
||||
|
@ -76,9 +77,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator TIME_NON_NEGATIVE = new Validator() {
|
||||
Validator TIME_NON_NEGATIVE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (value == null) {
|
||||
throw new NullPointerException("value must not be null");
|
||||
|
@ -95,9 +96,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator FLOAT = new Validator() {
|
||||
Validator FLOAT = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
Float.parseFloat(value);
|
||||
} catch (NumberFormatException ex) {
|
||||
|
@ -107,9 +108,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator NON_NEGATIVE_FLOAT = new Validator() {
|
||||
Validator NON_NEGATIVE_FLOAT = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Float.parseFloat(value) < 0.0) {
|
||||
return "the value of the setting " + setting + " must be a non negative float";
|
||||
|
@ -121,9 +122,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator DOUBLE = new Validator() {
|
||||
Validator DOUBLE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
Double.parseDouble(value);
|
||||
} catch (NumberFormatException ex) {
|
||||
|
@ -133,9 +134,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator NON_NEGATIVE_DOUBLE = new Validator() {
|
||||
Validator NON_NEGATIVE_DOUBLE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Double.parseDouble(value) < 0.0) {
|
||||
return "the value of the setting " + setting + " must be a non negative double";
|
||||
|
@ -147,9 +148,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator DOUBLE_GTE_2 = new Validator() {
|
||||
Validator DOUBLE_GTE_2 = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Double.parseDouble(value) < 2.0) {
|
||||
return "the value of the setting " + setting + " must be >= 2.0";
|
||||
|
@ -161,9 +162,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator INTEGER = new Validator() {
|
||||
Validator INTEGER = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
Integer.parseInt(value);
|
||||
} catch (NumberFormatException ex) {
|
||||
|
@ -173,9 +174,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator POSITIVE_INTEGER = new Validator() {
|
||||
Validator POSITIVE_INTEGER = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Integer.parseInt(value) <= 0) {
|
||||
return "the value of the setting " + setting + " must be a positive integer";
|
||||
|
@ -187,9 +188,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator NON_NEGATIVE_INTEGER = new Validator() {
|
||||
Validator NON_NEGATIVE_INTEGER = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Integer.parseInt(value) < 0) {
|
||||
return "the value of the setting " + setting + " must be a non negative integer";
|
||||
|
@ -201,9 +202,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator INTEGER_GTE_2 = new Validator() {
|
||||
Validator INTEGER_GTE_2 = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (Integer.parseInt(value) < 2) {
|
||||
return "the value of the setting " + setting + " must be >= 2";
|
||||
|
@ -215,9 +216,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator BYTES_SIZE = new Validator() {
|
||||
Validator BYTES_SIZE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
parseBytesSizeValue(value, setting);
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
|
@ -227,9 +228,9 @@ public interface Validator {
|
|||
}
|
||||
};
|
||||
|
||||
public static final Validator PERCENTAGE = new Validator() {
|
||||
Validator PERCENTAGE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
if (value == null) {
|
||||
return "the value of " + setting + " can not be null";
|
||||
|
@ -249,12 +250,12 @@ public interface Validator {
|
|||
};
|
||||
|
||||
|
||||
public static final Validator BYTES_SIZE_OR_PERCENTAGE = new Validator() {
|
||||
Validator BYTES_SIZE_OR_PERCENTAGE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
String byteSize = BYTES_SIZE.validate(setting, value);
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
String byteSize = BYTES_SIZE.validate(setting, value, clusterState);
|
||||
if (byteSize != null) {
|
||||
String percentage = PERCENTAGE.validate(setting, value);
|
||||
String percentage = PERCENTAGE.validate(setting, value, clusterState);
|
||||
if (percentage == null) {
|
||||
return null;
|
||||
}
|
||||
|
@ -265,9 +266,9 @@ public interface Validator {
|
|||
};
|
||||
|
||||
|
||||
public static final Validator MEMORY_SIZE = new Validator() {
|
||||
Validator MEMORY_SIZE = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
try {
|
||||
parseBytesSizeValueOrHeapRatio(value, setting);
|
||||
} catch (ElasticsearchParseException ex) {
|
||||
|
@ -279,7 +280,7 @@ public interface Validator {
|
|||
|
||||
public static final Validator BOOLEAN = new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
public String validate(String setting, String value, ClusterState clusterState) {
|
||||
|
||||
if (value != null && (Booleans.isExplicitFalse(value) || Booleans.isExplicitTrue(value))) {
|
||||
return null;
|
||||
|
|
|
@ -39,7 +39,7 @@ public abstract class ExtensionPoint<T> {
|
|||
/**
|
||||
* Creates a new extension point
|
||||
*
|
||||
* @param name the human readable underscore case name of the extension poing. This is used in error messages etc.
|
||||
* @param name the human readable underscore case name of the extension point. This is used in error messages etc.
|
||||
* @param extensionClass the base class that should be extended
|
||||
* @param singletons a list of singletons to bind with this extension point - these are bound in {@link #bind(Binder)}
|
||||
*/
|
||||
|
@ -55,9 +55,6 @@ public abstract class ExtensionPoint<T> {
|
|||
* @param binder the binder to use
|
||||
*/
|
||||
public final void bind(Binder binder) {
|
||||
if (singletons == null || singletons.length == 0) {
|
||||
throw new IllegalStateException("Can't bind empty or null singletons");
|
||||
}
|
||||
for (Class<?> c : singletons) {
|
||||
binder.bind(c).asEagerSingleton();
|
||||
}
|
||||
|
|
|
@ -150,7 +150,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
public ZenDiscovery(Settings settings, ClusterName clusterName, ThreadPool threadPool,
|
||||
TransportService transportService, final ClusterService clusterService, NodeSettingsService nodeSettingsService,
|
||||
ZenPingService pingService, ElectMasterService electMasterService,
|
||||
DiscoverySettings discoverySettings, @ClusterDynamicSettings DynamicSettings dynamicSettings) {
|
||||
DiscoverySettings discoverySettings) {
|
||||
super(settings);
|
||||
this.clusterName = clusterName;
|
||||
this.clusterService = clusterService;
|
||||
|
@ -198,24 +198,6 @@ public class ZenDiscovery extends AbstractLifecycleComponent<Discovery> implemen
|
|||
this.joinThreadControl = new JoinThreadControl(threadPool);
|
||||
|
||||
transportService.registerRequestHandler(DISCOVERY_REJOIN_ACTION_NAME, RejoinClusterRequest.class, ThreadPool.Names.SAME, new RejoinClusterRequestHandler());
|
||||
|
||||
dynamicSettings.addDynamicSetting(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES, new Validator() {
|
||||
@Override
|
||||
public String validate(String setting, String value) {
|
||||
int intValue;
|
||||
try {
|
||||
intValue = Integer.parseInt(value);
|
||||
} catch (NumberFormatException ex) {
|
||||
return "cannot parse value [" + value + "] as an integer";
|
||||
}
|
||||
int masterNodes = clusterService.state().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;
|
||||
}
|
||||
});
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -23,7 +23,9 @@ import com.carrotsearch.hppc.ObjectContainer;
|
|||
import com.google.common.collect.Lists;
|
||||
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.Settings;
|
||||
|
@ -36,6 +38,22 @@ import java.util.*;
|
|||
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;
|
||||
}
|
||||
};
|
||||
|
||||
// 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
|
||||
|
|
|
@ -1,139 +0,0 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.index.settings;
|
||||
|
||||
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||
import org.elasticsearch.gateway.PrimaryShardAllocator;
|
||||
import org.elasticsearch.index.shard.MergeSchedulerConfig;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.DisableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
||||
import org.elasticsearch.cluster.routing.allocation.decider.ShardsLimitAllocationDecider;
|
||||
import org.elasticsearch.cluster.settings.DynamicSettings;
|
||||
import org.elasticsearch.cluster.settings.Validator;
|
||||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.index.engine.EngineConfig;
|
||||
import org.elasticsearch.index.indexing.IndexingSlowLog;
|
||||
import org.elasticsearch.index.search.stats.SearchSlowLog;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.MergePolicyConfig;
|
||||
import org.elasticsearch.index.store.IndexStore;
|
||||
import org.elasticsearch.index.translog.TranslogConfig;
|
||||
import org.elasticsearch.index.translog.TranslogService;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
|
||||
import org.elasticsearch.indices.ttl.IndicesTTLService;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class IndexDynamicSettingsModule extends AbstractModule {
|
||||
|
||||
private final DynamicSettings indexDynamicSettings;
|
||||
|
||||
public IndexDynamicSettingsModule() {
|
||||
indexDynamicSettings = new DynamicSettings();
|
||||
indexDynamicSettings.addDynamicSetting(IndexStore.INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexStore.INDEX_STORE_THROTTLE_TYPE);
|
||||
indexDynamicSettings.addDynamicSetting(MergeSchedulerConfig.MAX_THREAD_COUNT);
|
||||
indexDynamicSettings.addDynamicSetting(MergeSchedulerConfig.MAX_MERGE_COUNT);
|
||||
indexDynamicSettings.addDynamicSetting(MergeSchedulerConfig.AUTO_THROTTLE);
|
||||
indexDynamicSettings.addDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_REQUIRE_GROUP + "*");
|
||||
indexDynamicSettings.addDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_INCLUDE_GROUP + "*");
|
||||
indexDynamicSettings.addDynamicSetting(FilterAllocationDecider.INDEX_ROUTING_EXCLUDE_GROUP + "*");
|
||||
indexDynamicSettings.addDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE);
|
||||
indexDynamicSettings.addDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE);
|
||||
indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_ALLOCATION);
|
||||
indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_NEW_ALLOCATION);
|
||||
indexDynamicSettings.addDynamicSetting(DisableAllocationDecider.INDEX_ROUTING_ALLOCATION_DISABLE_REPLICA_ALLOCATION);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogConfig.INDEX_TRANSLOG_FS_TYPE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, Validator.NON_NEGATIVE_INTEGER);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_READ_ONLY);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_READ);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_WRITE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_BLOCKS_METADATA);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_SHARED_FS_ALLOW_RECOVERY_ON_ANY_NODE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexMetaData.SETTING_PRIORITY, Validator.NON_NEGATIVE_INTEGER);
|
||||
indexDynamicSettings.addDynamicSetting(IndicesTTLService.INDEX_TTL_DISABLE_PURGE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_REFRESH_INTERVAL, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(PrimaryShardAllocator.INDEX_RECOVERY_INITIAL_SHARDS);
|
||||
indexDynamicSettings.addDynamicSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH, Validator.BOOLEAN);
|
||||
indexDynamicSettings.addDynamicSetting(EngineConfig.INDEX_GC_DELETES_SETTING, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(IndexShard.INDEX_FLUSH_ON_CLOSE, Validator.BOOLEAN);
|
||||
indexDynamicSettings.addDynamicSetting(EngineConfig.INDEX_VERSION_MAP_SIZE, Validator.BYTES_SIZE_OR_PERCENTAGE);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_WARN, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_INFO, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_DEBUG, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_THRESHOLD_INDEX_TRACE, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_REFORMAT);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_LEVEL);
|
||||
indexDynamicSettings.addDynamicSetting(IndexingSlowLog.INDEX_INDEXING_SLOWLOG_MAX_SOURCE_CHARS_TO_LOG);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_WARN, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_INFO, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_DEBUG, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_QUERY_TRACE, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_WARN, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_INFO, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_DEBUG, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_THRESHOLD_FETCH_TRACE, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_REFORMAT);
|
||||
indexDynamicSettings.addDynamicSetting(SearchSlowLog.INDEX_SEARCH_SLOWLOG_LEVEL);
|
||||
indexDynamicSettings.addDynamicSetting(ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE, Validator.INTEGER);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_EXPUNGE_DELETES_ALLOWED, Validator.DOUBLE);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_FLOOR_SEGMENT, Validator.BYTES_SIZE);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE, Validator.INTEGER_GTE_2);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_EXPLICIT, Validator.INTEGER_GTE_2);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_MAX_MERGED_SEGMENT, Validator.BYTES_SIZE);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER, Validator.DOUBLE_GTE_2);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_MERGE_POLICY_RECLAIM_DELETES_WEIGHT, Validator.NON_NEGATIVE_DOUBLE);
|
||||
indexDynamicSettings.addDynamicSetting(MergePolicyConfig.INDEX_COMPOUND_FORMAT);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_INTERVAL, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, Validator.INTEGER);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, Validator.BYTES_SIZE);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_FLUSH_THRESHOLD_PERIOD, Validator.TIME);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogService.INDEX_TRANSLOG_DISABLE_FLUSH);
|
||||
indexDynamicSettings.addDynamicSetting(TranslogConfig.INDEX_TRANSLOG_DURABILITY);
|
||||
indexDynamicSettings.addDynamicSetting(IndicesWarmer.INDEX_WARMER_ENABLED);
|
||||
indexDynamicSettings.addDynamicSetting(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED, Validator.BOOLEAN);
|
||||
indexDynamicSettings.addDynamicSetting(IndicesRequestCache.DEPRECATED_INDEX_CACHE_REQUEST_ENABLED, Validator.BOOLEAN);
|
||||
indexDynamicSettings.addDynamicSetting(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, Validator.TIME);
|
||||
}
|
||||
|
||||
public void addDynamicSettings(String... settings) {
|
||||
indexDynamicSettings.addDynamicSettings(settings);
|
||||
}
|
||||
|
||||
public void addDynamicSetting(String setting, Validator validator) {
|
||||
indexDynamicSettings.addDynamicSetting(setting, validator);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void configure() {
|
||||
bind(DynamicSettings.class).annotatedWith(IndexDynamicSettings.class).toInstance(indexDynamicSettings);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> iff the given setting is in the dynamic settings map. Otherwise <code>false</code>.
|
||||
*/
|
||||
public boolean containsSetting(String setting) {
|
||||
return indexDynamicSettings.hasDynamicSetting(setting);
|
||||
}
|
||||
}
|
|
@ -63,7 +63,7 @@ public class IndexSettingsService extends AbstractIndexComponent {
|
|||
}
|
||||
|
||||
/**
|
||||
* Only settings registered in {@link IndexDynamicSettingsModule} can be changed dynamically.
|
||||
* Only settings registered in {@link org.elasticsearch.cluster.ClusterModule} can be changed dynamically.
|
||||
*/
|
||||
public void addListener(Listener listener) {
|
||||
this.listeners.add(listener);
|
||||
|
@ -73,7 +73,7 @@ public class IndexSettingsService extends AbstractIndexComponent {
|
|||
this.listeners.remove(listener);
|
||||
}
|
||||
|
||||
public static interface Listener {
|
||||
public interface Listener {
|
||||
void onRefreshSettings(Settings settings);
|
||||
}
|
||||
}
|
|
@ -106,7 +106,7 @@ public class NodeSettingsService extends AbstractComponent implements ClusterSta
|
|||
}
|
||||
|
||||
/**
|
||||
* Only settings registered in {@link org.elasticsearch.cluster.settings.ClusterDynamicSettingsModule} can be changed dynamically.
|
||||
* Only settings registered in {@link org.elasticsearch.cluster.ClusterModule} can be changed dynamically.
|
||||
*/
|
||||
public void addListener(Listener listener) {
|
||||
this.listeners.add(listener);
|
||||
|
@ -116,7 +116,7 @@ public class NodeSettingsService extends AbstractComponent implements ClusterSta
|
|||
this.listeners.remove(listener);
|
||||
}
|
||||
|
||||
public static interface Listener {
|
||||
public interface Listener {
|
||||
void onRefreshSettings(Settings settings);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -392,7 +392,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
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());
|
||||
String error = dynamicSettings.validateDynamicSetting(entry.getKey(), entry.getValue(), clusterService.state());
|
||||
if (error == null) {
|
||||
persistentSettings.put(entry.getKey(), entry.getValue());
|
||||
changed = true;
|
||||
|
|
|
@ -130,9 +130,7 @@ 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, @ClusterDynamicSettings DynamicSettings dynamicSettings) {
|
||||
dynamicSettings.addDynamicSettings(SETTING_TRACE_LOG_INCLUDE, SETTING_TRACE_LOG_INCLUDE + ".*");
|
||||
dynamicSettings.addDynamicSettings(SETTING_TRACE_LOG_EXCLUDE, SETTING_TRACE_LOG_EXCLUDE + ".*");
|
||||
public void setDynamicSettings(NodeSettingsService nodeSettingsService) {
|
||||
nodeSettingsService.addListener(settingsListener);
|
||||
}
|
||||
|
||||
|
|
|
@ -54,9 +54,6 @@ public class BenchmarkNettyLargeMessages {
|
|||
.build();
|
||||
|
||||
NetworkService networkService = new NetworkService(settings);
|
||||
NodeSettingsService settingsService = new NodeSettingsService(settings);
|
||||
DynamicSettings dynamicSettings = new DynamicSettings();
|
||||
|
||||
|
||||
final ThreadPool threadPool = new ThreadPool("BenchmarkNettyLargeMessages");
|
||||
final TransportService transportServiceServer = new TransportService(
|
||||
|
|
|
@ -0,0 +1,175 @@
|
|||
/*
|
||||
* 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;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
|
||||
import org.elasticsearch.cluster.metadata.IndexTemplateFilter;
|
||||
import org.elasticsearch.cluster.metadata.IndexTemplateMetaData;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.allocation.FailedRerouteAllocation;
|
||||
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
||||
import org.elasticsearch.cluster.routing.allocation.StartedRerouteAllocation;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
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.index.settings.IndexDynamicSettings;
|
||||
|
||||
public class ClusterModuleTests extends ModuleTestCase {
|
||||
|
||||
public static class FakeAllocationDecider extends AllocationDecider {
|
||||
protected FakeAllocationDecider(Settings settings) {
|
||||
super(settings);
|
||||
}
|
||||
}
|
||||
|
||||
static class FakeShardsAllocator implements ShardsAllocator {
|
||||
@Override
|
||||
public void applyStartedShards(StartedRerouteAllocation allocation) {}
|
||||
@Override
|
||||
public void applyFailedShards(FailedRerouteAllocation allocation) {}
|
||||
@Override
|
||||
public boolean allocateUnassigned(RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public boolean rebalance(RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public boolean move(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
static class FakeIndexTemplateFilter implements IndexTemplateFilter {
|
||||
@Override
|
||||
public boolean apply(CreateIndexClusterStateUpdateRequest request, IndexTemplateMetaData template) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterClusterDynamicSettingDuplicate() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerClusterDynamicSetting(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Cannot register setting [" + EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE + "] twice");
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterClusterDynamicSetting() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
module.registerClusterDynamicSetting("foo.bar", Validator.EMPTY);
|
||||
assertInstanceBindingWithAnnotation(module, DynamicSettings.class, new Predicate<DynamicSettings>() {
|
||||
@Override
|
||||
public boolean apply(DynamicSettings dynamicSettings) {
|
||||
return dynamicSettings.hasDynamicSetting("foo.bar");
|
||||
}
|
||||
}, ClusterDynamicSettings.class);
|
||||
}
|
||||
|
||||
public void testRegisterIndexDynamicSettingDuplicate() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerIndexDynamicSetting(EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE, Validator.EMPTY);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Cannot register setting [" + EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE + "] twice");
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterIndexDynamicSetting() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
module.registerIndexDynamicSetting("foo.bar", Validator.EMPTY);
|
||||
assertInstanceBindingWithAnnotation(module, DynamicSettings.class, new Predicate<DynamicSettings>() {
|
||||
@Override
|
||||
public boolean apply(DynamicSettings dynamicSettings) {
|
||||
return dynamicSettings.hasDynamicSetting("foo.bar");
|
||||
}
|
||||
}, IndexDynamicSettings.class);
|
||||
}
|
||||
|
||||
public void testRegisterAllocationDeciderDuplicate() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerAllocationDecider(EnableAllocationDecider.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Can't register the same [allocation_decider] more than once for [" + EnableAllocationDecider.class.getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterAllocationDecider() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
module.registerAllocationDecider(FakeAllocationDecider.class);
|
||||
assertSetMultiBinding(module, AllocationDecider.class, FakeAllocationDecider.class);
|
||||
}
|
||||
|
||||
public void testRegisterShardsAllocator() {
|
||||
Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, "custom").build();
|
||||
ClusterModule module = new ClusterModule(settings);
|
||||
module.registerShardsAllocator("custom", FakeShardsAllocator.class);
|
||||
assertBinding(module, ShardsAllocator.class, FakeShardsAllocator.class);
|
||||
}
|
||||
|
||||
public void testRegisterShardsAllocatorAlreadyRegistered() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerShardsAllocator(ClusterModule.BALANCED_ALLOCATOR, FakeShardsAllocator.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Can't register the same [shards_allocator] more than once for [balanced]");
|
||||
}
|
||||
}
|
||||
|
||||
public void testUnknownShardsAllocator() {
|
||||
Settings settings = Settings.builder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, "dne").build();
|
||||
ClusterModule module = new ClusterModule(settings);
|
||||
assertBindingFailure(module, "Unknown [shards_allocator]");
|
||||
}
|
||||
|
||||
public void testEvenShardsAllocatorBackcompat() {
|
||||
Settings settings = Settings.builder()
|
||||
.put(ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, ClusterModule.EVEN_SHARD_COUNT_ALLOCATOR).build();
|
||||
ClusterModule module = new ClusterModule(settings);
|
||||
assertBinding(module, ShardsAllocator.class, BalancedShardsAllocator.class);
|
||||
}
|
||||
|
||||
public void testRegisterIndexTemplateFilterDuplicate() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
|
||||
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Can't register the same [index_template_filter] more than once for [" + FakeIndexTemplateFilter.class.getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterIndexTemplateFilter() {
|
||||
ClusterModule module = new ClusterModule(Settings.EMPTY);
|
||||
module.registerIndexTemplateFilter(FakeIndexTemplateFilter.class);
|
||||
assertSetMultiBinding(module, IndexTemplateFilter.class, FakeIndexTemplateFilter.class);
|
||||
}
|
||||
}
|
|
@ -19,7 +19,7 @@
|
|||
|
||||
package org.elasticsearch.cluster.allocation;
|
||||
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationModule;
|
||||
import org.elasticsearch.cluster.ClusterModule;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocator;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -40,10 +40,10 @@ public class ShardsAllocatorModuleIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
public void testLoadByShortKeyShardsAllocator() throws IOException {
|
||||
Settings build = settingsBuilder().put(AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, "even_shard") // legacy just to make sure we don't barf
|
||||
Settings build = settingsBuilder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, "even_shard") // legacy just to make sure we don't barf
|
||||
.build();
|
||||
assertAllocatorInstance(build, BalancedShardsAllocator.class);
|
||||
build = settingsBuilder().put(AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, AllocationModule.BALANCED_ALLOCATOR).build();
|
||||
build = settingsBuilder().put(ClusterModule.SHARDS_ALLOCATOR_TYPE_KEY, ClusterModule.BALANCED_ALLOCATOR).build();
|
||||
assertAllocatorInstance(build, BalancedShardsAllocator.class);
|
||||
}
|
||||
|
||||
|
|
|
@ -1,100 +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.routing.allocation;
|
||||
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator;
|
||||
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.common.inject.ModuleTestCase;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
||||
public class AllocationModuleTests extends ModuleTestCase {
|
||||
|
||||
public static class FakeAllocationDecider extends AllocationDecider {
|
||||
protected FakeAllocationDecider(Settings settings) {
|
||||
super(settings);
|
||||
}
|
||||
}
|
||||
|
||||
public static class FakeShardsAllocator implements ShardsAllocator {
|
||||
@Override
|
||||
public void applyStartedShards(StartedRerouteAllocation allocation) {}
|
||||
@Override
|
||||
public void applyFailedShards(FailedRerouteAllocation allocation) {}
|
||||
@Override
|
||||
public boolean allocateUnassigned(RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public boolean rebalance(RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
@Override
|
||||
public boolean move(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterAllocationDeciderDuplicate() {
|
||||
AllocationModule module = new AllocationModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerAllocationDecider(EnableAllocationDecider.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Can't register the same [allocation_decider] more than once for [" + EnableAllocationDecider.class.getName() + "]");
|
||||
}
|
||||
}
|
||||
|
||||
public void testRegisterAllocationDecider() {
|
||||
AllocationModule module = new AllocationModule(Settings.EMPTY);
|
||||
module.registerAllocationDecider(FakeAllocationDecider.class);
|
||||
assertSetMultiBinding(module, AllocationDecider.class, FakeAllocationDecider.class);
|
||||
}
|
||||
|
||||
public void testRegisterShardsAllocator() {
|
||||
Settings settings = Settings.builder().put(AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, "custom").build();
|
||||
AllocationModule module = new AllocationModule(settings);
|
||||
module.registerShardAllocator("custom", FakeShardsAllocator.class);
|
||||
assertBinding(module, ShardsAllocator.class, FakeShardsAllocator.class);
|
||||
}
|
||||
|
||||
public void testRegisterShardsAllocatorAlreadyRegistered() {
|
||||
AllocationModule module = new AllocationModule(Settings.EMPTY);
|
||||
try {
|
||||
module.registerShardAllocator(AllocationModule.BALANCED_ALLOCATOR, FakeShardsAllocator.class);
|
||||
} catch (IllegalArgumentException e) {
|
||||
assertEquals(e.getMessage(), "Can't register the same [shards_allocator] more than once for [balanced]");
|
||||
}
|
||||
}
|
||||
|
||||
public void testUnknownShardsAllocator() {
|
||||
Settings settings = Settings.builder().put(AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, "dne").build();
|
||||
AllocationModule module = new AllocationModule(settings);
|
||||
assertBindingFailure(module, "Unknown [shards_allocator]");
|
||||
}
|
||||
|
||||
public void testEvenShardsAllocatorBackcompat() {
|
||||
Settings settings = Settings.builder()
|
||||
.put(AllocationModule.SHARDS_ALLOCATOR_TYPE_KEY, AllocationModule.EVEN_SHARD_COUNT_ALLOCATOR).build();
|
||||
AllocationModule module = new AllocationModule(settings);
|
||||
assertBinding(module, ShardsAllocator.class, BalancedShardsAllocator.class);
|
||||
}
|
||||
}
|
|
@ -28,81 +28,81 @@ public class SettingsValidatorTests extends ESTestCase {
|
|||
|
||||
@Test
|
||||
public void testValidators() throws Exception {
|
||||
assertThat(Validator.EMPTY.validate("", "anything goes"), nullValue());
|
||||
assertThat(Validator.EMPTY.validate("", "anything goes", null), nullValue());
|
||||
|
||||
assertThat(Validator.TIME.validate("", "10m"), nullValue());
|
||||
assertThat(Validator.TIME.validate("", "10g"), notNullValue());
|
||||
assertThat(Validator.TIME.validate("", "bad timing"), notNullValue());
|
||||
assertThat(Validator.TIME.validate("", "10m", null), nullValue());
|
||||
assertThat(Validator.TIME.validate("", "10g", null), notNullValue());
|
||||
assertThat(Validator.TIME.validate("", "bad timing", null), notNullValue());
|
||||
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "10m"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "10g"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "bad"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "10m", null), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "10g", null), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE.validate("", "bad", null), notNullValue());
|
||||
|
||||
assertThat(Validator.FLOAT.validate("", "10.2"), nullValue());
|
||||
assertThat(Validator.FLOAT.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.FLOAT.validate("", "10.2", null), nullValue());
|
||||
assertThat(Validator.FLOAT.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "10.2"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "0.0"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "-1.0"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "10.2", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "0.0", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "-1.0", null), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_FLOAT.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.DOUBLE.validate("", "10.2"), nullValue());
|
||||
assertThat(Validator.DOUBLE.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.DOUBLE.validate("", "10.2", null), nullValue());
|
||||
assertThat(Validator.DOUBLE.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "10.2"), nullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "2.0"), nullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "1.0"), notNullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "10.2", null), nullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "2.0", null), nullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "1.0", null), notNullValue());
|
||||
assertThat(Validator.DOUBLE_GTE_2.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "10.2"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "0.0"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "-1.0"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "10.2", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "0.0", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "-1.0", null), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_DOUBLE.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.INTEGER.validate("", "10"), nullValue());
|
||||
assertThat(Validator.INTEGER.validate("", "10.2"), notNullValue());
|
||||
assertThat(Validator.INTEGER.validate("", "10", null), nullValue());
|
||||
assertThat(Validator.INTEGER.validate("", "10.2", null), notNullValue());
|
||||
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "2"), nullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "1"), notNullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "0"), notNullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "10.2.3"), notNullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "2", null), nullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "1", null), notNullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "0", null), notNullValue());
|
||||
assertThat(Validator.INTEGER_GTE_2.validate("", "10.2.3", null), notNullValue());
|
||||
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "2"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "1"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "0"), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "-1"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "10.2"), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "2", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "1", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "0", null), nullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "-1", null), notNullValue());
|
||||
assertThat(Validator.NON_NEGATIVE_INTEGER.validate("", "10.2", null), notNullValue());
|
||||
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "2"), nullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "1"), nullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "0"), notNullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "-1"), notNullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "10.2"), notNullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "2", null), nullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "1", null), nullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "0", null), notNullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "-1", null), notNullValue());
|
||||
assertThat(Validator.POSITIVE_INTEGER.validate("", "10.2", null), notNullValue());
|
||||
|
||||
assertThat(Validator.PERCENTAGE.validate("", "asdasd"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "-1"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "20"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "-1%"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "101%"), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "100%"), nullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "99%"), nullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "0%"), nullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "asdasd", null), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "-1", null), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "20", null), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "-1%", null), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "101%", null), notNullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "100%", null), nullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "99%", null), nullValue());
|
||||
assertThat(Validator.PERCENTAGE.validate("", "0%", null), nullValue());
|
||||
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "asdasd"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20mb"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "-1%"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%"), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "100%"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "99%"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "0%"), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "asdasd", null), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20", null), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "20mb", null), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "-1%", null), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "101%", null), notNullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "100%", null), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "99%", null), nullValue());
|
||||
assertThat(Validator.BYTES_SIZE_OR_PERCENTAGE.validate("", "0%", null), nullValue());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDynamicValidators() throws Exception {
|
||||
DynamicSettings ds = new DynamicSettings();
|
||||
ds.addDynamicSetting("my.test.*", Validator.POSITIVE_INTEGER);
|
||||
String valid = ds.validateDynamicSetting("my.test.setting", "-1");
|
||||
DynamicSettings.Builder ds = new DynamicSettings.Builder();
|
||||
ds.addSetting("my.test.*", Validator.POSITIVE_INTEGER);
|
||||
String valid = ds.build().validateDynamicSetting("my.test.setting", "-1", null);
|
||||
assertThat(valid, equalTo("the value of the setting my.test.setting must be a positive integer"));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,12 +18,15 @@
|
|||
*/
|
||||
package org.elasticsearch.common.inject;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import org.elasticsearch.common.inject.spi.Element;
|
||||
import org.elasticsearch.common.inject.spi.Elements;
|
||||
import org.elasticsearch.common.inject.spi.InstanceBinding;
|
||||
import org.elasticsearch.common.inject.spi.LinkedKeyBinding;
|
||||
import org.elasticsearch.common.inject.spi.ProviderInstanceBinding;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
import java.lang.annotation.Annotation;
|
||||
import java.lang.reflect.Type;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -135,5 +138,27 @@ public abstract class ModuleTestCase extends ESTestCase {
|
|||
assertTrue("Did not find provider for set of " + to.getName(), providerFound);
|
||||
}
|
||||
|
||||
// TODO: add assert for map multibinding
|
||||
public <T> void assertInstanceBinding(Module module, Class<T> to, Predicate<T> tester) {
|
||||
assertInstanceBindingWithAnnotation(module, to, tester, null);
|
||||
}
|
||||
|
||||
public <T> void assertInstanceBindingWithAnnotation(Module module, Class<T> to, Predicate<T> tester, Class<? extends Annotation> annotation) {
|
||||
List<Element> elements = Elements.getElements(module);
|
||||
for (Element element : elements) {
|
||||
if (element instanceof InstanceBinding) {
|
||||
InstanceBinding binding = (InstanceBinding) element;
|
||||
if (to.equals(binding.getKey().getTypeLiteral().getType())) {
|
||||
if (annotation == null || annotation.equals(binding.getKey().getAnnotationType())) {
|
||||
assertTrue(tester.apply(to.cast(binding.getInstance())));
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
StringBuilder s = new StringBuilder();
|
||||
for (Element element : elements) {
|
||||
s.append(element + "\n");
|
||||
}
|
||||
fail("Did not find any instance binding to " + to.getName() + ". Found these bindings:\n" + s);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,7 +29,15 @@ import org.apache.lucene.codecs.Codec;
|
|||
import org.apache.lucene.document.Field;
|
||||
import org.apache.lucene.document.NumericDocValuesField;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.*;
|
||||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.IndexDeletionPolicy;
|
||||
import org.apache.lucene.index.IndexWriterConfig;
|
||||
import org.apache.lucene.index.LiveIndexWriterConfig;
|
||||
import org.apache.lucene.index.LogByteSizeMergePolicy;
|
||||
import org.apache.lucene.index.MergePolicy;
|
||||
import org.apache.lucene.index.NoMergePolicy;
|
||||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.index.TieredMergePolicy;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
|
@ -61,13 +69,19 @@ import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy;
|
|||
import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy;
|
||||
import org.elasticsearch.index.engine.Engine.Searcher;
|
||||
import org.elasticsearch.index.indexing.ShardIndexingService;
|
||||
import org.elasticsearch.index.mapper.*;
|
||||
import org.elasticsearch.index.mapper.ContentPath;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.DocumentMapperParser;
|
||||
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
|
||||
import org.elasticsearch.index.mapper.MapperBuilders;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.Mapping;
|
||||
import org.elasticsearch.index.mapper.MetadataFieldMapper;
|
||||
import org.elasticsearch.index.mapper.ParseContext.Document;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.mapper.object.RootObjectMapper;
|
||||
import org.elasticsearch.index.settings.IndexDynamicSettingsModule;
|
||||
import org.elasticsearch.index.shard.MergeSchedulerConfig;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.ShardUtils;
|
||||
|
@ -93,7 +107,13 @@ import java.nio.charset.Charset;
|
|||
import java.nio.file.DirectoryStream;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
import java.util.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -101,7 +121,12 @@ import java.util.regex.Pattern;
|
|||
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
|
||||
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
|
||||
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
|
||||
import static org.hamcrest.Matchers.*;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.hasKey;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
||||
public class InternalEngineTests extends ESTestCase {
|
||||
|
||||
|
@ -1511,11 +1536,6 @@ public class InternalEngineTests extends ESTestCase {
|
|||
assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName());
|
||||
assertEquals(engine.config().getIndexConcurrency(), indexConcurrency);
|
||||
assertEquals(currentIndexWriterConfig.getMaxThreadStates(), indexConcurrency);
|
||||
|
||||
|
||||
IndexDynamicSettingsModule settings = new IndexDynamicSettingsModule();
|
||||
assertTrue(settings.containsSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH));
|
||||
assertTrue(settings.containsSetting(EngineConfig.INDEX_GC_DELETES_SETTING));
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -19,13 +19,13 @@
|
|||
package org.elasticsearch.test;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.ClusterModule;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.EmptyClusterInfoService;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingNode;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationModule;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
|
||||
import org.elasticsearch.cluster.routing.allocation.allocator.ShardsAllocators;
|
||||
|
@ -74,9 +74,9 @@ public abstract class ESAllocationTestCase extends ESTestCase {
|
|||
|
||||
|
||||
public static AllocationDeciders randomAllocationDeciders(Settings settings, NodeSettingsService nodeSettingsService, Random random) {
|
||||
final List<Class<? extends AllocationDecider>> defaultAllocationDeciders = AllocationModule.DEFAULT_ALLOCATION_DECIDERS;
|
||||
final List<Class<? extends AllocationDecider>> defaultAllocationDeciders = ClusterModule.DEFAULT_ALLOCATION_DECIDERS;
|
||||
final List<AllocationDecider> list = new ArrayList<>();
|
||||
for (Class<? extends AllocationDecider> deciderClass : AllocationModule.DEFAULT_ALLOCATION_DECIDERS) {
|
||||
for (Class<? extends AllocationDecider> deciderClass : ClusterModule.DEFAULT_ALLOCATION_DECIDERS) {
|
||||
try {
|
||||
try {
|
||||
Constructor<? extends AllocationDecider> constructor = deciderClass.getConstructor(Settings.class, NodeSettingsService.class);
|
||||
|
|
Loading…
Reference in New Issue