gerlowskija commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224352496


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplicaMigrationUtils.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cloud.api.collections;
+
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.solr.cloud.ActiveReplicaWatcher;
+import org.apache.solr.common.SolrCloseableLatch;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CollectionStateWatcher;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicaMigrationUtils {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Code to migrate replicas to already chosen nodes. This will create new 
replicas, and delete the
+   * old replicas after the creation is done.
+   *
+   * <p>If an error occurs during the creation of new replicas, all new 
replicas will be deleted.
+   *
+   * @param ccc The collection command context to use from the API that calls 
this method
+   * @param movements a map from replica to the new node that the replica 
should live on
+   * @param parallel whether the replica creations should be done in parallel
+   * @param waitForFinalState wait for the final state of all newly created 
replicas before
+   *     continuing
+   * @param timeout the amount of time to wait for new replicas to be created
+   * @param asyncId If provided, the command will be run under the given 
asyncId
+   * @param results push results (successful and failure) onto this list
+   * @return whether the command was successful
+   */
+  static boolean migrateReplicas(
+      CollectionCommandContext ccc,
+      Map<Replica, String> movements,
+      boolean parallel,
+      boolean waitForFinalState,
+      int timeout,
+      String asyncId,
+      NamedList<Object> results)
+      throws IOException, InterruptedException, KeeperException {
+    // how many leaders are we moving? for these replicas we have to make sure 
that either:
+    // * another existing replica can become a leader, or
+    // * we wait until the newly created replica completes recovery (and can 
become the new leader)
+    // If waitForFinalState=true we wait for all replicas
+    int numLeaders = 0;
+    for (Replica replica : movements.keySet()) {
+      if (replica.isLeader() || waitForFinalState) {
+        numLeaders++;
+      }
+    }
+    // map of collectionName_coreNodeName to watchers
+    Map<String, CollectionStateWatcher> watchers = new HashMap<>();
+    List<ZkNodeProps> createdReplicas = new ArrayList<>();
+
+    AtomicBoolean anyOneFailed = new AtomicBoolean(false);
+    SolrCloseableLatch countDownLatch =
+        new SolrCloseableLatch(movements.size(), ccc.getCloseableToLatchOn());
+
+    SolrCloseableLatch replicasToRecover =
+        new SolrCloseableLatch(numLeaders, ccc.getCloseableToLatchOn());
+
+    ClusterState clusterState = ccc.getZkStateReader().getClusterState();
+
+    for (Map.Entry<Replica, String> movement : movements.entrySet()) {
+      Replica sourceReplica = movement.getKey();
+      String targetNode = movement.getValue();
+      String sourceCollection = sourceReplica.getCollection();
+      if (log.isInfoEnabled()) {
+        log.info(
+            "Going to create replica for collection={} shard={} on node={}",
+            sourceCollection,
+            sourceReplica.getShard(),
+            targetNode);
+      }
+
+      ZkNodeProps msg =
+          sourceReplica
+              .toFullProps()
+              .plus("parallel", String.valueOf(parallel))
+              .plus(CoreAdminParams.NODE, targetNode);
+      if (asyncId != null) msg.getProperties().put(ASYNC, asyncId);
+      NamedList<Object> nl = new NamedList<>();
+      final ZkNodeProps addedReplica =
+          new AddReplicaCmd(ccc)
+              .addReplica(
+                  clusterState,
+                  msg,
+                  nl,
+                  () -> {
+                    countDownLatch.countDown();
+                    if (nl.get("failure") != null) {
+                      String errorString =
+                          String.format(
+                              Locale.ROOT,
+                              "Failed to create replica for collection=%s 
shard=%s" + " on node=%s",
+                              sourceCollection,
+                              sourceReplica.getShard(),
+                              targetNode);
+                      log.warn(errorString);
+                      // one replica creation failed. Make the best attempt to
+                      // delete all the replicas created so far in the target
+                      // and exit
+                      synchronized (results) {
+                        results.add("failure", errorString);
+                        anyOneFailed.set(true);
+                      }
+                    } else {
+                      if (log.isDebugEnabled()) {
+                        log.debug(
+                            "Successfully created replica for collection={} 
shard={} on node={}",
+                            sourceCollection,
+                            sourceReplica.getShard(),
+                            targetNode);
+                      }
+                    }
+                  })
+              .get(0);
+
+      if (addedReplica != null) {
+        createdReplicas.add(addedReplica);
+        if (sourceReplica.isLeader() || waitForFinalState) {
+          String shardName = sourceReplica.getShard();
+          String replicaName = sourceReplica.getName();
+          String key = sourceCollection + "_" + replicaName;
+          CollectionStateWatcher watcher;
+          if (waitForFinalState) {
+            watcher =
+                new ActiveReplicaWatcher(
+                    sourceCollection,
+                    null,
+                    
Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
+                    replicasToRecover);
+          } else {
+            watcher =
+                new LeaderRecoveryWatcher(
+                    sourceCollection,
+                    shardName,
+                    replicaName,
+                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
+                    replicasToRecover);
+          }
+          watchers.put(key, watcher);
+          log.debug("--- adding {}, {}", key, watcher);
+          
ccc.getZkStateReader().registerCollectionStateWatcher(sourceCollection, 
watcher);
+        } else {
+          log.debug("--- not waiting for {}", addedReplica);
+        }
+      }
+    }
+
+    log.debug("Waiting for replicas to be added");
+    if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
+      log.info("Timed out waiting for replicas to be added");
+      anyOneFailed.set(true);
+    } else {
+      log.debug("Finished waiting for replicas to be added");
+    }
+
+    // now wait for leader replicas to recover
+    log.debug("Waiting for {} leader replicas to recover", numLeaders);
+    if (!replicasToRecover.await(timeout, TimeUnit.SECONDS)) {
+      if (log.isInfoEnabled()) {
+        log.info(
+            "Timed out waiting for {} leader replicas to recover", 
replicasToRecover.getCount());
+      }
+      anyOneFailed.set(true);
+    } else {
+      log.debug("Finished waiting for leader replicas to recover");
+    }
+    // remove the watchers, we're done either way
+    for (Map.Entry<String, CollectionStateWatcher> e : watchers.entrySet()) {

Review Comment:
   [Q] Should this watcher cleanup be in a "finally" block, or are we OK with 
these watchers sticking around in the case of an exception being thrown 
somewhere above?



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplicaMigrationUtils.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cloud.api.collections;
+
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.solr.cloud.ActiveReplicaWatcher;
+import org.apache.solr.common.SolrCloseableLatch;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CollectionStateWatcher;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ReplicaMigrationUtils {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Code to migrate replicas to already chosen nodes. This will create new 
replicas, and delete the
+   * old replicas after the creation is done.
+   *
+   * <p>If an error occurs during the creation of new replicas, all new 
replicas will be deleted.
+   *
+   * @param ccc The collection command context to use from the API that calls 
this method
+   * @param movements a map from replica to the new node that the replica 
should live on
+   * @param parallel whether the replica creations should be done in parallel
+   * @param waitForFinalState wait for the final state of all newly created 
replicas before
+   *     continuing
+   * @param timeout the amount of time to wait for new replicas to be created
+   * @param asyncId If provided, the command will be run under the given 
asyncId
+   * @param results push results (successful and failure) onto this list
+   * @return whether the command was successful
+   */
+  static boolean migrateReplicas(

Review Comment:
   [0] I know a lot of it is the way our codestyle does linebreaks and 
indenting, but at ~180L this is still a pretty large method.  Could any more of 
it be broken out into private helper methods?



##########
solr/core/src/java/org/apache/solr/cluster/placement/BalanceRequest.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement;
+
+import java.util.Set;
+import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+
+/**
+ * A cluster related placement request that Solr asks a {@link 
PlacementPlugin} plugin to resolve

Review Comment:
   [0] drop the redundant second "plugin" from:
   
   > that Solr asks a {@link PlacementPlugin} plugin to resolve



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,604 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.plugins;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalanceRequest;
+import org.apache.solr.cluster.placement.DeleteCollectionRequest;
+import org.apache.solr.cluster.placement.DeleteReplicasRequest;
+import org.apache.solr.cluster.placement.DeleteShardsRequest;
+import org.apache.solr.cluster.placement.ModificationRequest;
+import org.apache.solr.cluster.placement.PlacementContext;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementModificationException;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.common.util.CollectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class OrderedNodePlacementPlugin implements PlacementPlugin {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Override
+  public List<PlacementPlan> computePlacements(
+      Collection<PlacementRequest> requests, PlacementContext placementContext)
+      throws PlacementException {
+    List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
+    Set<Node> allNodes = new HashSet<>();
+    Set<SolrCollection> allCollections = new HashSet<>();
+    for (PlacementRequest request : requests) {
+      allNodes.addAll(request.getTargetNodes());
+      allCollections.add(request.getCollection());
+    }
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(placementContext, allNodes, allCollections, 
true).values();
+    for (PlacementRequest request : requests) {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      List<WeightedNode> nodesForRequest =
+          weightedNodes.stream()
+              .filter(wn -> request.getTargetNodes().contains(wn.getNode()))
+              .collect(Collectors.toList());
+
+      Set<ReplicaPlacement> replicaPlacements =
+          CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
+
+      SolrCollection solrCollection = request.getCollection();
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Placing replicas for Collection: {}, Shard: {}, ReplicaType: 
{}",

Review Comment:
   [0] Should we log replicaCount here as well?



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java:
##########
@@ -451,6 +451,21 @@ List<ReplicaPosition> assign(
         SolrCloudManager solrCloudManager, List<AssignRequest> assignRequests)
         throws AssignmentException, IOException, InterruptedException;
 
+    /**
+     * Balance replicas across nodes.
+     *
+     * @param solrCloudManager current instance of {@link SolrCloudManager}.
+     * @param nodes to compute replica balancing across.
+     * @param maxBalanceSkew to ensure strictness of replica balancing.
+     * @return Map from Replica to the Node where that Replica should be moved.
+     * @throws AssignmentException when balance request cannot produce any 
valid assignments.
+     */
+    default Map<Replica, String> balanceReplicas(

Review Comment:
   [Q] Does this method actually move the replicas, or just compute the series 
of movements that need done?
   
   If the latter, might a different method name make this clearer?



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,604 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.plugins;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalanceRequest;
+import org.apache.solr.cluster.placement.DeleteCollectionRequest;
+import org.apache.solr.cluster.placement.DeleteReplicasRequest;
+import org.apache.solr.cluster.placement.DeleteShardsRequest;
+import org.apache.solr.cluster.placement.ModificationRequest;
+import org.apache.solr.cluster.placement.PlacementContext;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementModificationException;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.common.util.CollectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class OrderedNodePlacementPlugin implements PlacementPlugin {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Override
+  public List<PlacementPlan> computePlacements(
+      Collection<PlacementRequest> requests, PlacementContext placementContext)
+      throws PlacementException {
+    List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
+    Set<Node> allNodes = new HashSet<>();
+    Set<SolrCollection> allCollections = new HashSet<>();
+    for (PlacementRequest request : requests) {
+      allNodes.addAll(request.getTargetNodes());
+      allCollections.add(request.getCollection());
+    }
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(placementContext, allNodes, allCollections, 
true).values();
+    for (PlacementRequest request : requests) {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      List<WeightedNode> nodesForRequest =
+          weightedNodes.stream()
+              .filter(wn -> request.getTargetNodes().contains(wn.getNode()))
+              .collect(Collectors.toList());
+
+      Set<ReplicaPlacement> replicaPlacements =
+          CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
+
+      SolrCollection solrCollection = request.getCollection();
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Placing replicas for Collection: {}, Shard: {}, ReplicaType: 
{}",
+                solrCollection.getName(),
+                shardName,
+                replicaType);
+          }
+          Replica pr =
+              PlacementPlugin.createProjectedReplica(solrCollection, 
shardName, replicaType, null);
+          PriorityQueue<WeightedNode> nodesForReplicaType = new 
PriorityQueue<>();
+          nodesForRequest.stream()
+              .filter(n -> n.canAddReplica(pr))
+              .forEach(
+                  n -> {
+                    n.sortByRelevantWeightWithReplica(pr);
+                    n.addToSortedCollection(nodesForReplicaType);
+                  });
+
+          int replicasPlaced = 0;
+          while (!nodesForReplicaType.isEmpty() && replicasPlaced < 
replicaCount) {
+            WeightedNode node = nodesForReplicaType.poll();
+            if (!node.canAddReplica(pr)) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node can no longer accept replica, removing from 
selection list: {}",
+                    node.getNode());
+              }
+              continue;
+            }
+            if (node.hasWeightChangedSinceSort()) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node's sort is out-of-date, adding back to selection 
list: {}",
+                    node.getNode());
+              }
+              node.addToSortedCollection(nodesForReplicaType);
+              // The node will be re-sorted,
+              // so go back to the top of the loop to get the new 
lowest-sorted node
+              continue;
+            }
+            if (log.isDebugEnabled()) {
+              log.debug("Node chosen to host replica: {}", node.getNode());
+            }
+
+            boolean needsToResortAll =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, 
node.getNode()));
+            replicasPlaced += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), 
replicaType));
+            // Only update the priorityQueue if there are still replicas to be 
placed
+            if (replicasPlaced < replicaCount) {
+              if (needsToResortAll) {
+                if (log.isDebugEnabled()) {
+                  log.debug("Replica addition requires re-sorting of entire 
selection list");
+                }
+                List<WeightedNode> nodeList = new 
ArrayList<>(nodesForReplicaType);
+                nodesForReplicaType.clear();
+                nodeList.forEach(n -> 
n.addToSortedCollection(nodesForReplicaType));
+              }
+              // Add the chosen node back to the list if it can accept another 
replica of the
+              // shard/replicaType.
+              // The default implementation of "canAddReplica()" returns false 
for replicas
+              // of shards that the node already contains, so this will 
usually be false.
+              if (node.canAddReplica(pr)) {
+                nodesForReplicaType.add(node);
+              }
+            }
+          }
+
+          if (replicasPlaced < replicaCount) {
+            throw new PlacementException(
+                String.format(
+                    Locale.ROOT,
+                    "Not enough eligible nodes to place %d replica(s) of type 
%s for shard %s of collection %s. Only able to place %d replicas.",
+                    replicaCount,
+                    replicaType,
+                    shardName,
+                    solrCollection.getName(),
+                    replicasPlaced));
+          }
+        }
+      }
+
+      placementPlans.add(
+          placementContext
+              .getPlacementPlanFactory()
+              .createPlacementPlan(request, replicaPlacements));
+    }
+    return placementPlans;
+  }
+
+  @Override
+  public BalancePlan computeBalancing(
+      BalanceRequest balanceRequest, PlacementContext placementContext) throws 
PlacementException {
+    Map<Replica, Node> replicaMovements = new HashMap<>();
+    TreeSet<WeightedNode> orderedNodes = new TreeSet<>();
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(
+                placementContext,
+                balanceRequest.getNodes(),
+                placementContext.getCluster().collections(),
+                true)
+            .values();
+    // This is critical to store the last sort weight for this node
+    weightedNodes.forEach(
+        node -> {
+          node.sortWithoutChanges();
+          node.addToSortedCollection(orderedNodes);
+        });
+
+    // While the node with the least cores still has room to take a replica 
from the node with the
+    // most cores, loop

Review Comment:
   [0] I found this comment super helpful in understanding what this method is 
doing, so I hesitate to say anything. But...
   
   Is "weight" always synonymous with having the most cores?  Assuming not, 
maybe this should be reworded to be more generic, e.g.: 
   
   > While the node with the lowest weight (e.g. least cores) still has room to 
take a replica from the node with the highest weight (e.g. most cores), loop



##########
solr/core/src/java/org/apache/solr/handler/admin/api/BalanceReplicasAPI.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.handler.admin.api;
+
+import static 
org.apache.solr.client.solrj.impl.BinaryResponseParser.BINARY_CONTENT_TYPE_V2;
+import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.params.CollectionParams.NODES;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+import static 
org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
+import static 
org.apache.solr.handler.admin.CollectionsHandler.DEFAULT_COLLECTION_OP_TIMEOUT;
+import static 
org.apache.solr.security.PermissionNameProvider.Name.COLL_EDIT_PERM;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import io.swagger.v3.oas.annotations.Operation;
+import io.swagger.v3.oas.annotations.media.Schema;
+import io.swagger.v3.oas.annotations.parameters.RequestBody;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import javax.inject.Inject;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.jersey.JacksonReflectMapWriter;
+import org.apache.solr.jersey.PermissionName;
+import org.apache.solr.jersey.SolrJerseyResponse;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+
+/** V2 API for balancing the replicas that already exist across a set of 
nodes. */
+@Path("cluster/balanceReplicas")
+public class BalanceReplicasAPI extends AdminAPIBase {
+
+  @Inject
+  public BalanceReplicasAPI(
+      CoreContainer coreContainer,
+      SolrQueryRequest solrQueryRequest,
+      SolrQueryResponse solrQueryResponse) {
+    super(coreContainer, solrQueryRequest, solrQueryResponse);
+  }
+
+  @POST
+  @Produces({"application/json", "application/xml", BINARY_CONTENT_TYPE_V2})
+  @PermissionName(COLL_EDIT_PERM)
+  @Operation(summary = "Balance Replicas across the given set of Nodes.")
+  public SolrJerseyResponse balanceReplicas(
+      @RequestBody(description = "Contains user provided parameters")
+          BalanceReplicasRequestBody requestBody)
+      throws Exception {
+    final SolrJerseyResponse response = 
instantiateJerseyResponse(SolrJerseyResponse.class);
+    final CoreContainer coreContainer = 
fetchAndValidateZooKeeperAwareCoreContainer();
+    // TODO Record node for log and tracing
+    final ZkNodeProps remoteMessage = createRemoteMessage(requestBody);
+    final SolrResponse remoteResponse =
+        CollectionsHandler.submitCollectionApiCommand(
+            coreContainer,
+            coreContainer.getDistributedCollectionCommandRunner(),
+            remoteMessage,
+            CollectionAction.BALANCE_REPLICAS,
+            DEFAULT_COLLECTION_OP_TIMEOUT);
+    if (remoteResponse.getException() != null) {
+      throw remoteResponse.getException();
+    }
+
+    disableResponseCaching();
+    return response;
+  }
+
+  public ZkNodeProps createRemoteMessage(BalanceReplicasRequestBody 
requestBody) {
+    final Map<String, Object> remoteMessage = new HashMap<>();
+    if (requestBody != null) {
+      insertIfValueNotNull(remoteMessage, NODES, requestBody.nodes);
+      insertIfValueNotNull(remoteMessage, WAIT_FOR_FINAL_STATE, 
requestBody.waitForFinalState);
+      insertIfValueNotNull(remoteMessage, ASYNC, requestBody.async);
+    }
+    remoteMessage.put(QUEUE_OPERATION, 
CollectionAction.BALANCE_REPLICAS.toLower());
+
+    return new ZkNodeProps(remoteMessage);
+  }
+
+  private void insertIfValueNotNull(Map<String, Object> dest, String key, 
Object value) {

Review Comment:
   [-1] This method already exists in `AdminAPIBase`.  Could you use the 
version there and delete this?



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,604 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.plugins;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalanceRequest;
+import org.apache.solr.cluster.placement.DeleteCollectionRequest;
+import org.apache.solr.cluster.placement.DeleteReplicasRequest;
+import org.apache.solr.cluster.placement.DeleteShardsRequest;
+import org.apache.solr.cluster.placement.ModificationRequest;
+import org.apache.solr.cluster.placement.PlacementContext;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementModificationException;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.common.util.CollectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class OrderedNodePlacementPlugin implements PlacementPlugin {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Override
+  public List<PlacementPlan> computePlacements(
+      Collection<PlacementRequest> requests, PlacementContext placementContext)
+      throws PlacementException {
+    List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
+    Set<Node> allNodes = new HashSet<>();
+    Set<SolrCollection> allCollections = new HashSet<>();
+    for (PlacementRequest request : requests) {
+      allNodes.addAll(request.getTargetNodes());
+      allCollections.add(request.getCollection());
+    }
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(placementContext, allNodes, allCollections, 
true).values();
+    for (PlacementRequest request : requests) {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      List<WeightedNode> nodesForRequest =
+          weightedNodes.stream()
+              .filter(wn -> request.getTargetNodes().contains(wn.getNode()))
+              .collect(Collectors.toList());
+
+      Set<ReplicaPlacement> replicaPlacements =
+          CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
+
+      SolrCollection solrCollection = request.getCollection();
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Placing replicas for Collection: {}, Shard: {}, ReplicaType: 
{}",
+                solrCollection.getName(),
+                shardName,
+                replicaType);
+          }
+          Replica pr =
+              PlacementPlugin.createProjectedReplica(solrCollection, 
shardName, replicaType, null);
+          PriorityQueue<WeightedNode> nodesForReplicaType = new 
PriorityQueue<>();
+          nodesForRequest.stream()
+              .filter(n -> n.canAddReplica(pr))
+              .forEach(
+                  n -> {
+                    n.sortByRelevantWeightWithReplica(pr);
+                    n.addToSortedCollection(nodesForReplicaType);
+                  });
+
+          int replicasPlaced = 0;
+          while (!nodesForReplicaType.isEmpty() && replicasPlaced < 
replicaCount) {
+            WeightedNode node = nodesForReplicaType.poll();
+            if (!node.canAddReplica(pr)) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node can no longer accept replica, removing from 
selection list: {}",
+                    node.getNode());
+              }
+              continue;
+            }
+            if (node.hasWeightChangedSinceSort()) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node's sort is out-of-date, adding back to selection 
list: {}",
+                    node.getNode());
+              }
+              node.addToSortedCollection(nodesForReplicaType);
+              // The node will be re-sorted,
+              // so go back to the top of the loop to get the new 
lowest-sorted node
+              continue;
+            }
+            if (log.isDebugEnabled()) {
+              log.debug("Node chosen to host replica: {}", node.getNode());
+            }
+
+            boolean needsToResortAll =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, 
node.getNode()));
+            replicasPlaced += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), 
replicaType));
+            // Only update the priorityQueue if there are still replicas to be 
placed
+            if (replicasPlaced < replicaCount) {
+              if (needsToResortAll) {
+                if (log.isDebugEnabled()) {
+                  log.debug("Replica addition requires re-sorting of entire 
selection list");
+                }
+                List<WeightedNode> nodeList = new 
ArrayList<>(nodesForReplicaType);
+                nodesForReplicaType.clear();
+                nodeList.forEach(n -> 
n.addToSortedCollection(nodesForReplicaType));
+              }
+              // Add the chosen node back to the list if it can accept another 
replica of the
+              // shard/replicaType.
+              // The default implementation of "canAddReplica()" returns false 
for replicas
+              // of shards that the node already contains, so this will 
usually be false.
+              if (node.canAddReplica(pr)) {
+                nodesForReplicaType.add(node);
+              }
+            }
+          }
+
+          if (replicasPlaced < replicaCount) {
+            throw new PlacementException(
+                String.format(
+                    Locale.ROOT,
+                    "Not enough eligible nodes to place %d replica(s) of type 
%s for shard %s of collection %s. Only able to place %d replicas.",
+                    replicaCount,
+                    replicaType,
+                    shardName,
+                    solrCollection.getName(),
+                    replicasPlaced));
+          }
+        }
+      }
+
+      placementPlans.add(
+          placementContext
+              .getPlacementPlanFactory()
+              .createPlacementPlan(request, replicaPlacements));
+    }
+    return placementPlans;
+  }
+
+  @Override
+  public BalancePlan computeBalancing(
+      BalanceRequest balanceRequest, PlacementContext placementContext) throws 
PlacementException {
+    Map<Replica, Node> replicaMovements = new HashMap<>();
+    TreeSet<WeightedNode> orderedNodes = new TreeSet<>();
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(
+                placementContext,
+                balanceRequest.getNodes(),
+                placementContext.getCluster().collections(),
+                true)
+            .values();
+    // This is critical to store the last sort weight for this node
+    weightedNodes.forEach(
+        node -> {
+          node.sortWithoutChanges();
+          node.addToSortedCollection(orderedNodes);
+        });
+
+    // While the node with the least cores still has room to take a replica 
from the node with the
+    // most cores, loop
+    Map<Replica, Node> newReplicaMovements = new HashMap<>();
+    ArrayList<WeightedNode> traversedHighNodes = new 
ArrayList<>(orderedNodes.size() - 1);
+    while (orderedNodes.size() > 1
+        && orderedNodes.first().calcWeight() < 
orderedNodes.last().calcWeight()) {
+      WeightedNode lowestWeight = orderedNodes.pollFirst();
+      if (lowestWeight == null) {
+        break;
+      }
+      if (lowestWeight.hasWeightChangedSinceSort()) {
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Re-sorting lowest weighted node: {}, sorting weight is 
out-of-date.",
+              lowestWeight.getNode().getName());
+        }
+        // Re-sort this node and go back to find the lowest weight
+        lowestWeight.addToSortedCollection(orderedNodes);
+        continue;
+      }
+      if (log.isDebugEnabled()) {
+        log.debug(
+            "Lowest weighted node: {}, weight: {}",
+            lowestWeight.getNode().getName(),
+            lowestWeight.calcWeight());
+      }
+
+      newReplicaMovements.clear();
+      // If a compatible node was found to move replicas, break and find the 
lowest weighted node
+      // again
+      while (newReplicaMovements.isEmpty()
+          && !orderedNodes.isEmpty()
+          && orderedNodes.last().calcWeight() > lowestWeight.calcWeight() + 1) 
{
+        WeightedNode highestWeight = orderedNodes.pollLast();
+        if (highestWeight == null) {
+          break;
+        }
+        if (highestWeight.hasWeightChangedSinceSort()) {
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Re-sorting highest weighted node: {}, sorting weight is 
out-of-date.",
+                highestWeight.getNode().getName());
+          }
+          // Re-sort this node and go back to find the highest weight
+          highestWeight.addToSortedCollection(orderedNodes);
+          continue;
+        }
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Highest weighted node: {}, weight: {}",
+              highestWeight.getNode().getName(),
+              highestWeight.calcWeight());
+        }
+
+        traversedHighNodes.add(highestWeight);
+        // select a replica from the node with the most cores to move to the 
node with the least
+        // cores
+        Set<Replica> availableReplicasToMove = 
highestWeight.getAllReplicasOnNode();
+        int combinedNodeWeights = highestWeight.calcWeight() + 
lowestWeight.calcWeight();
+        for (Replica r : availableReplicasToMove) {
+          // Only continue if the replica can be removed from the old node and 
moved to the new node
+          if (!highestWeight.canRemoveReplicas(Set.of(r)).isEmpty()
+              || !lowestWeight.canAddReplica(r)) {
+            continue;
+          }
+          lowestWeight.addReplica(r);
+          highestWeight.removeReplica(r);
+          int lowestWeightWithReplica = lowestWeight.calcWeight();
+          int highestWeightWithoutReplica = highestWeight.calcWeight();
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Replica: {}, toNode weight with replica: {}, fromNode weight 
without replica: {}",
+                r.getReplicaName(),
+                lowestWeightWithReplica,
+                highestWeightWithoutReplica);
+          }
+
+          // If the combined weight of both nodes is lower after the move, 
make the move.
+          // Otherwise, make the move if it doesn't cause the weight of the 
higher node to
+          // go below the weight of the lower node, because that is 
over-correction.

Review Comment:
   [+1] Really helpful comments throughout, but especially here.



##########
solr/core/src/java/org/apache/solr/cluster/placement/impl/BalancePlanFactoryImpl.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.impl;
+
+import java.util.Map;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalancePlanFactory;
+import org.apache.solr.cluster.placement.BalanceRequest;
+
+/** Simple implementation of {@link BalancePlanFactory}. */
+public class BalancePlanFactoryImpl implements BalancePlanFactory {

Review Comment:
   [0] I understand there are few (1?) implementation for BalancePlanFactory 
and BalancePlan currently, but this name might end up confusing if Solr ever 
gets multiple impl's.  Or are there unlikely to ever be other 
BalancePlanFactory implementations?
   
   Idt there's any great options, but maybe something along the lines of 
`MovementMapConsumingBalancePlanFactory`?  That's what seems to differentiate 
this class, that it takes in the movements largely pre-decided, right?



##########
solr/core/src/java/org/apache/solr/cluster/placement/PlacementContext.java:
##########
@@ -39,4 +39,7 @@ public interface PlacementContext {
 
   /** Factory used to create instances of {@link PlacementPlan} to return 
computed decision. */
   PlacementPlanFactory getPlacementPlanFactory();
+
+  /** Factory used to create instances of {@link PlacementPlan} to return 
computed decision. */

Review Comment:
   [0] Copy paste mistake: replace "PlacementPlan" with "BalancePlan"



##########
solr/core/src/java/org/apache/solr/cluster/placement/PlacementPlugin.java:
##########
@@ -85,5 +104,80 @@ List<PlacementPlan> computePlacements(
    */
   default void verifyAllowedModification(
       ModificationRequest modificationRequest, PlacementContext 
placementContext)
-      throws PlacementModificationException, InterruptedException {}
+      throws PlacementException, InterruptedException {}
+
+  static Replica createProjectedReplica(

Review Comment:
   [0] I gather that this method is used to create a Replica instance 
representing what a replica will look like after it has been moved (but before 
that has actually happened).
   
   But it took a bit of digging to get there.  So this might be a candidate for 
some javadocs, if you think it's something other code might want to reuse at 
some point.



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/SimplePlacementFactory.java:
##########
@@ -53,118 +43,116 @@ public PlacementPlugin createPluginInstance() {
     return new SimplePlacementPlugin();
   }
 
-  public static class SimplePlacementPlugin implements PlacementPlugin {
-    @Override
-    public List<PlacementPlan> computePlacements(
-        Collection<PlacementRequest> requests, PlacementContext 
placementContext)
-        throws PlacementException {
-      List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
-      Map<Node, ReplicaCount> nodeVsShardCount = 
getNodeVsShardCount(placementContext);
-      for (PlacementRequest request : requests) {
-        int totalReplicasPerShard = 0;
-        for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-          totalReplicasPerShard += request.getCountReplicasToCreate(rt);
-        }
-
-        Set<ReplicaPlacement> replicaPlacements =
-            CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
-
-        Collection<ReplicaCount> replicaCounts = nodeVsShardCount.values();
-
-        if (request.getTargetNodes().size() < replicaCounts.size()) {
-          replicaCounts =
-              replicaCounts.stream()
-                  .filter(rc -> request.getTargetNodes().contains(rc.node()))
-                  .collect(Collectors.toList());
-        }
-
-        for (String shard : request.getShardNames()) {
-          // Reset the ordering of the nodes for each shard, using the 
replicas added in the
-          // previous shards and assign requests
-          List<Node> nodeList =
-              replicaCounts.stream()
-                  .sorted(
-                      Comparator.<ReplicaCount>comparingInt(
-                              rc -> 
rc.weight(request.getCollection().getName()))
-                          .thenComparing(ReplicaCount::nodeName))
-                  .map(ReplicaCount::node)
-                  .collect(Collectors.toList());
-          int replicaNumOfShard = 0;
-          for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) 
{
-            for (int i = 0; i < request.getCountReplicasToCreate(replicaType); 
i++) {
-              Node assignedNode = nodeList.get(replicaNumOfShard++ % 
nodeList.size());
-
-              replicaPlacements.add(
-                  placementContext
-                      .getPlacementPlanFactory()
-                      .createReplicaPlacement(
-                          request.getCollection(), shard, assignedNode, 
replicaType));
-
-              ReplicaCount replicaCount =
-                  nodeVsShardCount.computeIfAbsent(assignedNode, 
ReplicaCount::new);
-              replicaCount.totalReplicas++;
-              replicaCount.collectionReplicas.merge(
-                  request.getCollection().getName(), 1, Integer::sum);
-            }
-          }
-        }
-
-        placementPlans.add(
-            placementContext
-                .getPlacementPlanFactory()
-                .createPlacementPlan(request, replicaPlacements));
-      }
-      return placementPlans;
-    }
+  public static class SimplePlacementPlugin extends OrderedNodePlacementPlugin 
{
 
-    private Map<Node, ReplicaCount> getNodeVsShardCount(PlacementContext 
placementContext) {
-      HashMap<Node, ReplicaCount> nodeVsShardCount = new HashMap<>();
-
-      for (Node s : placementContext.getCluster().getLiveDataNodes()) {
-        nodeVsShardCount.computeIfAbsent(s, ReplicaCount::new);
+    @Override
+    protected Map<Node, WeightedNode> getBaseWeightedNodes(
+        PlacementContext placementContext,
+        Set<Node> nodes,
+        Iterable<SolrCollection> relevantCollections,
+        boolean skipNodesWithErrors) {
+      HashMap<Node, WeightedNode> nodeVsShardCount = new HashMap<>();
+
+      for (Node n : nodes) {
+        nodeVsShardCount.computeIfAbsent(n, SameCollWeightedNode::new);
       }
 
-      // if we get here we were not given a createNodeList, build a map with 
real counts.
-      for (SolrCollection collection : 
placementContext.getCluster().collections()) {
-        // identify suitable nodes  by checking the no:of cores in each of them
-        for (Shard shard : collection.shards()) {
-          for (Replica replica : shard.replicas()) {
-            ReplicaCount count = nodeVsShardCount.get(replica.getNode());
-            if (count != null) {
-              count.addReplica(collection.getName(), shard.getShardName());
-            }
-          }
-        }
-      }
       return nodeVsShardCount;
     }
   }
 
-  static class ReplicaCount {
-    public final Node node;
+  private static class SameCollWeightedNode extends 
OrderedNodePlacementPlugin.WeightedNode {
+    private static final int SAME_COL_MULT = 5;
+    private static final int SAME_SHARD_MULT = 1000;
     public Map<String, Integer> collectionReplicas;
-    public int totalReplicas = 0;
+    public int totalWeight = 0;
 
-    ReplicaCount(Node node) {
-      this.node = node;
+    SameCollWeightedNode(Node node) {
+      super(node);
       this.collectionReplicas = new HashMap<>();
     }
 
-    public int weight(String collection) {
-      return (collectionReplicas.getOrDefault(collection, 0) * 5) + 
totalReplicas;
+    /**
+     * The weight of the SameCollWeightedNode is the sum of:

Review Comment:
   [Q] Did this formula pre-exist this PR somewhere, or is it new?  If it's 
new, is there a rationale or theory behind to coefficients and the math more 
generally?
   
   (I looked through AffinityPlacementFactory and a few other places, but 
couldn't find it, but it's a big PR so very possible I missed it)



##########
solr/core/src/test/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactoryTest.java:
##########
@@ -1443,60 +1445,6 @@ private void testSpreadDomains(boolean 
hasExistingCollection) throws Exception {
     }
   }
 
-  @Test
-  @SuppressWarnings("SelfComparison")
-  public void testCompareSpreadDomainWithNodes() {

Review Comment:
   [Q] Are we deleting this test because the functionality will go away in 
10.x?  Or has it moved somewhere that I've missed...



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,604 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.plugins;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalanceRequest;
+import org.apache.solr.cluster.placement.DeleteCollectionRequest;
+import org.apache.solr.cluster.placement.DeleteReplicasRequest;
+import org.apache.solr.cluster.placement.DeleteShardsRequest;
+import org.apache.solr.cluster.placement.ModificationRequest;
+import org.apache.solr.cluster.placement.PlacementContext;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementModificationException;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.common.util.CollectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class OrderedNodePlacementPlugin implements PlacementPlugin {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Override
+  public List<PlacementPlan> computePlacements(
+      Collection<PlacementRequest> requests, PlacementContext placementContext)
+      throws PlacementException {
+    List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
+    Set<Node> allNodes = new HashSet<>();
+    Set<SolrCollection> allCollections = new HashSet<>();
+    for (PlacementRequest request : requests) {
+      allNodes.addAll(request.getTargetNodes());
+      allCollections.add(request.getCollection());
+    }
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(placementContext, allNodes, allCollections, 
true).values();
+    for (PlacementRequest request : requests) {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      List<WeightedNode> nodesForRequest =
+          weightedNodes.stream()
+              .filter(wn -> request.getTargetNodes().contains(wn.getNode()))
+              .collect(Collectors.toList());
+
+      Set<ReplicaPlacement> replicaPlacements =
+          CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
+
+      SolrCollection solrCollection = request.getCollection();
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Placing replicas for Collection: {}, Shard: {}, ReplicaType: 
{}",
+                solrCollection.getName(),
+                shardName,
+                replicaType);
+          }
+          Replica pr =
+              PlacementPlugin.createProjectedReplica(solrCollection, 
shardName, replicaType, null);
+          PriorityQueue<WeightedNode> nodesForReplicaType = new 
PriorityQueue<>();
+          nodesForRequest.stream()
+              .filter(n -> n.canAddReplica(pr))
+              .forEach(
+                  n -> {
+                    n.sortByRelevantWeightWithReplica(pr);
+                    n.addToSortedCollection(nodesForReplicaType);
+                  });
+
+          int replicasPlaced = 0;
+          while (!nodesForReplicaType.isEmpty() && replicasPlaced < 
replicaCount) {
+            WeightedNode node = nodesForReplicaType.poll();
+            if (!node.canAddReplica(pr)) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node can no longer accept replica, removing from 
selection list: {}",
+                    node.getNode());
+              }
+              continue;
+            }
+            if (node.hasWeightChangedSinceSort()) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node's sort is out-of-date, adding back to selection 
list: {}",
+                    node.getNode());
+              }
+              node.addToSortedCollection(nodesForReplicaType);
+              // The node will be re-sorted,
+              // so go back to the top of the loop to get the new 
lowest-sorted node
+              continue;
+            }
+            if (log.isDebugEnabled()) {
+              log.debug("Node chosen to host replica: {}", node.getNode());
+            }
+
+            boolean needsToResortAll =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, 
node.getNode()));
+            replicasPlaced += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), 
replicaType));
+            // Only update the priorityQueue if there are still replicas to be 
placed
+            if (replicasPlaced < replicaCount) {
+              if (needsToResortAll) {
+                if (log.isDebugEnabled()) {
+                  log.debug("Replica addition requires re-sorting of entire 
selection list");
+                }
+                List<WeightedNode> nodeList = new 
ArrayList<>(nodesForReplicaType);
+                nodesForReplicaType.clear();
+                nodeList.forEach(n -> 
n.addToSortedCollection(nodesForReplicaType));
+              }
+              // Add the chosen node back to the list if it can accept another 
replica of the
+              // shard/replicaType.
+              // The default implementation of "canAddReplica()" returns false 
for replicas
+              // of shards that the node already contains, so this will 
usually be false.
+              if (node.canAddReplica(pr)) {
+                nodesForReplicaType.add(node);
+              }
+            }
+          }
+
+          if (replicasPlaced < replicaCount) {
+            throw new PlacementException(
+                String.format(
+                    Locale.ROOT,
+                    "Not enough eligible nodes to place %d replica(s) of type 
%s for shard %s of collection %s. Only able to place %d replicas.",
+                    replicaCount,
+                    replicaType,
+                    shardName,
+                    solrCollection.getName(),
+                    replicasPlaced));
+          }
+        }
+      }
+
+      placementPlans.add(
+          placementContext
+              .getPlacementPlanFactory()
+              .createPlacementPlan(request, replicaPlacements));
+    }
+    return placementPlans;
+  }
+
+  @Override
+  public BalancePlan computeBalancing(
+      BalanceRequest balanceRequest, PlacementContext placementContext) throws 
PlacementException {
+    Map<Replica, Node> replicaMovements = new HashMap<>();
+    TreeSet<WeightedNode> orderedNodes = new TreeSet<>();
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(
+                placementContext,
+                balanceRequest.getNodes(),
+                placementContext.getCluster().collections(),
+                true)
+            .values();
+    // This is critical to store the last sort weight for this node
+    weightedNodes.forEach(
+        node -> {
+          node.sortWithoutChanges();
+          node.addToSortedCollection(orderedNodes);
+        });
+
+    // While the node with the least cores still has room to take a replica 
from the node with the
+    // most cores, loop
+    Map<Replica, Node> newReplicaMovements = new HashMap<>();
+    ArrayList<WeightedNode> traversedHighNodes = new 
ArrayList<>(orderedNodes.size() - 1);
+    while (orderedNodes.size() > 1
+        && orderedNodes.first().calcWeight() < 
orderedNodes.last().calcWeight()) {
+      WeightedNode lowestWeight = orderedNodes.pollFirst();
+      if (lowestWeight == null) {
+        break;
+      }
+      if (lowestWeight.hasWeightChangedSinceSort()) {
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Re-sorting lowest weighted node: {}, sorting weight is 
out-of-date.",
+              lowestWeight.getNode().getName());
+        }
+        // Re-sort this node and go back to find the lowest weight
+        lowestWeight.addToSortedCollection(orderedNodes);
+        continue;
+      }
+      if (log.isDebugEnabled()) {
+        log.debug(
+            "Lowest weighted node: {}, weight: {}",
+            lowestWeight.getNode().getName(),
+            lowestWeight.calcWeight());
+      }
+
+      newReplicaMovements.clear();
+      // If a compatible node was found to move replicas, break and find the 
lowest weighted node
+      // again
+      while (newReplicaMovements.isEmpty()
+          && !orderedNodes.isEmpty()
+          && orderedNodes.last().calcWeight() > lowestWeight.calcWeight() + 1) 
{
+        WeightedNode highestWeight = orderedNodes.pollLast();
+        if (highestWeight == null) {
+          break;
+        }
+        if (highestWeight.hasWeightChangedSinceSort()) {
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Re-sorting highest weighted node: {}, sorting weight is 
out-of-date.",
+                highestWeight.getNode().getName());
+          }
+          // Re-sort this node and go back to find the highest weight
+          highestWeight.addToSortedCollection(orderedNodes);
+          continue;
+        }
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Highest weighted node: {}, weight: {}",
+              highestWeight.getNode().getName(),
+              highestWeight.calcWeight());
+        }
+
+        traversedHighNodes.add(highestWeight);
+        // select a replica from the node with the most cores to move to the 
node with the least
+        // cores
+        Set<Replica> availableReplicasToMove = 
highestWeight.getAllReplicasOnNode();
+        int combinedNodeWeights = highestWeight.calcWeight() + 
lowestWeight.calcWeight();
+        for (Replica r : availableReplicasToMove) {
+          // Only continue if the replica can be removed from the old node and 
moved to the new node
+          if (!highestWeight.canRemoveReplicas(Set.of(r)).isEmpty()
+              || !lowestWeight.canAddReplica(r)) {
+            continue;
+          }
+          lowestWeight.addReplica(r);
+          highestWeight.removeReplica(r);
+          int lowestWeightWithReplica = lowestWeight.calcWeight();
+          int highestWeightWithoutReplica = highestWeight.calcWeight();
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Replica: {}, toNode weight with replica: {}, fromNode weight 
without replica: {}",
+                r.getReplicaName(),
+                lowestWeightWithReplica,
+                highestWeightWithoutReplica);
+          }
+
+          // If the combined weight of both nodes is lower after the move, 
make the move.
+          // Otherwise, make the move if it doesn't cause the weight of the 
higher node to
+          // go below the weight of the lower node, because that is 
over-correction.
+          if (highestWeightWithoutReplica + lowestWeightWithReplica >= 
combinedNodeWeights
+              && highestWeightWithoutReplica < lowestWeightWithReplica) {
+            // Undo the move
+            lowestWeight.removeReplica(r);
+            highestWeight.addReplica(r);
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Replica Movement chosen. From: {}, To: {}, Replica: {}",
+                highestWeight.getNode().getName(),
+                lowestWeight.getNode().getName(),
+                r);
+          }
+          newReplicaMovements.put(r, lowestWeight.getNode());
+
+          // Do not go beyond here, do another loop and see if other nodes can 
move replicas.
+          // It might end up being the same nodes in the next loop that end up 
moving another
+          // replica, but that's ok.
+          break;
+        }
+      }
+      traversedHighNodes.forEach(n -> n.addToSortedCollection(orderedNodes));
+      traversedHighNodes.clear();
+      if (newReplicaMovements.size() > 0) {
+        replicaMovements.putAll(newReplicaMovements);
+        // There are no replicas to move to the lowestWeight, remove it from 
our loop
+        lowestWeight.addToSortedCollection(orderedNodes);
+      }
+    }
+
+    return placementContext
+        .getBalancePlanFactory()
+        .createBalancePlan(balanceRequest, replicaMovements);
+  }
+
+  protected Map<Node, WeightedNode> getWeightedNodes(
+      PlacementContext placementContext,
+      Set<Node> nodes,
+      Iterable<SolrCollection> relevantCollections,
+      boolean skipNodesWithErrors)
+      throws PlacementException {
+    Map<Node, WeightedNode> weightedNodes =
+        getBaseWeightedNodes(placementContext, nodes, relevantCollections, 
skipNodesWithErrors);
+
+    for (SolrCollection collection : 
placementContext.getCluster().collections()) {
+      for (Shard shard : collection.shards()) {
+        for (Replica replica : shard.replicas()) {
+          WeightedNode weightedNode = weightedNodes.get(replica.getNode());
+          if (weightedNode != null) {
+            weightedNode.initReplica(replica);
+          }
+        }
+      }
+    }
+
+    return weightedNodes;
+  }
+
+  protected abstract Map<Node, WeightedNode> getBaseWeightedNodes(
+      PlacementContext placementContext,
+      Set<Node> nodes,
+      Iterable<SolrCollection> relevantCollections,
+      boolean skipNodesWithErrors)
+      throws PlacementException;
+
+  @Override
+  public void verifyAllowedModification(
+      ModificationRequest modificationRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    if (modificationRequest instanceof DeleteShardsRequest) {
+      log.warn("DeleteShardsRequest not implemented yet, skipping: {}", 
modificationRequest);
+    } else if (modificationRequest instanceof DeleteCollectionRequest) {
+      verifyDeleteCollection((DeleteCollectionRequest) modificationRequest, 
placementContext);
+    } else if (modificationRequest instanceof DeleteReplicasRequest) {
+      verifyDeleteReplicas((DeleteReplicasRequest) modificationRequest, 
placementContext);
+    } else {
+      log.warn("unsupported request type, skipping: {}", modificationRequest);
+    }
+  }
+
+  protected void verifyDeleteCollection(
+      DeleteCollectionRequest deleteCollectionRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    // NO-OP
+  }
+
+  protected void verifyDeleteReplicas(
+      DeleteReplicasRequest deleteReplicasRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    Map<Node, List<Replica>> nodesRepresented =
+        deleteReplicasRequest.getReplicas().stream()
+            .collect(Collectors.groupingBy(Replica::getNode));
+
+    Map<Node, WeightedNode> weightedNodes =
+        getWeightedNodes(
+            placementContext,
+            nodesRepresented.keySet(),
+            placementContext.getCluster().collections(),
+            false);
+
+    PlacementModificationException placementModificationException =
+        new PlacementModificationException("delete replica(s) rejected");
+    for (Map.Entry<Node, List<Replica>> entry : nodesRepresented.entrySet()) {
+      WeightedNode node = weightedNodes.get(entry.getKey());
+      if (node == null) {
+        entry
+            .getValue()
+            .forEach(
+                replica ->
+                    placementModificationException.addRejectedModification(
+                        replica.toString(),
+                        "could not load information for node: " + 
entry.getKey().getName()));
+      } else {
+        node.canRemoveReplicas(entry.getValue())
+            .forEach(
+                (replica, reason) ->
+                    placementModificationException.addRejectedModification(
+                        replica.toString(), reason));
+      }
+    }
+    if (!placementModificationException.getRejectedModifications().isEmpty()) {
+      throw placementModificationException;
+    }
+  }
+
+  /**
+   * A class that determines the weight of a given node and the replicas that 
reside on it.
+   *
+   * <p>The OrderedNodePlacementPlugin uses the weights determined here to 
place and balance
+   * replicas across the cluster.
+   *
+   * @lucene.experimental
+   */
+  public abstract static class WeightedNode implements 
Comparable<WeightedNode> {
+    private final Node node;
+    private final Map<String, Map<String, Set<Replica>>> replicas;
+    private IntSupplier sortWeightCalculator;
+    private int lastSortedWeight;
+
+    public WeightedNode(Node node) {
+      this.node = node;
+      this.replicas = new HashMap<>();
+      this.lastSortedWeight = 0;
+      this.sortWeightCalculator = this::calcWeight;
+    }
+
+    public void sortByRelevantWeightWithReplica(Replica replica) {
+      sortWeightCalculator = () -> calcRelevantWeightWithReplica(replica);
+    }
+
+    public void sortWithoutChanges() {
+      sortWeightCalculator = this::calcWeight;
+    }
+
+    public Node getNode() {
+      return node;
+    }
+
+    public Set<Replica> getAllReplicasOnNode() {
+      return replicas.values().stream()
+          .flatMap(shard -> shard.values().stream())
+          .flatMap(Collection::stream)
+          .collect(Collectors.toSet());
+    }
+
+    public Set<String> getCollectionsOnNode() {
+      return replicas.keySet();
+    }
+
+    public boolean hasCollectionOnNode(String collection) {
+      return replicas.containsKey(collection);
+    }
+
+    public Set<String> getShardsOnNode(String collection) {
+      return replicas.getOrDefault(collection, 
Collections.emptyMap()).keySet();
+    }
+
+    public boolean hasShardOnNode(Shard shard) {
+      return replicas
+          .getOrDefault(shard.getCollection().getName(), 
Collections.emptyMap())
+          .containsKey(shard.getShardName());
+    }
+
+    public Set<Replica> getReplicasForShardOnNode(Shard shard) {
+      return Optional.ofNullable(replicas.get(shard.getCollection().getName()))
+          .map(m -> m.get(shard.getShardName()))
+          .orElseGet(Collections::emptySet);
+    }
+
+    public void addToSortedCollection(Collection<WeightedNode> collection) {

Review Comment:
   [0] Would there be any value in narrowing the parameter type accepted by 
this method (e.g. to SortedSet, etc.) to ensure that callers aren't passing in 
an unordered Collection implementation.
   
   As-is, this method accepts a bunch of valid Collection types that would 
quietly "succeed", but not do what this method intended at all.



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,604 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF 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.apache.solr.cluster.placement.plugins;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.PriorityQueue;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.IntSupplier;
+import java.util.stream.Collectors;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+import org.apache.solr.cluster.placement.BalancePlan;
+import org.apache.solr.cluster.placement.BalanceRequest;
+import org.apache.solr.cluster.placement.DeleteCollectionRequest;
+import org.apache.solr.cluster.placement.DeleteReplicasRequest;
+import org.apache.solr.cluster.placement.DeleteShardsRequest;
+import org.apache.solr.cluster.placement.ModificationRequest;
+import org.apache.solr.cluster.placement.PlacementContext;
+import org.apache.solr.cluster.placement.PlacementException;
+import org.apache.solr.cluster.placement.PlacementModificationException;
+import org.apache.solr.cluster.placement.PlacementPlan;
+import org.apache.solr.cluster.placement.PlacementPlugin;
+import org.apache.solr.cluster.placement.PlacementRequest;
+import org.apache.solr.cluster.placement.ReplicaPlacement;
+import org.apache.solr.common.util.CollectionUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class OrderedNodePlacementPlugin implements PlacementPlugin {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @Override
+  public List<PlacementPlan> computePlacements(
+      Collection<PlacementRequest> requests, PlacementContext placementContext)
+      throws PlacementException {
+    List<PlacementPlan> placementPlans = new ArrayList<>(requests.size());
+    Set<Node> allNodes = new HashSet<>();
+    Set<SolrCollection> allCollections = new HashSet<>();
+    for (PlacementRequest request : requests) {
+      allNodes.addAll(request.getTargetNodes());
+      allCollections.add(request.getCollection());
+    }
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(placementContext, allNodes, allCollections, 
true).values();
+    for (PlacementRequest request : requests) {
+      int totalReplicasPerShard = 0;
+      for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
+        totalReplicasPerShard += request.getCountReplicasToCreate(rt);
+      }
+
+      List<WeightedNode> nodesForRequest =
+          weightedNodes.stream()
+              .filter(wn -> request.getTargetNodes().contains(wn.getNode()))
+              .collect(Collectors.toList());
+
+      Set<ReplicaPlacement> replicaPlacements =
+          CollectionUtil.newHashSet(totalReplicasPerShard * 
request.getShardNames().size());
+
+      SolrCollection solrCollection = request.getCollection();
+      // Now place randomly all replicas of all shards on available nodes
+      for (String shardName : request.getShardNames()) {
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Placing replicas for Collection: {}, Shard: {}, ReplicaType: 
{}",
+                solrCollection.getName(),
+                shardName,
+                replicaType);
+          }
+          Replica pr =
+              PlacementPlugin.createProjectedReplica(solrCollection, 
shardName, replicaType, null);
+          PriorityQueue<WeightedNode> nodesForReplicaType = new 
PriorityQueue<>();
+          nodesForRequest.stream()
+              .filter(n -> n.canAddReplica(pr))
+              .forEach(
+                  n -> {
+                    n.sortByRelevantWeightWithReplica(pr);
+                    n.addToSortedCollection(nodesForReplicaType);
+                  });
+
+          int replicasPlaced = 0;
+          while (!nodesForReplicaType.isEmpty() && replicasPlaced < 
replicaCount) {
+            WeightedNode node = nodesForReplicaType.poll();
+            if (!node.canAddReplica(pr)) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node can no longer accept replica, removing from 
selection list: {}",
+                    node.getNode());
+              }
+              continue;
+            }
+            if (node.hasWeightChangedSinceSort()) {
+              if (log.isDebugEnabled()) {
+                log.debug(
+                    "Node's sort is out-of-date, adding back to selection 
list: {}",
+                    node.getNode());
+              }
+              node.addToSortedCollection(nodesForReplicaType);
+              // The node will be re-sorted,
+              // so go back to the top of the loop to get the new 
lowest-sorted node
+              continue;
+            }
+            if (log.isDebugEnabled()) {
+              log.debug("Node chosen to host replica: {}", node.getNode());
+            }
+
+            boolean needsToResortAll =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, 
node.getNode()));
+            replicasPlaced += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), 
replicaType));
+            // Only update the priorityQueue if there are still replicas to be 
placed
+            if (replicasPlaced < replicaCount) {
+              if (needsToResortAll) {
+                if (log.isDebugEnabled()) {
+                  log.debug("Replica addition requires re-sorting of entire 
selection list");
+                }
+                List<WeightedNode> nodeList = new 
ArrayList<>(nodesForReplicaType);
+                nodesForReplicaType.clear();
+                nodeList.forEach(n -> 
n.addToSortedCollection(nodesForReplicaType));
+              }
+              // Add the chosen node back to the list if it can accept another 
replica of the
+              // shard/replicaType.
+              // The default implementation of "canAddReplica()" returns false 
for replicas
+              // of shards that the node already contains, so this will 
usually be false.
+              if (node.canAddReplica(pr)) {
+                nodesForReplicaType.add(node);
+              }
+            }
+          }
+
+          if (replicasPlaced < replicaCount) {
+            throw new PlacementException(
+                String.format(
+                    Locale.ROOT,
+                    "Not enough eligible nodes to place %d replica(s) of type 
%s for shard %s of collection %s. Only able to place %d replicas.",
+                    replicaCount,
+                    replicaType,
+                    shardName,
+                    solrCollection.getName(),
+                    replicasPlaced));
+          }
+        }
+      }
+
+      placementPlans.add(
+          placementContext
+              .getPlacementPlanFactory()
+              .createPlacementPlan(request, replicaPlacements));
+    }
+    return placementPlans;
+  }
+
+  @Override
+  public BalancePlan computeBalancing(
+      BalanceRequest balanceRequest, PlacementContext placementContext) throws 
PlacementException {
+    Map<Replica, Node> replicaMovements = new HashMap<>();
+    TreeSet<WeightedNode> orderedNodes = new TreeSet<>();
+    Collection<WeightedNode> weightedNodes =
+        getWeightedNodes(
+                placementContext,
+                balanceRequest.getNodes(),
+                placementContext.getCluster().collections(),
+                true)
+            .values();
+    // This is critical to store the last sort weight for this node
+    weightedNodes.forEach(
+        node -> {
+          node.sortWithoutChanges();
+          node.addToSortedCollection(orderedNodes);
+        });
+
+    // While the node with the least cores still has room to take a replica 
from the node with the
+    // most cores, loop
+    Map<Replica, Node> newReplicaMovements = new HashMap<>();
+    ArrayList<WeightedNode> traversedHighNodes = new 
ArrayList<>(orderedNodes.size() - 1);
+    while (orderedNodes.size() > 1
+        && orderedNodes.first().calcWeight() < 
orderedNodes.last().calcWeight()) {
+      WeightedNode lowestWeight = orderedNodes.pollFirst();
+      if (lowestWeight == null) {
+        break;
+      }
+      if (lowestWeight.hasWeightChangedSinceSort()) {
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Re-sorting lowest weighted node: {}, sorting weight is 
out-of-date.",
+              lowestWeight.getNode().getName());
+        }
+        // Re-sort this node and go back to find the lowest weight
+        lowestWeight.addToSortedCollection(orderedNodes);
+        continue;
+      }
+      if (log.isDebugEnabled()) {
+        log.debug(
+            "Lowest weighted node: {}, weight: {}",
+            lowestWeight.getNode().getName(),
+            lowestWeight.calcWeight());
+      }
+
+      newReplicaMovements.clear();
+      // If a compatible node was found to move replicas, break and find the 
lowest weighted node
+      // again
+      while (newReplicaMovements.isEmpty()
+          && !orderedNodes.isEmpty()
+          && orderedNodes.last().calcWeight() > lowestWeight.calcWeight() + 1) 
{
+        WeightedNode highestWeight = orderedNodes.pollLast();
+        if (highestWeight == null) {
+          break;
+        }
+        if (highestWeight.hasWeightChangedSinceSort()) {
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Re-sorting highest weighted node: {}, sorting weight is 
out-of-date.",
+                highestWeight.getNode().getName());
+          }
+          // Re-sort this node and go back to find the highest weight
+          highestWeight.addToSortedCollection(orderedNodes);
+          continue;
+        }
+        if (log.isDebugEnabled()) {
+          log.debug(
+              "Highest weighted node: {}, weight: {}",
+              highestWeight.getNode().getName(),
+              highestWeight.calcWeight());
+        }
+
+        traversedHighNodes.add(highestWeight);
+        // select a replica from the node with the most cores to move to the 
node with the least
+        // cores
+        Set<Replica> availableReplicasToMove = 
highestWeight.getAllReplicasOnNode();
+        int combinedNodeWeights = highestWeight.calcWeight() + 
lowestWeight.calcWeight();
+        for (Replica r : availableReplicasToMove) {
+          // Only continue if the replica can be removed from the old node and 
moved to the new node
+          if (!highestWeight.canRemoveReplicas(Set.of(r)).isEmpty()
+              || !lowestWeight.canAddReplica(r)) {
+            continue;
+          }
+          lowestWeight.addReplica(r);
+          highestWeight.removeReplica(r);
+          int lowestWeightWithReplica = lowestWeight.calcWeight();
+          int highestWeightWithoutReplica = highestWeight.calcWeight();
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Replica: {}, toNode weight with replica: {}, fromNode weight 
without replica: {}",
+                r.getReplicaName(),
+                lowestWeightWithReplica,
+                highestWeightWithoutReplica);
+          }
+
+          // If the combined weight of both nodes is lower after the move, 
make the move.
+          // Otherwise, make the move if it doesn't cause the weight of the 
higher node to
+          // go below the weight of the lower node, because that is 
over-correction.
+          if (highestWeightWithoutReplica + lowestWeightWithReplica >= 
combinedNodeWeights
+              && highestWeightWithoutReplica < lowestWeightWithReplica) {
+            // Undo the move
+            lowestWeight.removeReplica(r);
+            highestWeight.addReplica(r);
+            continue;
+          }
+          if (log.isDebugEnabled()) {
+            log.debug(
+                "Replica Movement chosen. From: {}, To: {}, Replica: {}",
+                highestWeight.getNode().getName(),
+                lowestWeight.getNode().getName(),
+                r);
+          }
+          newReplicaMovements.put(r, lowestWeight.getNode());
+
+          // Do not go beyond here, do another loop and see if other nodes can 
move replicas.
+          // It might end up being the same nodes in the next loop that end up 
moving another
+          // replica, but that's ok.
+          break;
+        }
+      }
+      traversedHighNodes.forEach(n -> n.addToSortedCollection(orderedNodes));
+      traversedHighNodes.clear();
+      if (newReplicaMovements.size() > 0) {
+        replicaMovements.putAll(newReplicaMovements);
+        // There are no replicas to move to the lowestWeight, remove it from 
our loop
+        lowestWeight.addToSortedCollection(orderedNodes);
+      }
+    }
+
+    return placementContext
+        .getBalancePlanFactory()
+        .createBalancePlan(balanceRequest, replicaMovements);
+  }
+
+  protected Map<Node, WeightedNode> getWeightedNodes(
+      PlacementContext placementContext,
+      Set<Node> nodes,
+      Iterable<SolrCollection> relevantCollections,
+      boolean skipNodesWithErrors)
+      throws PlacementException {
+    Map<Node, WeightedNode> weightedNodes =
+        getBaseWeightedNodes(placementContext, nodes, relevantCollections, 
skipNodesWithErrors);
+
+    for (SolrCollection collection : 
placementContext.getCluster().collections()) {
+      for (Shard shard : collection.shards()) {
+        for (Replica replica : shard.replicas()) {
+          WeightedNode weightedNode = weightedNodes.get(replica.getNode());
+          if (weightedNode != null) {
+            weightedNode.initReplica(replica);
+          }
+        }
+      }
+    }
+
+    return weightedNodes;
+  }
+
+  protected abstract Map<Node, WeightedNode> getBaseWeightedNodes(
+      PlacementContext placementContext,
+      Set<Node> nodes,
+      Iterable<SolrCollection> relevantCollections,
+      boolean skipNodesWithErrors)
+      throws PlacementException;
+
+  @Override
+  public void verifyAllowedModification(
+      ModificationRequest modificationRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    if (modificationRequest instanceof DeleteShardsRequest) {
+      log.warn("DeleteShardsRequest not implemented yet, skipping: {}", 
modificationRequest);
+    } else if (modificationRequest instanceof DeleteCollectionRequest) {
+      verifyDeleteCollection((DeleteCollectionRequest) modificationRequest, 
placementContext);
+    } else if (modificationRequest instanceof DeleteReplicasRequest) {
+      verifyDeleteReplicas((DeleteReplicasRequest) modificationRequest, 
placementContext);
+    } else {
+      log.warn("unsupported request type, skipping: {}", modificationRequest);
+    }
+  }
+
+  protected void verifyDeleteCollection(
+      DeleteCollectionRequest deleteCollectionRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    // NO-OP
+  }
+
+  protected void verifyDeleteReplicas(
+      DeleteReplicasRequest deleteReplicasRequest, PlacementContext 
placementContext)
+      throws PlacementException {
+    Map<Node, List<Replica>> nodesRepresented =
+        deleteReplicasRequest.getReplicas().stream()
+            .collect(Collectors.groupingBy(Replica::getNode));
+
+    Map<Node, WeightedNode> weightedNodes =
+        getWeightedNodes(
+            placementContext,
+            nodesRepresented.keySet(),
+            placementContext.getCluster().collections(),
+            false);
+
+    PlacementModificationException placementModificationException =
+        new PlacementModificationException("delete replica(s) rejected");
+    for (Map.Entry<Node, List<Replica>> entry : nodesRepresented.entrySet()) {
+      WeightedNode node = weightedNodes.get(entry.getKey());
+      if (node == null) {
+        entry
+            .getValue()
+            .forEach(
+                replica ->
+                    placementModificationException.addRejectedModification(
+                        replica.toString(),
+                        "could not load information for node: " + 
entry.getKey().getName()));
+      } else {
+        node.canRemoveReplicas(entry.getValue())
+            .forEach(
+                (replica, reason) ->
+                    placementModificationException.addRejectedModification(
+                        replica.toString(), reason));
+      }
+    }
+    if (!placementModificationException.getRejectedModifications().isEmpty()) {
+      throw placementModificationException;
+    }
+  }
+
+  /**
+   * A class that determines the weight of a given node and the replicas that 
reside on it.
+   *
+   * <p>The OrderedNodePlacementPlugin uses the weights determined here to 
place and balance
+   * replicas across the cluster.
+   *

Review Comment:
   +1 for the javadocs.  Is it worth stating explicitly here (or on the 
OrderedNodePlacementPlugin) whether low or high weights are preferred?  (i.e. 
Does ONPP put replicas on low-weight nodes, or on high-weight nodes?)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@solr.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@solr.apache.org
For additional commands, e-mail: issues-h...@solr.apache.org


Reply via email to