Browse Source

MINOR: Remove Dead Code in Routing (#35074)

* MINOR: Remove Dead Code in Routing
Armin Braun 7 years ago
parent
commit
216c761a5d

+ 0 - 10
server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java

@@ -65,7 +65,6 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
     final List<ShardRouting> activeShards;
     final List<ShardRouting> assignedShards;
     final Set<String> allAllocationIds;
-    static final List<ShardRouting> NO_SHARDS = Collections.emptyList();
     final boolean allShardsStarted;
 
     private volatile Map<AttributesKey, AttributesRoutings> activeShardsByAttributes = emptyMap();
@@ -220,15 +219,6 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
         return this.assignedShards;
     }
 
-    /**
-     * Returns a {@link List} of assigned shards
-     *
-     * @return a {@link List} of shards
-     */
-    public List<ShardRouting> getAssignedShards() {
-        return this.assignedShards;
-    }
-
     public ShardIterator shardsRandomIt() {
         return new PlainShardIterator(shardId, shuffler.shuffle(shards));
     }

+ 3 - 9
server/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java

@@ -582,7 +582,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
                         moveToUnassigned(failedShard, unassignedInfo);
                     } else {
                         movePrimaryToUnassignedAndDemoteToReplica(failedShard, unassignedInfo);
-                        promoteReplicaToPrimary(activeReplica, indexMetaData, routingChangesObserver);
+                        promoteReplicaToPrimary(activeReplica, routingChangesObserver);
                     }
                 } else {
                     // initializing shard that is not relocation target, just move to unassigned
@@ -611,7 +611,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
                     moveToUnassigned(failedShard, unassignedInfo);
                 } else {
                     movePrimaryToUnassignedAndDemoteToReplica(failedShard, unassignedInfo);
-                    promoteReplicaToPrimary(activeReplica, indexMetaData, routingChangesObserver);
+                    promoteReplicaToPrimary(activeReplica, routingChangesObserver);
                 }
             } else {
                 assert failedShard.primary() == false;
@@ -627,8 +627,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
             " was matched but wasn't removed";
     }
 
-    private void promoteReplicaToPrimary(ShardRouting activeReplica, IndexMetaData indexMetaData,
-                                         RoutingChangesObserver routingChangesObserver) {
+    private void promoteReplicaToPrimary(ShardRouting activeReplica, RoutingChangesObserver routingChangesObserver) {
         // if the activeReplica was relocating before this call to failShard, its relocation was cancelled earlier when we
         // failed initializing replica shards (and moved replica relocation source back to started)
         assert activeReplica.started() : "replica relocation should have been cancelled: " + activeReplica;
@@ -832,11 +831,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
          */
         public int size() { return unassigned.size(); }
 
-        /**
-         * Returns the size of the temporarily marked as ignored unassigned shards
-         */
-        public int ignoredSize() { return ignored.size(); }
-
         /**
          * Returns the number of non-ignored unassigned primaries
          */

+ 2 - 10
server/src/main/java/org/elasticsearch/cluster/routing/RoutingTable.java

@@ -266,7 +266,7 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
                                     Collections.singletonList(shardRouting.getTargetRelocatingShard())));
                         }
                     } else if (includeEmpty) { // we need this for counting properly, just make it an empty one
-                        set.add(new PlainShardIterator(shardRouting.shardId(), Collections.<ShardRouting>emptyList()));
+                        set.add(new PlainShardIterator(shardRouting.shardId(), Collections.emptyList()));
                     }
                 }
             }
@@ -328,7 +328,7 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
                 if (primary.active()) {
                     set.add(primary.shardsIt());
                 } else if (includeEmpty) { // we need this for counting properly, just make it an empty one
-                    set.add(new PlainShardIterator(primary.shardId(), Collections.<ShardRouting>emptyList()));
+                    set.add(new PlainShardIterator(primary.shardId(), Collections.emptyList()));
                 }
             }
         }
@@ -567,14 +567,6 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
             return this;
         }
 
-        public Builder indicesRouting(Map<String, IndexRoutingTable> indicesRouting) {
-            if (indicesRouting == null) {
-                throw new IllegalStateException("once build is called the builder cannot be reused");
-            }
-            this.indicesRouting.putAll(indicesRouting);
-            return this;
-        }
-
         public Builder remove(String index) {
             if (indicesRouting == null) {
                 throw new IllegalStateException("once build is called the builder cannot be reused");

+ 3 - 9
server/src/main/java/org/elasticsearch/cluster/routing/allocation/DiskThresholdSettings.java

@@ -70,7 +70,6 @@ public class DiskThresholdSettings {
     private volatile boolean includeRelocations;
     private volatile boolean enabled;
     private volatile TimeValue rerouteInterval;
-    private volatile String floodStageRaw;
     private volatile Double freeDiskThresholdFloodStage;
     private volatile ByteSizeValue freeBytesThresholdFloodStage;
 
@@ -80,13 +79,13 @@ public class DiskThresholdSettings {
         final String floodStage = CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING.get(settings);
         setHighWatermark(highWatermark);
         setLowWatermark(lowWatermark);
-        setFloodStageRaw(floodStage);
+        setFloodStage(floodStage);
         this.includeRelocations = CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING.get(settings);
         this.rerouteInterval = CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING.get(settings);
         this.enabled = CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING.get(settings);
         clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING, this::setLowWatermark);
         clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING, this::setHighWatermark);
-        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING, this::setFloodStageRaw);
+        clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING, this::setFloodStage);
         clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_INCLUDE_RELOCATIONS_SETTING, this::setIncludeRelocations);
         clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_REROUTE_INTERVAL_SETTING, this::setRerouteInterval);
         clusterSettings.addSettingsUpdateConsumer(CLUSTER_ROUTING_ALLOCATION_DISK_THRESHOLD_ENABLED_SETTING, this::setEnabled);
@@ -230,9 +229,8 @@ public class DiskThresholdSettings {
             CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey());
     }
 
-    private void setFloodStageRaw(String floodStageRaw) {
+    private void setFloodStage(String floodStageRaw) {
         // Watermark is expressed in terms of used data, but we need "free" data watermark
-        this.floodStageRaw = floodStageRaw;
         this.freeDiskThresholdFloodStage = 100.0 - thresholdPercentageFromWatermark(floodStageRaw);
         this.freeBytesThresholdFloodStage = thresholdBytesFromWatermark(floodStageRaw,
             CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING.getKey());
@@ -276,10 +274,6 @@ public class DiskThresholdSettings {
         return freeBytesThresholdFloodStage;
     }
 
-    public String getFloodStageRaw() {
-        return floodStageRaw;
-    }
-
     public boolean includeRelocations() {
         return includeRelocations;
     }

+ 0 - 23
server/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java

@@ -24,7 +24,6 @@ import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IntroSorter;
 import org.elasticsearch.cluster.metadata.IndexMetaData;
 import org.elasticsearch.cluster.metadata.MetaData;
-import org.elasticsearch.cluster.node.DiscoveryNode;
 import org.elasticsearch.cluster.routing.RoutingNode;
 import org.elasticsearch.cluster.routing.RoutingNodes;
 import org.elasticsearch.cluster.routing.ShardRouting;
@@ -447,28 +446,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
             }
         }
 
-        public Map<DiscoveryNode, Float> weighShard(ShardRouting shard) {
-            final ModelNode[] modelNodes = sorter.modelNodes;
-            final float[] weights = sorter.weights;
-
-            buildWeightOrderedIndices();
-            Map<DiscoveryNode, Float> nodes = new HashMap<>(modelNodes.length);
-            float currentNodeWeight = 0.0f;
-            for (int i = 0; i < modelNodes.length; i++) {
-                if (modelNodes[i].getNodeId().equals(shard.currentNodeId())) {
-                    // If a node was found with the shard, use that weight instead of 0.0
-                    currentNodeWeight = weights[i];
-                    break;
-                }
-            }
-
-            for (int i = 0; i < modelNodes.length; i++) {
-                final float delta = currentNodeWeight - weights[i];
-                nodes.put(modelNodes[i].getRoutingNode().node(), delta);
-            }
-            return nodes;
-        }
-
         /**
          * Balances the nodes on the cluster model according to the weight
          * function. The configured threshold is the minimum delta between the

+ 0 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java

@@ -207,10 +207,6 @@ public class AwarenessAllocationDecider extends AllocationDecider {
                         currentNodeCount,
                         requiredCountPerAttribute + leftoverPerAttribute);
             }
-            // all is well, we are below or same as average
-            if (currentNodeCount <= requiredCountPerAttribute) {
-                continue;
-            }
         }
 
         return allocation.decision(Decision.YES, NAME, "node meets all awareness attribute requirements");

+ 0 - 4
server/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java

@@ -92,10 +92,6 @@ public abstract class Decision implements ToXContent, Writeable {
             this.id = id;
         }
 
-        public static Type resolve(String s) {
-            return Type.valueOf(s.toUpperCase(Locale.ROOT));
-        }
-
         public static Type readFrom(StreamInput in) throws IOException {
             int i = in.readVInt();
             switch (i) {