Browse Source

Simplify watching settings in deciders (#93075)

Ievgen Degtiarenko 2 years ago
parent
commit
8b92cd4bf5
25 changed files with 115 additions and 136 deletions
  1. 6 6
      server/src/main/java/org/elasticsearch/cluster/ClusterModule.java
  2. 2 1
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java
  3. 2 4
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java
  4. 3 5
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ConcurrentRebalanceAllocationDecider.java
  5. 3 5
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java
  6. 2 4
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java
  7. 2 4
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java
  8. 4 10
      server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java
  9. 4 3
      server/src/test/java/org/elasticsearch/action/fieldcaps/RequestDispatcherTests.java
  10. 1 1
      server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java
  11. 3 5
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationServiceTests.java
  12. 2 3
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java
  13. 2 5
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java
  14. 3 9
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java
  15. 10 1
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java
  16. 16 16
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerTests.java
  17. 4 3
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java
  18. 5 4
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java
  19. 3 2
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java
  20. 3 2
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/NodeReplacementAllocationDeciderTests.java
  21. 3 6
      server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/NodeShutdownAllocationDeciderTests.java
  22. 2 5
      server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java
  23. 3 2
      server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java
  24. 24 28
      test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java
  25. 3 2
      x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java

+ 6 - 6
server/src/main/java/org/elasticsearch/cluster/ClusterModule.java

@@ -343,19 +343,19 @@ public class ClusterModule extends AbstractModule {
         addAllocationDecider(deciders, new ResizeAllocationDecider());
         addAllocationDecider(deciders, new ReplicaAfterPrimaryActiveAllocationDecider());
         addAllocationDecider(deciders, new RebalanceOnlyWhenActiveAllocationDecider());
-        addAllocationDecider(deciders, new ClusterRebalanceAllocationDecider(settings, clusterSettings));
-        addAllocationDecider(deciders, new ConcurrentRebalanceAllocationDecider(settings, clusterSettings));
-        addAllocationDecider(deciders, new EnableAllocationDecider(settings, clusterSettings));
+        addAllocationDecider(deciders, new ClusterRebalanceAllocationDecider(clusterSettings));
+        addAllocationDecider(deciders, new ConcurrentRebalanceAllocationDecider(clusterSettings));
+        addAllocationDecider(deciders, new EnableAllocationDecider(clusterSettings));
         addAllocationDecider(deciders, new NodeVersionAllocationDecider());
         addAllocationDecider(deciders, new SnapshotInProgressAllocationDecider());
         addAllocationDecider(deciders, new RestoreInProgressAllocationDecider());
         addAllocationDecider(deciders, new NodeShutdownAllocationDecider());
         addAllocationDecider(deciders, new NodeReplacementAllocationDecider());
         addAllocationDecider(deciders, new FilterAllocationDecider(settings, clusterSettings));
-        addAllocationDecider(deciders, new SameShardAllocationDecider(settings, clusterSettings));
+        addAllocationDecider(deciders, new SameShardAllocationDecider(clusterSettings));
         addAllocationDecider(deciders, new DiskThresholdDecider(settings, clusterSettings));
-        addAllocationDecider(deciders, new ThrottlingAllocationDecider(settings, clusterSettings));
-        addAllocationDecider(deciders, new ShardsLimitAllocationDecider(settings, clusterSettings));
+        addAllocationDecider(deciders, new ThrottlingAllocationDecider(clusterSettings));
+        addAllocationDecider(deciders, new ShardsLimitAllocationDecider(clusterSettings));
         addAllocationDecider(deciders, new AwarenessAllocationDecider(settings, clusterSettings));
 
         clusterPlugins.stream()

+ 2 - 1
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java

@@ -59,6 +59,7 @@ import java.util.function.BiFunction;
 import java.util.stream.StreamSupport;
 
 import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 
 /**
  * The {@link BalancedShardsAllocator} re-balances the nodes allocations
@@ -128,7 +129,7 @@ public class BalancedShardsAllocator implements ShardsAllocator {
     }
 
     public BalancedShardsAllocator(Settings settings) {
-        this(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), WriteLoadForecaster.DEFAULT);
+        this(createBuiltInClusterSettings(settings), WriteLoadForecaster.DEFAULT);
     }
 
     public BalancedShardsAllocator(ClusterSettings clusterSettings) {

+ 2 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ClusterRebalanceAllocationDecider.java

@@ -16,7 +16,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 
 import java.util.Locale;
 
@@ -89,10 +88,9 @@ public class ClusterRebalanceAllocationDecider extends AllocationDecider {
 
     private volatile ClusterRebalanceType type;
 
-    public ClusterRebalanceAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        type = CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.get(settings);
+    public ClusterRebalanceAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, this::setType);
         logger.debug("using [{}] with [{}]", CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, type);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING, this::setType);
     }
 
     private void setType(ClusterRebalanceType type) {

+ 3 - 5
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ConcurrentRebalanceAllocationDecider.java

@@ -15,7 +15,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 
 /**
  * Similar to the {@link ClusterRebalanceAllocationDecider} this
@@ -44,13 +43,12 @@ public class ConcurrentRebalanceAllocationDecider extends AllocationDecider {
     );
     private volatile int clusterConcurrentRebalance;
 
-    public ConcurrentRebalanceAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        this.clusterConcurrentRebalance = CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.get(settings);
-        logger.debug("using [cluster_concurrent_rebalance] with [{}]", clusterConcurrentRebalance);
-        clusterSettings.addSettingsUpdateConsumer(
+    public ConcurrentRebalanceAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(
             CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING,
             this::setClusterConcurrentRebalance
         );
+        logger.debug("using [cluster_concurrent_rebalance] with [{}]", clusterConcurrentRebalance);
     }
 
     private void setClusterConcurrentRebalance(int concurrentRebalance) {

+ 3 - 5
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationDecider.java

@@ -84,11 +84,9 @@ public class EnableAllocationDecider extends AllocationDecider {
     private volatile Rebalance enableRebalance;
     private volatile Allocation enableAllocation;
 
-    public EnableAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        this.enableAllocation = CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.get(settings);
-        this.enableRebalance = CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.get(settings);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, this::setEnableAllocation);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, this::setEnableRebalance);
+    public EnableAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING, this::setEnableAllocation);
+        clusterSettings.initializeAndWatch(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING, this::setEnableRebalance);
     }
 
     private void setEnableRebalance(Rebalance enableRebalance) {

+ 2 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java

@@ -16,7 +16,6 @@ import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 
 /**
  * An allocation decider that prevents multiple instances of the same shard to
@@ -46,9 +45,8 @@ public class SameShardAllocationDecider extends AllocationDecider {
 
     private volatile boolean sameHost;
 
-    public SameShardAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        this.sameHost = CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING.get(settings);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, this::setSameHost);
+    public SameShardAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(CLUSTER_ROUTING_ALLOCATION_SAME_HOST_SETTING, this::setSameHost);
     }
 
     /**

+ 2 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ShardsLimitAllocationDecider.java

@@ -16,7 +16,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 
 import java.util.function.BiPredicate;
 
@@ -70,9 +69,8 @@ public class ShardsLimitAllocationDecider extends AllocationDecider {
         Property.NodeScope
     );
 
-    public ShardsLimitAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        this.clusterShardLimit = CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING.get(settings);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING, this::setClusterShardLimit);
+    public ShardsLimitAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(CLUSTER_TOTAL_SHARDS_PER_NODE_SETTING, this::setClusterShardLimit);
     }
 
     private void setClusterShardLimit(int clusterShardLimit) {

+ 4 - 10
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/ThrottlingAllocationDecider.java

@@ -18,7 +18,6 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Setting.Property;
-import org.elasticsearch.common.settings.Settings;
 
 import static org.elasticsearch.cluster.routing.allocation.decider.Decision.THROTTLE;
 import static org.elasticsearch.cluster.routing.allocation.decider.Decision.YES;
@@ -81,24 +80,19 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
     private volatile int concurrentIncomingRecoveries;
     private volatile int concurrentOutgoingRecoveries;
 
-    public ThrottlingAllocationDecider(Settings settings, ClusterSettings clusterSettings) {
-        this.primariesInitialRecoveries = CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING.get(settings);
-        concurrentIncomingRecoveries = CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.get(settings);
-        concurrentOutgoingRecoveries = CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.get(settings);
-
-        clusterSettings.addSettingsUpdateConsumer(
+    public ThrottlingAllocationDecider(ClusterSettings clusterSettings) {
+        clusterSettings.initializeAndWatch(
             CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING,
             this::setPrimariesInitialRecoveries
         );
-        clusterSettings.addSettingsUpdateConsumer(
+        clusterSettings.initializeAndWatch(
             CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING,
             this::setConcurrentIncomingRecoverries
         );
-        clusterSettings.addSettingsUpdateConsumer(
+        clusterSettings.initializeAndWatch(
             CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING,
             this::setConcurrentOutgoingRecoverries
         );
-
         logger.debug(
             "using node_concurrent_outgoing_recoveries [{}], node_concurrent_incoming_recoveries [{}], "
                 + "node_initial_primaries_recoveries [{}]",

+ 4 - 3
server/src/test/java/org/elasticsearch/action/fieldcaps/RequestDispatcherTests.java

@@ -78,6 +78,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.elasticsearch.test.VersionUtils.randomVersion;
 import static org.hamcrest.Matchers.anEmptyMap;
 import static org.hamcrest.Matchers.equalTo;
@@ -904,10 +905,10 @@ public class RequestDispatcherTests extends ESAllocationTestCase {
             .routingTable(routingTable.build())
             .build();
         final Settings settings = Settings.EMPTY;
-        final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+        final ClusterSettings clusterSettings = createBuiltInClusterSettings(settings);
         final ArrayList<AllocationDecider> deciders = new ArrayList<>();
-        deciders.add(new EnableAllocationDecider(settings, clusterSettings));
-        deciders.add(new SameShardAllocationDecider(settings, clusterSettings));
+        deciders.add(new EnableAllocationDecider(clusterSettings));
+        deciders.add(new SameShardAllocationDecider(clusterSettings));
         deciders.add(new ReplicaAfterPrimaryActiveAllocationDecider());
         Collections.shuffle(deciders, random());
         final MockAllocationService allocationService = new MockAllocationService(

+ 1 - 1
server/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java

@@ -153,7 +153,7 @@ public class ClusterModuleTests extends ModuleTestCase {
             () -> new ClusterModule(Settings.EMPTY, clusterService, Collections.<ClusterPlugin>singletonList(new ClusterPlugin() {
                 @Override
                 public Collection<AllocationDecider> createAllocationDeciders(Settings settings, ClusterSettings clusterSettings) {
-                    return Collections.singletonList(new EnableAllocationDecider(settings, clusterSettings));
+                    return Collections.singletonList(new EnableAllocationDecider(clusterSettings));
                 }
             }), clusterInfoService, null, threadPool, EmptySystemIndices.INSTANCE, WriteLoadForecaster.DEFAULT)
         );

+ 3 - 5
server/src/test/java/org/elasticsearch/cluster/routing/allocation/AllocationServiceTests.java

@@ -50,6 +50,7 @@ import static org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus.
 import static org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING;
 import static org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING;
 import static org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
@@ -109,13 +110,10 @@ public class AllocationServiceTests extends ESTestCase {
             .put(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 1)
             .put(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), Integer.MAX_VALUE)
             .build();
-        final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+        final ClusterSettings clusterSettings = createBuiltInClusterSettings(settings);
         final AllocationService allocationService = new AllocationService(
             new AllocationDeciders(
-                Arrays.asList(
-                    new SameShardAllocationDecider(settings, clusterSettings),
-                    new ThrottlingAllocationDecider(settings, clusterSettings)
-                )
+                Arrays.asList(new SameShardAllocationDecider(clusterSettings), new ThrottlingAllocationDecider(clusterSettings))
             ),
             new ShardsAllocator() {
                 @Override

+ 2 - 3
server/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java

@@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllo
 import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider;
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.test.gateway.TestGatewayAllocator;
 import org.hamcrest.Matchers;
 
 import java.util.stream.Collectors;
@@ -61,7 +60,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
         settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance);
         settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceThreshold);
 
-        AllocationService strategy = createAllocationService(settings.build(), new TestGatewayAllocator());
+        AllocationService strategy = createAllocationService(settings.build());
 
         ClusterState clusterState = initCluster(strategy);
         assertIndexBalance(
@@ -112,7 +111,7 @@ public class BalanceConfigurationTests extends ESAllocationTestCase {
         settings.put(BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING.getKey(), replicaBalance);
         settings.put(BalancedShardsAllocator.THRESHOLD_SETTING.getKey(), balanceThreshold);
 
-        AllocationService strategy = createAllocationService(settings.build(), new TestGatewayAllocator());
+        AllocationService strategy = createAllocationService(settings.build());
 
         ClusterState clusterState = initCluster(strategy);
         assertReplicaBalance(

+ 2 - 5
server/src/test/java/org/elasticsearch/cluster/routing/allocation/RandomAllocationDeciderTests.java

@@ -28,7 +28,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
 import org.elasticsearch.cluster.routing.allocation.decider.ReplicaAfterPrimaryActiveAllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
-import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.snapshots.EmptySnapshotsInfoService;
 import org.elasticsearch.test.gateway.TestGatewayAllocator;
@@ -41,6 +40,7 @@ import java.util.Random;
 
 import static org.elasticsearch.cluster.routing.RoutingNodesHelper.shardsWithState;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.equalTo;
 
 public class RandomAllocationDeciderTests extends ESAllocationTestCase {
@@ -55,10 +55,7 @@ public class RandomAllocationDeciderTests extends ESAllocationTestCase {
             new AllocationDeciders(
                 new HashSet<>(
                     Arrays.asList(
-                        new SameShardAllocationDecider(
-                            Settings.EMPTY,
-                            new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-                        ),
+                        new SameShardAllocationDecider(createBuiltInClusterSettings()),
                         new ReplicaAfterPrimaryActiveAllocationDecider(),
                         randomAllocationDecider
                     )

+ 3 - 9
server/src/test/java/org/elasticsearch/cluster/routing/allocation/SameShardRoutingTests.java

@@ -27,7 +27,6 @@ import org.elasticsearch.cluster.routing.TestShardRouting;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
 import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
-import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.core.Strings;
 import org.elasticsearch.index.Index;
@@ -43,6 +42,7 @@ import static org.elasticsearch.cluster.routing.RoutingNodesHelper.shardsWithSta
 import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
 import static org.elasticsearch.cluster.routing.allocation.RoutingNodesUtils.numberOfShardsOfType;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 
@@ -206,10 +206,7 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
         } else {
             final ShardRouting unassignedShard = unassignedShards.get(0);
 
-            final SameShardAllocationDecider decider = new SameShardAllocationDecider(
-                sameHostSetting,
-                new ClusterSettings(sameHostSetting, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-            );
+            final SameShardAllocationDecider decider = new SameShardAllocationDecider(createBuiltInClusterSettings(sameHostSetting));
 
             final RoutingNode emptyNode = clusterState.getRoutingNodes()
                 .stream()
@@ -308,10 +305,7 @@ public class SameShardRoutingTests extends ESAllocationTestCase {
     }
 
     public void testForceAllocatePrimaryOnSameNodeNotAllowed() {
-        SameShardAllocationDecider decider = new SameShardAllocationDecider(
-            Settings.EMPTY,
-            new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-        );
+        SameShardAllocationDecider decider = new SameShardAllocationDecider(createBuiltInClusterSettings());
         ClusterState clusterState = ClusterStateCreationUtils.state("idx", randomIntBetween(2, 4), 1);
         Index index = clusterState.getMetadata().index("idx").getIndex();
         ShardRouting primaryShard = clusterState.routingTable().index(index).shard(0).primaryShard();

+ 10 - 1
server/src/test/java/org/elasticsearch/cluster/routing/allocation/ThrottlingAllocationTests.java

@@ -14,6 +14,7 @@ import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.ESAllocationTestCase;
+import org.elasticsearch.cluster.EmptyClusterInfoService;
 import org.elasticsearch.cluster.RestoreInProgress;
 import org.elasticsearch.cluster.TestShardRoutingRoleStrategies;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
@@ -70,6 +71,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
                 .put("cluster.routing.allocation.node_initial_primaries_recoveries", 3)
                 .build(),
             gatewayAllocator,
+            EmptyClusterInfoService.INSTANCE,
             snapshotsInfoService
         );
 
@@ -128,6 +130,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
                 .put("cluster.routing.allocation.node_initial_primaries_recoveries", 3)
                 .build(),
             gatewayAllocator,
+            EmptyClusterInfoService.INSTANCE,
             snapshotsInfoService
         );
 
@@ -191,7 +194,12 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
             .put("cluster.routing.allocation.node_initial_primaries_recoveries", 5)
             .put("cluster.routing.allocation.cluster_concurrent_rebalance", 5)
             .build();
-        AllocationService strategy = createAllocationService(settings, gatewayAllocator, snapshotsInfoService);
+        AllocationService strategy = createAllocationService(
+            settings,
+            gatewayAllocator,
+            EmptyClusterInfoService.INSTANCE,
+            snapshotsInfoService
+        );
         logger.info("Building initial routing table");
 
         Metadata metadata = Metadata.builder()
@@ -250,6 +258,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
         AllocationService strategy = createAllocationService(
             Settings.builder().put("cluster.routing.allocation.node_concurrent_outgoing_recoveries", 1).build(),
             gatewayAllocator,
+            EmptyClusterInfoService.INSTANCE,
             snapshotsInfoService
         );
 

+ 16 - 16
server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/DesiredBalanceReconcilerTests.java

@@ -227,9 +227,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
 
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings),
+            new ThrottlingAllocationDecider(clusterSettings),
             new AllocationDecider() {
                 @Override
                 public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
@@ -310,9 +310,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
 
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings)
+            new ThrottlingAllocationDecider(clusterSettings)
         );
 
         final var stateWithInitializingPrimaries = startInitializingShardsAndReroute(allocationService, clusterState);
@@ -400,9 +400,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
 
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings),
+            new ThrottlingAllocationDecider(clusterSettings),
             new AllocationDecider() {
                 @Override
                 public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
@@ -508,7 +508,7 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
         final var desiredBalance = desiredBalance(clusterState, (shardId, nodeId) -> true);
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider()
         );
 
@@ -601,7 +601,7 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
             () -> clusterInfo,
             () -> snapshotShardSizeInfo,
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider()
         );
 
@@ -639,7 +639,7 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
         final var desiredBalance = desiredBalance(clusterState, (shardId, nodeId) -> true);
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
             new AllocationDecider() {
                 @Override
@@ -698,7 +698,7 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
         final var desiredBalance = desiredBalance(clusterState, (shardId, nodeId) -> true);
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
             new AllocationDecider() {
                 @Override
@@ -756,9 +756,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
 
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings),
+            new ThrottlingAllocationDecider(clusterSettings),
             new AllocationDecider() {
                 @Override
                 public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
@@ -834,9 +834,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
         final var desiredBalance = new AtomicReference<>(desiredBalance(clusterState, (shardId, nodeId) -> true));
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance.get()),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings),
+            new ThrottlingAllocationDecider(clusterSettings),
             new FilterAllocationDecider(settings, clusterSettings),
             new NodeShutdownAllocationDecider(),
             new NodeReplacementAllocationDecider(),
@@ -955,9 +955,9 @@ public class DesiredBalanceReconcilerTests extends ESTestCase {
         );
         final var allocationService = createTestAllocationService(
             routingAllocation -> reconcile(routingAllocation, desiredBalance.get()),
-            new SameShardAllocationDecider(settings, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
-            new ThrottlingAllocationDecider(settings, clusterSettings),
+            new ThrottlingAllocationDecider(clusterSettings),
             new AllocationDecider() {
                 @Override
                 public Decision canRebalance(RoutingAllocation allocation) {

+ 4 - 3
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java

@@ -73,6 +73,7 @@ import static org.elasticsearch.cluster.routing.ShardRoutingState.RELOCATING;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
 import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.is;
@@ -1273,15 +1274,15 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
     }
 
     private DiskThresholdDecider createDiskThresholdDecider(Settings settings) {
-        return new DiskThresholdDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+        return new DiskThresholdDecider(settings, createBuiltInClusterSettings(settings));
     }
 
     private SameShardAllocationDecider createSameShardAllocationDecider(Settings settings) {
-        return new SameShardAllocationDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+        return new SameShardAllocationDecider(createBuiltInClusterSettings(settings));
     }
 
     private EnableAllocationDecider createEnableAllocationDecider(Settings settings) {
-        return new EnableAllocationDecider(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS));
+        return new EnableAllocationDecider(createBuiltInClusterSettings(settings));
     }
 
     /**

+ 5 - 4
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/EnableAllocationShortCircuitTests.java

@@ -198,15 +198,16 @@ public class EnableAllocationShortCircuitTests extends ESAllocationTestCase {
         assertThat(plugin.canAllocateAttempts, equalTo(0));
     }
 
-    private static AllocationService createAllocationService(Settings.Builder settings, ClusterPlugin plugin) {
-        final ClusterSettings emptyClusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+    private static AllocationService createAllocationService(Settings.Builder settingsBuilder, ClusterPlugin plugin) {
+        var settings = settingsBuilder.build();
+        var clusterSettings = ClusterSettings.createBuiltInClusterSettings(settings);
         List<AllocationDecider> deciders = new ArrayList<>(
-            ClusterModule.createAllocationDeciders(settings.build(), emptyClusterSettings, Collections.singletonList(plugin))
+            ClusterModule.createAllocationDeciders(settings, clusterSettings, Collections.singletonList(plugin))
         );
         return new MockAllocationService(
             new AllocationDeciders(deciders),
             new TestGatewayAllocator(),
-            new BalancedShardsAllocator(Settings.EMPTY),
+            new BalancedShardsAllocator(clusterSettings),
             EmptyClusterInfoService.INSTANCE,
             EmptySnapshotsInfoService.INSTANCE
         );

+ 3 - 2
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/FilterAllocationDeciderTests.java

@@ -45,17 +45,18 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_RESIZE_SOUR
 import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.STARTED;
 import static org.elasticsearch.cluster.routing.ShardRoutingState.UNASSIGNED;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.equalTo;
 
 public class FilterAllocationDeciderTests extends ESAllocationTestCase {
 
     public void testFilterInitialRecovery() {
-        ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+        ClusterSettings clusterSettings = createBuiltInClusterSettings();
         FilterAllocationDecider filterAllocationDecider = new FilterAllocationDecider(Settings.EMPTY, clusterSettings);
         AllocationDeciders allocationDeciders = new AllocationDeciders(
             Arrays.asList(
                 filterAllocationDecider,
-                new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
+                new SameShardAllocationDecider(clusterSettings),
                 new ReplicaAfterPrimaryActiveAllocationDecider()
             )
         );

+ 3 - 2
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/NodeReplacementAllocationDeciderTests.java

@@ -39,6 +39,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 
@@ -53,12 +54,12 @@ public class NodeReplacementAllocationDeciderTests extends ESAllocationTestCase
         new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "index created"),
         ShardRouting.Role.DEFAULT
     );
-    private final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+    private final ClusterSettings clusterSettings = createBuiltInClusterSettings();
     private NodeReplacementAllocationDecider decider = new NodeReplacementAllocationDecider();
     private final AllocationDeciders allocationDeciders = new AllocationDeciders(
         Arrays.asList(
             decider,
-            new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider(),
             new NodeShutdownAllocationDecider()
         )

+ 3 - 6
server/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/NodeShutdownAllocationDeciderTests.java

@@ -39,6 +39,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashMap;
 
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.equalTo;
 
 public class NodeShutdownAllocationDeciderTests extends ESAllocationTestCase {
@@ -50,14 +51,10 @@ public class NodeShutdownAllocationDeciderTests extends ESAllocationTestCase {
         new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "index created"),
         ShardRouting.Role.DEFAULT
     );
-    private final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+    private final ClusterSettings clusterSettings = createBuiltInClusterSettings();
     private final NodeShutdownAllocationDecider decider = new NodeShutdownAllocationDecider();
     private final AllocationDeciders allocationDeciders = new AllocationDeciders(
-        Arrays.asList(
-            decider,
-            new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
-            new ReplicaAfterPrimaryActiveAllocationDecider()
-        )
+        Arrays.asList(decider, new SameShardAllocationDecider(clusterSettings), new ReplicaAfterPrimaryActiveAllocationDecider())
     );
     private final AllocationService service = new AllocationService(
         allocationDeciders,

+ 2 - 5
server/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java

@@ -33,7 +33,6 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider;
 import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
 import org.elasticsearch.cluster.routing.allocation.decider.Decision;
 import org.elasticsearch.cluster.routing.allocation.decider.SameShardAllocationDecider;
-import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.util.set.Sets;
 import org.elasticsearch.core.TimeValue;
@@ -60,6 +59,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 
 import static java.util.Collections.unmodifiableMap;
 import static org.elasticsearch.cluster.routing.RoutingNodesHelper.shardsWithState;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.hasSize;
@@ -369,10 +369,7 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase {
             new AllocationDeciders(
                 Arrays.asList(
                     new TestAllocateDecision(Decision.YES),
-                    new SameShardAllocationDecider(
-                        Settings.EMPTY,
-                        new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-                    ),
+                    new SameShardAllocationDecider(createBuiltInClusterSettings()),
                     new AllocationDecider() {
                         @Override
                         public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {

+ 3 - 2
server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java

@@ -113,6 +113,7 @@ import java.util.function.Function;
 
 import static com.carrotsearch.randomizedtesting.RandomizedTest.getRandom;
 import static java.util.stream.Collectors.toMap;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.elasticsearch.env.Environment.PATH_HOME_SETTING;
 import static org.elasticsearch.test.CheckedFunctionUtils.anyCheckedFunction;
 import static org.elasticsearch.test.ESTestCase.between;
@@ -142,12 +143,12 @@ public class ClusterStateChanges {
 
     @SuppressWarnings("unchecked")
     public ClusterStateChanges(NamedXContentRegistry xContentRegistry, ThreadPool threadPool) {
-        ClusterSettings clusterSettings = new ClusterSettings(SETTINGS, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+        ClusterSettings clusterSettings = createBuiltInClusterSettings(SETTINGS);
         allocationService = new AllocationService(
             new AllocationDeciders(
                 new HashSet<>(
                     Arrays.asList(
-                        new SameShardAllocationDecider(SETTINGS, clusterSettings),
+                        new SameShardAllocationDecider(clusterSettings),
                         new ReplicaAfterPrimaryActiveAllocationDecider(),
                         new RandomAllocationDeciderTests.RandomAllocationDecider(getRandom())
                     )

+ 24 - 28
test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java

@@ -89,7 +89,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
     }
 
     public static MockAllocationService createAllocationService(Settings settings) {
-        return createAllocationService(settings, EMPTY_CLUSTER_SETTINGS);
+        return createAllocationService(settings, createBuiltInClusterSettings(settings));
     }
 
     public static MockAllocationService createAllocationService(Settings settings, ClusterSettings clusterSettings) {
@@ -113,7 +113,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
     private static DesiredBalanceShardsAllocator createDesiredBalanceShardsAllocator(Settings settings) {
         var queue = new DeterministicTaskQueue();
         return new DesiredBalanceShardsAllocator(
-            createBuiltInClusterSettings(),
+            createBuiltInClusterSettings(settings),
             new BalancedShardsAllocator(settings),
             queue.getThreadPool(),
             null,
@@ -141,34 +141,35 @@ public abstract class ESAllocationTestCase extends ESTestCase {
         };
     }
 
-    public static MockAllocationService createAllocationService(Settings settings, ClusterInfoService clusterInfoService) {
-        return new MockAllocationService(
-            randomAllocationDeciders(settings, EMPTY_CLUSTER_SETTINGS),
-            new TestGatewayAllocator(),
-            new BalancedShardsAllocator(settings),
-            clusterInfoService,
+    public static MockAllocationService createAllocationService(Settings settings, GatewayAllocator gatewayAllocator) {
+        return createAllocationService(
+            settings,
+            gatewayAllocator,
+            EmptyClusterInfoService.INSTANCE,
             SNAPSHOT_INFO_SERVICE_WITH_NO_SHARD_SIZES
         );
     }
 
-    public static MockAllocationService createAllocationService(Settings settings, GatewayAllocator gatewayAllocator) {
-        return createAllocationService(settings, gatewayAllocator, SNAPSHOT_INFO_SERVICE_WITH_NO_SHARD_SIZES);
+    public static MockAllocationService createAllocationService(Settings settings, ClusterInfoService clusterInfoService) {
+        return createAllocationService(settings, new TestGatewayAllocator(), clusterInfoService, SNAPSHOT_INFO_SERVICE_WITH_NO_SHARD_SIZES);
     }
 
     public static MockAllocationService createAllocationService(Settings settings, SnapshotsInfoService snapshotsInfoService) {
-        return createAllocationService(settings, new TestGatewayAllocator(), snapshotsInfoService);
+        return createAllocationService(settings, new TestGatewayAllocator(), EmptyClusterInfoService.INSTANCE, snapshotsInfoService);
     }
 
     public static MockAllocationService createAllocationService(
         Settings settings,
         GatewayAllocator gatewayAllocator,
+        ClusterInfoService clusterInfoService,
         SnapshotsInfoService snapshotsInfoService
     ) {
+        var clusterSettings = createBuiltInClusterSettings(settings);
         return new MockAllocationService(
-            randomAllocationDeciders(settings, EMPTY_CLUSTER_SETTINGS),
+            randomAllocationDeciders(settings, clusterSettings),
             gatewayAllocator,
-            new BalancedShardsAllocator(settings),
-            EmptyClusterInfoService.INSTANCE,
+            new BalancedShardsAllocator(clusterSettings, WriteLoadForecaster.DEFAULT),
+            clusterInfoService,
             snapshotsInfoService
         );
     }
@@ -217,13 +218,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
 
     protected static AllocationDeciders yesAllocationDeciders() {
         return new AllocationDeciders(
-            Arrays.asList(
-                new TestAllocateDecision(Decision.YES),
-                new SameShardAllocationDecider(
-                    Settings.EMPTY,
-                    new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-                )
-            )
+            Arrays.asList(new TestAllocateDecision(Decision.YES), new SameShardAllocationDecider(createBuiltInClusterSettings()))
         );
     }
 
@@ -233,13 +228,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
 
     protected static AllocationDeciders throttleAllocationDeciders() {
         return new AllocationDeciders(
-            Arrays.asList(
-                new TestAllocateDecision(Decision.THROTTLE),
-                new SameShardAllocationDecider(
-                    Settings.EMPTY,
-                    new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)
-                )
-            )
+            Arrays.asList(new TestAllocateDecision(Decision.THROTTLE), new SameShardAllocationDecider(createBuiltInClusterSettings()))
         );
     }
 
@@ -352,6 +341,8 @@ public abstract class ESAllocationTestCase extends ESTestCase {
 
         private volatile long nanoTimeOverride = -1L;
 
+        private final GatewayAllocator gatewayAllocator;
+
         public MockAllocationService(
             AllocationDeciders allocationDeciders,
             GatewayAllocator gatewayAllocator,
@@ -367,6 +358,7 @@ public abstract class ESAllocationTestCase extends ESTestCase {
                 snapshotsInfoService,
                 TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY
             );
+            this.gatewayAllocator = gatewayAllocator;
         }
 
         public void setNanoTimeOverride(long nanoTime) {
@@ -377,6 +369,10 @@ public abstract class ESAllocationTestCase extends ESTestCase {
         protected long currentNanoTime() {
             return nanoTimeOverride == -1L ? super.currentNanoTime() : nanoTimeOverride;
         }
+
+        public GatewayAllocator getGatewayAllocator() {
+            return gatewayAllocator;
+        }
     }
 
     /**

+ 3 - 2
x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/cluster/routing/allocation/DataTierAllocationDeciderTests.java

@@ -51,6 +51,7 @@ import java.util.stream.Collectors;
 
 import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_COLD;
 import static org.elasticsearch.cluster.routing.allocation.DataTier.DATA_FROZEN;
+import static org.elasticsearch.common.settings.ClusterSettings.createBuiltInClusterSettings;
 import static org.elasticsearch.node.Node.NODE_EXTERNAL_ID_SETTING;
 import static org.elasticsearch.node.NodeRoleSettings.NODE_ROLES_SETTING;
 import static org.hamcrest.Matchers.containsString;
@@ -73,11 +74,11 @@ public class DataTierAllocationDeciderTests extends ESAllocationTestCase {
     private static final DesiredNode CONTENT_DESIRED_NODE = newDesiredNode("node-content", DiscoveryNodeRole.DATA_CONTENT_NODE_ROLE);
     private static final DesiredNode DATA_DESIRED_NODE = newDesiredNode("node-data", DiscoveryNodeRole.DATA_ROLE);
 
-    private final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS);
+    private final ClusterSettings clusterSettings = createBuiltInClusterSettings();
     private final AllocationDeciders allocationDeciders = new AllocationDeciders(
         Arrays.asList(
             DataTierAllocationDecider.INSTANCE,
-            new SameShardAllocationDecider(Settings.EMPTY, clusterSettings),
+            new SameShardAllocationDecider(clusterSettings),
             new ReplicaAfterPrimaryActiveAllocationDecider()
         )
     );