From d0a8d715d233e8d88aaf982d166d84916917cd84 Mon Sep 17 00:00:00 2001 From: Dianna Hohensee Date: Thu, 25 Sep 2025 12:50:51 -0700 Subject: [PATCH] Revert "Add an option to return early from an allocate call (#134786)" This reverts commit 43741c232ce463f5ff3cb09b4e6c1220273d2697. --- .../allocator/BalancedShardsAllocator.java | 123 +++------------- .../allocator/BalancerSettings.java | 9 -- .../BalancedShardsAllocatorTests.java | 134 +----------------- .../cluster/ESAllocationTestCase.java | 4 +- .../org/elasticsearch/test/ESTestCase.java | 4 - 5 files changed, 18 insertions(+), 256 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 3ae392661e98e..310c1a5963f94 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -140,83 +140,25 @@ public BalancedShardsAllocator( @Override public void allocate(RoutingAllocation allocation) { - assert allocation.isSimulating() == false || balancerSettings.completeEarlyOnShardAssignmentChange() - : "inconsistent states: isSimulating [" - + allocation.isSimulating() - + "] vs completeEarlyOnShardAssignmentChange [" - + balancerSettings.completeEarlyOnShardAssignmentChange() - + "]"; if (allocation.metadata().hasAnyIndices()) { // must not use licensed features when just starting up writeLoadForecaster.refreshLicense(); } assert allocation.ignoreDisable() == false; - assert allocation.isSimulating() == false || allocation.routingNodes().hasInactiveShards() == false - : "expect no initializing shard, but got " + allocation.routingNodes(); - // TODO: ES-12943 cannot assert the following because shards moved by commands are not simulated promptly in DesiredBalanceComputer - // assert allocation.isSimulating() == false || allocation.routingNodes().getRelocatingShardCount() == 0 - // : "expect no relocating shard, but got " + allocation.routingNodes(); if (allocation.routingNodes().size() == 0) { failAllocationOfNewPrimaries(allocation); return; } final BalancingWeights balancingWeights = balancingWeightsFactory.create(); - final Balancer balancer = new Balancer( - writeLoadForecaster, - allocation, - balancerSettings.getThreshold(), - balancingWeights, - balancerSettings.completeEarlyOnShardAssignmentChange() - ); - - boolean shardAssigned = false, shardMoved = false, shardBalanced = false; - try { - shardAssigned = balancer.allocateUnassigned(); - if (shardAssigned && balancerSettings.completeEarlyOnShardAssignmentChange()) { - return; - } - - shardMoved = balancer.moveShards(); - if (shardMoved && balancerSettings.completeEarlyOnShardAssignmentChange()) { - return; - } - - shardBalanced = balancer.balance(); - } finally { - if (logger.isDebugEnabled()) { - logger.debug( - "shards assigned: {}, shards moved: {}, shards balanced: {}, " - + "routingNodes hasInactiveShards [{}], relocation count [{}]", - shardAssigned, - shardMoved, - shardBalanced, - allocation.routingNodes().hasInactiveShards(), - allocation.routingNodes().getRelocatingShardCount() - ); - } - assert assertShardAssignmentChanges(allocation, shardAssigned, shardMoved, shardBalanced); - // Node weights are calculated after each internal balancing round and saved to the RoutingNodes copy. - collectAndRecordNodeWeightStats(balancer, balancingWeights, allocation); - } - } + final Balancer balancer = new Balancer(writeLoadForecaster, allocation, balancerSettings.getThreshold(), balancingWeights); + balancer.allocateUnassigned(); + balancer.moveShards(); + balancer.balance(); - private boolean assertShardAssignmentChanges( - RoutingAllocation allocation, - boolean shardAssigned, - boolean shardMoved, - boolean shardBalanced - ) { - if (allocation.isSimulating() == false) { - return true; - } - assert shardAssigned == false || allocation.routingNodes().hasInactiveShards() - : "expect initializing shard, but got " + allocation.routingNodes(); - - assert (shardMoved == false && shardBalanced == false) || allocation.routingNodes().getRelocatingShardCount() > 0 - : "expect relocating shard, but got " + allocation.routingNodes(); - return true; + // Node weights are calculated after each internal balancing round and saved to the RoutingNodes copy. + collectAndRecordNodeWeightStats(balancer, balancingWeights, allocation); } private void collectAndRecordNodeWeightStats(Balancer balancer, BalancingWeights balancingWeights, RoutingAllocation allocation) { @@ -246,8 +188,7 @@ public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, f writeLoadForecaster, allocation, balancerSettings.getThreshold(), - balancingWeightsFactory.create(), - balancerSettings.completeEarlyOnShardAssignmentChange() + balancingWeightsFactory.create() ); AllocateUnassignedDecision allocateUnassignedDecision = AllocateUnassignedDecision.NOT_TAKEN; MoveDecision moveDecision = MoveDecision.NOT_TAKEN; @@ -307,14 +248,12 @@ public static class Balancer { private final Map nodes; private final BalancingWeights balancingWeights; private final NodeSorters nodeSorters; - private final boolean completeEarlyOnShardAssignmentChange; private Balancer( WriteLoadForecaster writeLoadForecaster, RoutingAllocation allocation, float threshold, - BalancingWeights balancingWeights, - boolean completeEarlyOnShardAssignmentChange + BalancingWeights balancingWeights ) { this.writeLoadForecaster = writeLoadForecaster; this.allocation = allocation; @@ -327,7 +266,6 @@ private Balancer( nodes = Collections.unmodifiableMap(buildModelFromAssigned()); this.nodeSorters = balancingWeights.createNodeSorters(nodesArray(), this); this.balancingWeights = balancingWeights; - this.completeEarlyOnShardAssignmentChange = completeEarlyOnShardAssignmentChange; } private static long getShardDiskUsageInBytes(ShardRouting shardRouting, IndexMetadata indexMetadata, ClusterInfo clusterInfo) { @@ -420,7 +358,7 @@ private IndexMetadata indexMetadata(ProjectIndex index) { * Balances the nodes on the cluster model according to the weight function. * The actual balancing is delegated to {@link #balanceByWeights(NodeSorter)} */ - private boolean balance() { + private void balance() { if (logger.isTraceEnabled()) { logger.trace("Start balancing cluster"); } @@ -433,27 +371,21 @@ private boolean balance() { * Therefore we only do a rebalance if we have fetched all information. */ logger.debug("skipping rebalance due to in-flight shard/store fetches"); - return false; + return; } if (allocation.deciders().canRebalance(allocation).type() != Type.YES) { logger.trace("skipping rebalance as it is disabled"); - return false; + return; } - boolean shardBalanced = false; // Balance each partition for (NodeSorter nodeSorter : nodeSorters) { if (nodeSorter.modelNodes.length < 2) { /* skip if we only have one node */ logger.trace("skipping rebalance as the partition has single node only"); continue; } - shardBalanced |= balanceByWeights(nodeSorter); - // TODO: We could choose to account shardBalanced separately for each partition since they do not overlap. - if (shardBalanced && completeEarlyOnShardAssignmentChange) { - return true; - } + balanceByWeights(nodeSorter); } - return shardBalanced; } /** @@ -599,8 +531,7 @@ private MoveDecision decideRebalance(final ProjectIndex index, final ShardRoutin * only, or in other words relocations that move the weight delta closer * to {@code 0.0} */ - private boolean balanceByWeights(NodeSorter sorter) { - boolean shardBalanced = false; + private void balanceByWeights(NodeSorter sorter) { final AllocationDeciders deciders = allocation.deciders(); final ModelNode[] modelNodes = sorter.modelNodes; final float[] weights = sorter.weights; @@ -699,15 +630,6 @@ private boolean balanceByWeights(NodeSorter sorter) { sorter.sort(0, relevantNodes); lowIdx = 0; highIdx = relevantNodes - 1; - - shardBalanced = true; - if (completeEarlyOnShardAssignmentChange && routingNodes.getRelocatingShardCount() > 0) { - // ES-12955: Check routingNodes.getRelocatingShardCount() > 0 in case the first relocation is a THROTTLE. - // It should not happen in production, i.e, throttling should not happen unless there is a prior shard - // that is already relocating. But in tests, we have decider like RandomAllocationDecider that can - // randomly return THROTTLE when there is no existing relocation. - return true; - } continue; } } @@ -729,7 +651,6 @@ private boolean balanceByWeights(NodeSorter sorter) { } } } - return shardBalanced; } /** @@ -800,8 +721,7 @@ protected int comparePivot(int j) { * shard is created with an incremented version in the state * {@link ShardRoutingState#INITIALIZING}. */ - public boolean moveShards() { - boolean shardMoved = false; + public void moveShards() { // Iterate over the started shards interleaving between nodes, and check if they can remain. In the presence of throttling // shard movements, the goal of this iteration order is to achieve a fairer movement of shards from the nodes that are // offloading the shards. @@ -825,15 +745,10 @@ public boolean moveShards() { if (logger.isTraceEnabled()) { logger.trace("Moved shard [{}] to node [{}]", shardRouting, targetNode.getRoutingNode()); } - shardMoved = true; - if (completeEarlyOnShardAssignmentChange) { - return true; - } } else if (moveDecision.isDecisionTaken() && moveDecision.canRemain() == false) { logger.trace("[{}][{}] can't move", shardRouting.index(), shardRouting.id()); } } - return shardMoved; } /** @@ -973,14 +888,14 @@ private Map buildModelFromAssigned() { * Allocates all given shards on the minimal eligible node for the shards index * with respect to the weight function. All given shards must be unassigned. */ - private boolean allocateUnassigned() { + private void allocateUnassigned() { RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned(); assert nodes.isEmpty() == false; if (logger.isTraceEnabled()) { logger.trace("Start allocating unassigned shards"); } if (unassigned.isEmpty()) { - return false; + return; } /* @@ -1017,7 +932,6 @@ private boolean allocateUnassigned() { int secondaryLength = 0; int primaryLength = primary.length; ArrayUtil.timSort(primary, comparator); - boolean shardAssignmentChanged = false; do { for (int i = 0; i < primaryLength; i++) { ShardRouting shard = primary[i]; @@ -1035,7 +949,6 @@ private boolean allocateUnassigned() { final long shardSize = getExpectedShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation); shard = routingNodes.initializeShard(shard, minNode.getNodeId(), null, shardSize, allocation.changes()); - shardAssignmentChanged = true; minNode.addShard(index, shard); if (shard.primary() == false) { // copy over the same replica shards to the secondary array so they will get allocated @@ -1059,9 +972,6 @@ private boolean allocateUnassigned() { assert allocationDecision.getAllocationStatus() == AllocationStatus.DECIDERS_THROTTLED; final long shardSize = getExpectedShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation); minNode.addShard(projectIndex(shard), shard.initialize(minNode.getNodeId(), null, shardSize)); - // If we see a throttle decision in simulation, there must be other shards that got assigned before it. - assert allocation.isSimulating() == false || shardAssignmentChanged - : "shard " + shard + " was throttled but no other shards were assigned"; } else { if (logger.isTraceEnabled()) { logger.trace("No Node found to assign shard [{}]", shard); @@ -1084,7 +994,6 @@ private boolean allocateUnassigned() { secondaryLength = 0; } while (primaryLength > 0); // clear everything we have either added it or moved to ignoreUnassigned - return shardAssignmentChanged; } private ProjectIndex projectIndex(ShardRouting shardRouting) { diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancerSettings.java b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancerSettings.java index ea45e46a278e9..31932bc1c2079 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancerSettings.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancerSettings.java @@ -9,7 +9,6 @@ package org.elasticsearch.cluster.routing.allocation.allocator; -import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -27,7 +26,6 @@ public class BalancerSettings { private volatile float writeLoadBalanceFactor; private volatile float diskUsageBalanceFactor; private volatile float threshold; - private final boolean completeEarlyOnShardAssignmentChange; public BalancerSettings(Settings settings) { this(ClusterSettings.createBuiltInClusterSettings(settings)); @@ -39,9 +37,6 @@ public BalancerSettings(ClusterSettings clusterSettings) { clusterSettings.initializeAndWatch(WRITE_LOAD_BALANCE_FACTOR_SETTING, value -> this.writeLoadBalanceFactor = value); clusterSettings.initializeAndWatch(DISK_USAGE_BALANCE_FACTOR_SETTING, value -> this.diskUsageBalanceFactor = value); clusterSettings.initializeAndWatch(THRESHOLD_SETTING, value -> this.threshold = value); - this.completeEarlyOnShardAssignmentChange = ClusterModule.DESIRED_BALANCE_ALLOCATOR.equals( - clusterSettings.get(ClusterModule.SHARDS_ALLOCATOR_TYPE_SETTING) - ); } /** @@ -72,8 +67,4 @@ public float getDiskUsageBalanceFactor() { public float getThreshold() { return threshold; } - - public boolean completeEarlyOnShardAssignmentChange() { - return completeEarlyOnShardAssignmentChange; - } } diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocatorTests.java b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocatorTests.java index 2bacd38325f41..3667de9c65e4e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocatorTests.java @@ -38,10 +38,8 @@ 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.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; @@ -62,7 +60,6 @@ import java.util.function.Function; import java.util.stream.Collector; import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.stream.StreamSupport; import static java.util.stream.Collectors.mapping; @@ -680,135 +677,6 @@ public void testPartitionedClusterWithSeparateWeights() { assertThat(shardBalancedPartition.get("shardsOnly-2"), hasSize(3)); } - public void testReturnEarlyOnShardAssignmentChanges() { - var allocationService = new MockAllocationService( - prefixAllocationDeciders(), - new TestGatewayAllocator(), - new BalancedShardsAllocator(BalancerSettings.DEFAULT, TEST_WRITE_LOAD_FORECASTER), - EmptyClusterInfoService.INSTANCE, - SNAPSHOT_INFO_SERVICE_WITH_NO_SHARD_SIZES - ); - - final var nodeNames = List.of("large-1", "large-2", "small-1"); - DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(); - for (String nodeName : nodeNames) { - discoveryNodesBuilder.add(newNode(nodeName)); - } - final var projectMetadataBuilder = ProjectMetadata.builder(ProjectId.DEFAULT); - final var routingTableBuilder = RoutingTable.builder(TestShardRoutingRoleStrategies.DEFAULT_ROLE_ONLY); - - // An index with 2 unassigned primary shards, no replica - final IndexMetadata unassignedPrimary = anIndex("large-unassigned-primary", indexSettings(IndexVersion.current(), 2, 0)).build(); - projectMetadataBuilder.put(unassignedPrimary, false); - routingTableBuilder.addAsNew(unassignedPrimary); - - // An index with 1 started primary and 1 unassigned replica - final IndexMetadata unassignedReplica = anIndex("large-unassigned-replica", indexSettings(IndexVersion.current(), 1, 1)) - .putInSyncAllocationIds(0, Set.of(UUIDs.randomBase64UUID())) - .build(); - projectMetadataBuilder.put(unassignedReplica, false); - routingTableBuilder.add( - IndexRoutingTable.builder(unassignedReplica.getIndex()) - .addShard( - shardRoutingBuilder(unassignedReplica.getIndex().getName(), 0, "large-1", true, ShardRoutingState.STARTED) - .withAllocationId(AllocationId.newInitializing(unassignedReplica.inSyncAllocationIds(0).iterator().next())) - .build() - ) - .addShard(shardRoutingBuilder(unassignedReplica.getIndex().getName(), 0, null, false, ShardRoutingState.UNASSIGNED).build()) - ); - - // A started index with undesired allocation (cannot remain) - final IndexMetadata undesiredAllocation = anIndex("large-undesired-allocation", indexSettings(IndexVersion.current(), 1, 0)) - .putInSyncAllocationIds(0, Set.of(UUIDs.randomBase64UUID())) - .build(); - projectMetadataBuilder.put(undesiredAllocation, false); - routingTableBuilder.add( - IndexRoutingTable.builder(undesiredAllocation.getIndex()) - .addShard( - shardRoutingBuilder(undesiredAllocation.getIndex().getName(), 0, "small-1", true, ShardRoutingState.STARTED) - .withAllocationId(AllocationId.newInitializing(undesiredAllocation.inSyncAllocationIds(0).iterator().next())) - .build() - ) - ); - - // Indices with unbalanced weight of write loads - final var numWriteLoadIndices = between(3, 5); - for (int i = 0; i < numWriteLoadIndices; i++) { - final IndexMetadata writeLoadIndex = anIndex("large-write-load-" + i, indexSettings(IndexVersion.current(), 1, 0)) - .putInSyncAllocationIds(0, Set.of(UUIDs.randomBase64UUID())) - .indexWriteLoadForecast(100.0) - .build(); - projectMetadataBuilder.put(writeLoadIndex, false); - routingTableBuilder.add( - IndexRoutingTable.builder(writeLoadIndex.getIndex()) - .addShard( - shardRoutingBuilder(writeLoadIndex.getIndex().getName(), 0, "large-1", true, ShardRoutingState.STARTED) - .withAllocationId(AllocationId.newInitializing(writeLoadIndex.inSyncAllocationIds(0).iterator().next())) - .build() - ) - ); - } - - var clusterState = ClusterState.builder(ClusterName.DEFAULT) - .nodes(discoveryNodesBuilder) - .putProjectMetadata(projectMetadataBuilder) - .putRoutingTable(ProjectId.DEFAULT, routingTableBuilder.build()) - .build(); - - // First reroute - clusterState = startInitializingShardsAndReroute(allocationService, clusterState); - { - // Unassigned primary and replica shards are assigned - final RoutingTable routingTable = clusterState.routingTable(ProjectId.DEFAULT); - for (int shardId = 0; shardId < 2; shardId++) { - final var shard = routingTable.shardRoutingTable(unassignedPrimary.getIndex().getName(), shardId).primaryShard(); - assertTrue("unexpected shard state: " + shard, shard.initializing()); - } - final var replicaShard = routingTable.shardRoutingTable(unassignedReplica.getIndex().getName(), 0).replicaShards().getFirst(); - assertTrue("unexpected shard state: " + replicaShard, replicaShard.initializing()); - - // Undesired allocation is not moved because allocate call returns early - final var shard = routingTable.shardRoutingTable(undesiredAllocation.getIndex().getName(), 0).primaryShard(); - assertTrue("unexpected shard state: " + shard, shard.started()); - - // Also no rebalancing for indices with unbalanced write loads due to returning early - for (int i = 0; i < numWriteLoadIndices; i++) { - final var writeLoadShard = routingTable.shardRoutingTable("large-write-load-" + i, 0).primaryShard(); - assertTrue("unexpected shard state: " + writeLoadShard, writeLoadShard.started()); - } - } - - // Second reroute - clusterState = startInitializingShardsAndReroute(allocationService, clusterState); - { - // Undesired allocation is now relocating - final RoutingTable routingTable = clusterState.routingTable(ProjectId.DEFAULT); - final var shard = routingTable.shardRoutingTable(undesiredAllocation.getIndex().getName(), 0).primaryShard(); - assertTrue("unexpected shard state: " + shard, shard.relocating()); - - // Still no rebalancing for indices with unbalanced write loads due to returning early - for (int i = 0; i < numWriteLoadIndices; i++) { - final var writeLoadShard = routingTable.shardRoutingTable("large-write-load-" + i, 0).primaryShard(); - assertTrue("unexpected shard state: " + writeLoadShard, writeLoadShard.started()); - } - } - - // Third reroute - clusterState = startInitializingShardsAndReroute(allocationService, clusterState); - { - // Rebalance should happen for one and only one of the indices with unbalanced write loads due to returning early - final RoutingTable routingTable = clusterState.routingTable(ProjectId.DEFAULT); - final List relocatingShards = IntStream.range(0, numWriteLoadIndices) - .mapToObj(i -> routingTable.shardRoutingTable("large-write-load-" + i, 0).primaryShard()) - .filter(ShardRouting::relocating) - .toList(); - assertThat(relocatingShards, hasSize(1)); - } - - // Ensure allocate to the balancer eventually stop after sufficient iterations - applyStartedShardsUntilNoChange(clusterState, allocationService); - } - private Map getTargetShardPerNodeCount(IndexRoutingTable indexRoutingTable) { var counts = new HashMap(); for (int shardId = 0; shardId < indexRoutingTable.size(); shardId++) { @@ -1014,7 +882,7 @@ private Decision nodePrefixMatchesIndexPrefix(ShardRouting shardRouting, Routing var nodePrefix = prefix(node.node().getId()); return nodePrefix.equals(indexPrefix) ? Decision.YES : Decision.NO; } - }, new SameShardAllocationDecider(ClusterSettings.createBuiltInClusterSettings()))); + })); } private static String prefix(String value) { diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java index 7d12d0e77cb99..44236024a40b0 100644 --- a/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/cluster/ESAllocationTestCase.java @@ -158,9 +158,7 @@ public static AllocationDeciders randomAllocationDeciders(Settings settings, Clu protected static ShardsAllocator createShardsAllocator(Settings settings) { return switch (pickShardsAllocator(settings)) { - case BALANCED_ALLOCATOR -> new BalancedShardsAllocator( - Settings.builder().put(settings).put(SHARDS_ALLOCATOR_TYPE_SETTING.getKey(), BALANCED_ALLOCATOR).build() - ); + case BALANCED_ALLOCATOR -> new BalancedShardsAllocator(settings); case DESIRED_BALANCE_ALLOCATOR -> createDesiredBalanceShardsAllocator(settings); default -> throw new AssertionError("Unknown allocator"); }; diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 8150456194b76..e173790c285ed 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -695,10 +695,6 @@ protected List filteredWarnings() { ); filtered.add("Configuring [path.data] with a list is deprecated. Instead specify as a string value"); filtered.add("setting [path.shared_data] is deprecated and will be removed in a future release"); - filtered.add( - "[cluster.routing.allocation.type] setting was deprecated in Elasticsearch and will be removed " - + "in a future release. See the breaking changes documentation for the next major version." - ); return filtered; }