From 008dc8ec31c405600baaadd1129ad76fd6e13bf0 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Sat, 15 Aug 2015 17:12:06 -0700 Subject: [PATCH] Internal: Flatten ClusterModule and add more tests The ClusterModule contained a couple submodules. This moves the functionality from those modules into ClusterModule. Two of those had to do with DynamicSettings. This change also cleans up how DynamicSettings are built, and enforces they are added, with validators, in ClusterModule. See #12783. --- .../TransportClusterUpdateSettingsAction.java | 4 +- .../elasticsearch/cluster/ClusterModule.java | 298 ++++++++++++++++-- .../MetaDataUpdateSettingsService.java | 2 +- .../routing/allocation/AllocationModule.java | 128 -------- .../ClusterRebalanceAllocationDecider.java | 3 +- .../ClusterDynamicSettingsModule.java | 124 -------- .../cluster/settings/DynamicSettings.java | 49 +-- .../cluster/settings/Validator.java | 77 ++--- .../common/util/ExtensionPoint.java | 5 +- .../discovery/zen/ZenDiscovery.java | 20 +- .../zen/elect/ElectMasterService.java | 18 ++ .../settings/IndexDynamicSettingsModule.java | 139 -------- .../index/settings/IndexSettingsService.java | 4 +- .../node/settings/NodeSettingsService.java | 4 +- .../snapshots/RestoreService.java | 2 +- .../transport/TransportService.java | 4 +- .../BenchmarkNettyLargeMessages.java | 3 - .../cluster/ClusterModuleTests.java | 175 ++++++++++ .../allocation/ShardsAllocatorModuleIT.java | 6 +- .../allocation/AllocationModuleTests.java | 100 ------ .../settings/SettingsValidatorTests.java | 116 +++---- .../common/inject/ModuleTestCase.java | 27 +- .../index/engine/InternalEngineTests.java | 40 ++- .../test/ESAllocationTestCase.java | 6 +- 24 files changed, 656 insertions(+), 698 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationModule.java delete mode 100644 core/src/main/java/org/elasticsearch/cluster/settings/ClusterDynamicSettingsModule.java delete mode 100644 core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java create mode 100644 core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java delete mode 100644 core/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationModuleTests.java diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index ef64bbdd05e..ff916357037 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -183,7 +183,7 @@ public class TransportClusterUpdateSettingsAction extends TransportMasterNodeAct transientSettings.put(currentState.metaData().transientSettings()); for (Map.Entry 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 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()); diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java index e57d9562f78..7a4a3e0722d 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -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> 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> indexTemplateFilters = new HashSet<>(); + private final DynamicSettings.Builder clusterDynamicSettings = new DynamicSettings.Builder(); + private final DynamicSettings.Builder indexDynamicSettings = new DynamicSettings.Builder(); + private final ExtensionPoint.TypeExtensionPoint shardsAllocators = new ExtensionPoint.TypeExtensionPoint<>("shards_allocator", ShardsAllocator.class); + private final ExtensionPoint.SetExtensionPoint allocationDeciders = new ExtensionPoint.SetExtensionPoint<>("allocation_decider", AllocationDecider.class, AllocationDeciders.class); + private final ExtensionPoint.SetExtensionPoint indexTemplateFilters = new ExtensionPoint.SetExtensionPoint<>("index_template_filter", IndexTemplateFilter.class); // pkg private so tests can mock Class clusterInfoServiceImpl = InternalClusterInfoService.class; public ClusterModule(Settings settings) { this.settings = settings; + + registerBuiltinClusterSettings(); + registerBuiltinIndexSettings(); + + for (Class 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 allocationDecider) { + allocationDeciders.registerExtension(allocationDecider); + } + + public void registerShardsAllocator(String name, Class clazz) { + shardsAllocators.registerExtension(name, clazz); } public void registerIndexTemplateFilter(Class indexTemplateFilter) { - indexTemplateFilters.add(indexTemplateFilter); - } - - @Override - public Iterable 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 mbinder = Multibinder.newSetBinder(binder(), IndexTemplateFilter.class); - for (Class indexTemplateFilter : indexTemplateFilters) { - mbinder.addBinding().to(indexTemplateFilter); - } } } \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java index 9fbe2c5e06c..46656338862 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataUpdateSettingsService.java @@ -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); } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationModule.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationModule.java deleted file mode 100644 index 62e14ec0471..00000000000 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationModule.java +++ /dev/null @@ -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. - *
    - *
  • An {@link AllocationDecider} decides *when* an allocation should be attempted.
  • - *
  • A {@link ShardsAllocator} determins *how* an allocation takes place
  • - *
- */ -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> 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 shardsAllocators = new ExtensionPoint.TypeExtensionPoint<>("shards_allocator", ShardsAllocator.class); - private final ExtensionPoint.SetExtensionPoint allocationDeciders = new ExtensionPoint.SetExtensionPoint<>("allocation_decider", AllocationDecider.class, AllocationDeciders.class); - - public AllocationModule(Settings settings) { - this.settings = settings; - for (Class 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 allocationDecider) { - allocationDeciders.registerExtension(allocationDecider); - } - - /** Register a custom shard allocator with the given name */ - public void registerShardAllocator(String name, Class 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(); - } - -} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java index b057307cafb..00f6575287f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/cluster/settings/ClusterDynamicSettingsModule.java b/core/src/main/java/org/elasticsearch/cluster/settings/ClusterDynamicSettingsModule.java deleted file mode 100644 index a68ef64f44a..00000000000 --- a/core/src/main/java/org/elasticsearch/cluster/settings/ClusterDynamicSettingsModule.java +++ /dev/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.of(null)); - } -} diff --git a/core/src/main/java/org/elasticsearch/cluster/settings/DynamicSettings.java b/core/src/main/java/org/elasticsearch/cluster/settings/DynamicSettings.java index 1848a9251f7..c4137fcc07e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/settings/DynamicSettings.java +++ b/core/src/main/java/org/elasticsearch/cluster/settings/DynamicSettings.java @@ -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 dynamicSettings = ImmutableMap.of(); + private final Map dynamicSettings; + public static class Builder { + private Map 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 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 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 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 updatedSettings = MapBuilder.newMapBuilder(dynamicSettings); - for (String setting : settings) { - updatedSettings.put(setting, Validator.EMPTY); - } - dynamicSettings = updatedSettings.immutableMap(); - } - } diff --git a/core/src/main/java/org/elasticsearch/cluster/settings/Validator.java b/core/src/main/java/org/elasticsearch/cluster/settings/Validator.java index 822b8c7a129..12049abed9b 100644 --- a/core/src/main/java/org/elasticsearch/cluster/settings/Validator.java +++ b/core/src/main/java/org/elasticsearch/cluster/settings/Validator.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java b/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java index 435c3ae4066..5414c4eb7a6 100644 --- a/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java +++ b/core/src/main/java/org/elasticsearch/common/util/ExtensionPoint.java @@ -39,7 +39,7 @@ public abstract class ExtensionPoint { /** * 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 { * @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(); } diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 8917e674624..d1b790672ee 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -150,7 +150,7 @@ public class ZenDiscovery extends AbstractLifecycleComponent 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 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 diff --git a/core/src/main/java/org/elasticsearch/discovery/zen/elect/ElectMasterService.java b/core/src/main/java/org/elasticsearch/discovery/zen/elect/ElectMasterService.java index 6d80d2589e0..ce65861df61 100644 --- a/core/src/main/java/org/elasticsearch/discovery/zen/elect/ElectMasterService.java +++ b/core/src/main/java/org/elasticsearch/discovery/zen/elect/ElectMasterService.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java b/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java deleted file mode 100644 index c9fe0b4207f..00000000000 --- a/core/src/main/java/org/elasticsearch/index/settings/IndexDynamicSettingsModule.java +++ /dev/null @@ -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 true iff the given setting is in the dynamic settings map. Otherwise false. - */ - public boolean containsSetting(String setting) { - return indexDynamicSettings.hasDynamicSetting(setting); - } -} diff --git a/core/src/main/java/org/elasticsearch/index/settings/IndexSettingsService.java b/core/src/main/java/org/elasticsearch/index/settings/IndexSettingsService.java index 62964b55414..39d1437983f 100644 --- a/core/src/main/java/org/elasticsearch/index/settings/IndexSettingsService.java +++ b/core/src/main/java/org/elasticsearch/index/settings/IndexSettingsService.java @@ -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); } } \ No newline at end of file diff --git a/core/src/main/java/org/elasticsearch/node/settings/NodeSettingsService.java b/core/src/main/java/org/elasticsearch/node/settings/NodeSettingsService.java index 8a96e9f1b8f..dbe6a33172b 100644 --- a/core/src/main/java/org/elasticsearch/node/settings/NodeSettingsService.java +++ b/core/src/main/java/org/elasticsearch/node/settings/NodeSettingsService.java @@ -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); } } diff --git a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java index d88d0381b3e..d8c6fdc8b07 100644 --- a/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/core/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -392,7 +392,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis Settings.Builder persistentSettings = Settings.settingsBuilder().put(); for (Map.Entry 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; diff --git a/core/src/main/java/org/elasticsearch/transport/TransportService.java b/core/src/main/java/org/elasticsearch/transport/TransportService.java index b4ff29b76d2..81599996a6f 100644 --- a/core/src/main/java/org/elasticsearch/transport/TransportService.java +++ b/core/src/main/java/org/elasticsearch/transport/TransportService.java @@ -130,9 +130,7 @@ public class TransportService extends AbstractLifecycleComponent() { + @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() { + @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); + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/allocation/ShardsAllocatorModuleIT.java b/core/src/test/java/org/elasticsearch/cluster/allocation/ShardsAllocatorModuleIT.java index 75dfac53138..1faa82ad1b8 100644 --- a/core/src/test/java/org/elasticsearch/cluster/allocation/ShardsAllocatorModuleIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/allocation/ShardsAllocatorModuleIT.java @@ -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); } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationModuleTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationModuleTests.java deleted file mode 100644 index 7b57ef07190..00000000000 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationModuleTests.java +++ /dev/null @@ -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); - } -} diff --git a/core/src/test/java/org/elasticsearch/cluster/settings/SettingsValidatorTests.java b/core/src/test/java/org/elasticsearch/cluster/settings/SettingsValidatorTests.java index fdb54cbb6f7..1e041aae1b7 100644 --- a/core/src/test/java/org/elasticsearch/cluster/settings/SettingsValidatorTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/settings/SettingsValidatorTests.java @@ -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")); } } diff --git a/core/src/test/java/org/elasticsearch/common/inject/ModuleTestCase.java b/core/src/test/java/org/elasticsearch/common/inject/ModuleTestCase.java index 60c3ca126d5..323b9f5ca4a 100644 --- a/core/src/test/java/org/elasticsearch/common/inject/ModuleTestCase.java +++ b/core/src/test/java/org/elasticsearch/common/inject/ModuleTestCase.java @@ -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 void assertInstanceBinding(Module module, Class to, Predicate tester) { + assertInstanceBindingWithAnnotation(module, to, tester, null); + } + + public void assertInstanceBindingWithAnnotation(Module module, Class to, Predicate tester, Class annotation) { + List 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); + } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index bad431f67e5..974a4a21557 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -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 diff --git a/core/src/test/java/org/elasticsearch/test/ESAllocationTestCase.java b/core/src/test/java/org/elasticsearch/test/ESAllocationTestCase.java index e238272795c..a4f8f3b2fa8 100644 --- a/core/src/test/java/org/elasticsearch/test/ESAllocationTestCase.java +++ b/core/src/test/java/org/elasticsearch/test/ESAllocationTestCase.java @@ -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> defaultAllocationDeciders = AllocationModule.DEFAULT_ALLOCATION_DECIDERS; + final List> defaultAllocationDeciders = ClusterModule.DEFAULT_ALLOCATION_DECIDERS; final List list = new ArrayList<>(); - for (Class deciderClass : AllocationModule.DEFAULT_ALLOCATION_DECIDERS) { + for (Class deciderClass : ClusterModule.DEFAULT_ALLOCATION_DECIDERS) { try { try { Constructor constructor = deciderClass.getConstructor(Settings.class, NodeSettingsService.class);