You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by "HoustonPutman (via GitHub)" <gi...@apache.org> on 2023/05/19 16:27:52 UTC

[GitHub] [solr] HoustonPutman opened a new pull request, #1650: SOLR-16806: Create a BalanceReplicas API

HoustonPutman opened a new pull request, #1650:
URL: https://github.com/apache/solr/pull/1650

   https://issues.apache.org/jira/browse/SOLR-16806
   
   This is very much still a WIP
   
   Things to do:
   
   - [ ] Add Balance() to Assign
   - [ ] Add Balance() to PlacementPlugin
   - [ ] Add all necessary abstractions to make a usable (and extendable) Balance() API in PlacementPlugin
   - [ ] Add the BalanceReplicasCmd
   - [ ] Add the actual V2 API that will call BalanceReplicasCmd
   - [ ] Implement BalanceReplicas for existing PlacementPlugins
     - [ ] Random - will probably skip since random balancing doesn't make much sense
     - [ ] Simple
     - [ ] MinimizeCores
     - [ ] Affinity
   - [ ] Add documentation
   - [ ] Add tests


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] gerlowskija commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "gerlowskija (via GitHub)" <gi...@apache.org>.
gerlowskija commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1568431292

   Morning @HoustonPutman - is this mature enough to review at this point, or still very much undergoing heavy change?
   
   Just figured I'd check before diving in.  Exciting PR!


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] janhoy commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "janhoy (via GitHub)" <gi...@apache.org>.
janhoy commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1587155169

   Excited to see this balance API taking shape. Won't have a chance to review now, but following along from the sideline.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226903265


##########
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:
   So I disagree with this whole Interface/Impl thing for most of these classes. I have just followed the standard for the rest of the classes in this package.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] radu-gheorghe commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "radu-gheorghe (via GitHub)" <gi...@apache.org>.
radu-gheorghe commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1587759700

   Awesome stuff, @HoustonPutman! Unfortunately, I think it's too much for me to really understand everything your code does, so I'll add some higher-level comments here instead. I hope they're not too dumb, but if you are, ignore them, if they're useful, good 😄 And of course let me know if you have any additional comments/questions.
   
   * `OrderedNodePlacementPlugin` changes into its own PR - I don't think so, I think the changes are too tightly integrated and it will make things even harder to grok. It looks like that's the direction this PR is going towards anyway 🙂 
   * I saw a "parallel" flag, but I think replicas are moved one at a time, correct? If so, it will likely prove to be a bottleneck down the road. Some ideas to fix it (maybe in a separate PR):
   ** Allow rebalancing to happen in parallel for movements where the source AND the destination node are not on the list of source/destination nodes that are already participating in rebalancing. I would say this is pretty safe to do, because if one allows/triggers rebalancing, one would assume there's this extra load that can happen on any node
   ** Go for the Elasticsearch route and limit the number of concurrent rebalancing AND the number of concurrent incoming/outgoing rebalances per node. The problem is, IMO, with network throughput used by replication. AFAIK, in Solr you can limit it per core instead of the whole node, one has to keep that in mind to make sure rebalancing doesn't choke the network
   * Am I reading the code right that we compute all the shard movements that have to be done, then do them all, then check if we were successful? That seems a little bit risky because, if rebalancing takes a long time, changes can happen in the cluster (e.g. node added/failed, new collections added/removed...) so I guess one would have to cancel the rebalancing call and try again. I think the current approach has some advantages, but I have a feeling that if a call would just move one shard at a time (or N in parallel), then come back to see what's the next best thing, etc. it would be safer (and play nicer with the Operator loop, too). I'm thinking again of Elasticsearch (because all I have is a hammer 😅 ) and how it doesn't rebalance more stuff until all recoveries are done. Meaning that if a node went down, something/someone (kubernetes or human operator) could decide to stop the rebalance, replace the node, then resume. I hope I'm not way off here 🙈 
   * I saw a `maxBalanceSkew` property, but I'm not really sure what it does. Is this the number of "weight points" difference between nodes under which we say "don't bother?" And that would be different based on the placement plugin?
   * The balancing code that I saw for the Simple placement plugin seems more complicated than the `minimizeCores` one. In Affinity, we don't seem to really deal with the `withCollection` case, in the sense that we would probably want to move shards of both collections at the same time - we check if the destination node has that collection, correct? Anyway, this is not meant to be criticism, just to "sell" the crazy idea of having existing placement plugins **not** implement rebalancing and just have new ones, like:
   ** the Simple plugin as you've just implemented it (with a different name)
   ** the minimizeCores plugins as it is? Or maybe this is simple enough to simply remain as you extended it now?
   ** with the Affinity plugin, starting from scratch would avoid dealing (at least initially) with edge cases like `withCollection` or maybe even replica types.
   * Do I get it right that we soft-enforce constraints like AZ-awareness by adding big weights? If so, would it be better down the road to add them as constraints? Maybe in `canAddReplica`? This way we only make balancing moves that are "legal", e.g. not to have two copies of the same shard in the same AZ or on the same node, or if the node has too little space, etc. Maybe down the road we can add other constraints. One that I find really useful is the total number of shards in a collection in a node: if the cluster shrinks below a size, I prefer not to choke existing nodes with more replicas and create a domino effect.
   
   This last bit on "total shards per collection per node" isn't really about rebalancing, but about placing a new - e.g. recovered - replica. But now that we're in the realm of placement plugins, we might as well use weights for "placing" replicas in other contexts. And I think your PR already does that, no? If I simply create a new replica, the placement plugin will put it on the node with the lowest weight, correct?


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1588135823

   Thanks for the review Radu!
   
   > I saw a "parallel" flag, but I think replicas are moved one at a time, correct? If so, it will likely prove to be a bottleneck down the road.
   
   The replicas will be moved in parallel, but in the future we definitely should add a bottleneck option here. That's a great call-out. Separate PR IMO.
   
   > Am I reading the code right that we compute all the shard movements that have to be done, then do them all, then check if we were successful?
   
   Yes, that is correct. And this is also a good future edition I think. We can limit the balanceReplicas logic to `x` movements, then implement those movements then wait for the cluster to become healthy. We could do that logic in a loop until 0 movements are returned. Also a separate PR, but I love the idea.
   
   > I saw a maxBalanceSkew property, but I'm not really sure what it does. Is this the number of "weight points" difference between nodes under which we say "don't bother?" And that would be different based on the placement plugin?
   
   Currently it does absolutely nothing, but yes, the idea is that you move things around until the difference between the lowest weight and the highest weight is <= `maxBalanceSkew`. I didn't implement it yet, because I'm not sure how useful it would be honestly... I was also thinking originally that it would return an error if the maxBalanceSkew couldn't be achieved, but once again I'm not convinced of a use case here. I should remove it for now, since it is unused.
   
   > The balancing code that I saw for the Simple placement plugin seems more complicated than the minimizeCores one.
   
   Yeah, simple is more complex than minimizeCores. It was the Legacy implementation, but the name "Simple" was chosen over "Legacy".
   
   > In Affinity, we don't seem to really deal with the withCollection case, in the sense that we would probably want to move shards of both collections at the same time - we check if the destination node has that collection, correct?
   
   That is correct. It makes sure that a move works with the existing replicas, but it is not yet smart enough to move replicas together. I have ideas on how to improve this down the line, but I don't think its necessary for the first implementation.
   
   Basically for `canRemoveReplica(r)`, it would return the list of other replicas that would need to be removed along with that replica, and `canAddReplica(r)`, it would return the list of other replicas that would need to be added along with that replica. Then the orderedNode logic could make it work. Once again, this adds some complexity, and it can likely wait for another PR.
   
   > Do I get it right that we soft-enforce constraints like AZ-awareness by adding big weights? If so, would it be better down the road to add them as constraints?
   
   So yeah, they can absolutely be used as constraints, but right now (before this PR) they aren't, so this logic is keeping in-line with the logic that existed before.
   
   > And I think your PR already does that, no? If I simply create a new replica, the placement plugin will put it on the node with the lowest weight, correct?
   
   This is correct, and we can definitely add new constraints (like shardsPerNode/replicasPerNode) to the different PlacementPlugins. It will be especially easy to do given this new framework going forward!


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224394581


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);

Review Comment:
   So I think this is confusing. The asyncId is passed to the child commands (addReplica and cleanupReplicas), but the commands aren't async themselves. They would have to be added to the overseer queue for that to be the case (right?). (Edit: addReplica is definitely async, but we do wait on the adds to "finish" via a countdown latch, though it doesn't check the state if waitForFinalState=false)
   
   I think the asyncId is just used for tracking. So when it hits this point, the migration is actually successful.
   
   As for the waitForFinalState=false, I think "completed" is fine, since the user didn't ask for the final state to be healthy at the end of the migration. And it's much closer to "completed" than "submitted" IMO.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226879328


##########
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:
   Yeah we can call it `computeBalancing` or something.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226879953


##########
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:
   I deleted it, because it uses a low level class that no longer exists, and is very difficult to be replicated using the new logic/classes.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1582635343

   So this should be ready to start reviewing.
   
   It still needs testing and docs, but I don't want to do that until the design has a +1 from others.
   
   This PR now shifts the existing PlacementPlugins to be subclasses of OrderedNodePlacementPlugin, which gives default implementations of `computePlacements()` and `computeBalancing()`. These subclasses merely need to implement a `getWeightedNodes()` method, which returns a mapping from `Node` to `WeightedNode`. Each plugin makes its own implementation of `WeightedNode`, because the "weighting" is determined by what that plugin wants to prioritize when making selections.
   
   The `WeightedNode` abstract class has become a bit bloated, but basically it allows a Node to keep track of a state, and make an overall "weight" for that node with the given state. (It also has a method to project relevant weight with a given replica added, that's what used for `computePlacements()`).
   
   I'm sure I need to do a lot more explanation, but in general, the given plugins should work almost exactly as they do today, with the added benefit of having `computeBalancing()` come for "free". (though I'm sure the implementation of that shared method can be improved with time)
   
   This OrderedNodePlacementPlugin is just a class that the existing PlacementPlugins extend, so it is back-compat with custom plugins. We should mark the class expirimental for 9.x, so that we can make improvements to OrderedNodePlacementPlugin without having to worry about back-compat guarantees while we are improving these APIs. (e.g., the WeightedNode class may need additional methods if we want to make improvements to the sorting).
   
   I'm also happy to split the `OrderedNodePlacementPlugin` changes into its own PR, so that it can be reviewed independently of the BalanceReplica code. Let me know what you think!


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1583323386

   I'm pretty sure we can revert https://github.com/apache/solr/pull/1661 ([SOLR-16816](https://issues.apache.org/jira/browse/SOLR-16816)), if we merge this with the existing functionality. No reason to update the metrics if the WeightedNodes have the state stored outside of the metrics.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226875203


##########
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:
   Yeah good catch, this was originally just implemented for minimizeCores, but got generalized when things moved to "weights"



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] gerlowskija commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "gerlowskija (via GitHub)" <gi...@apache.org>.
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] sonatype-lift[bot] commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "sonatype-lift[bot] (via GitHub)" <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1223157963


##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,544 @@
+/*
+ * 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.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()) {
+        log.info("Collection: {}, shard: {}", solrCollection.getName(), shardName);
+
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          log.info("ReplicaType: {}", replicaType);
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          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);
+                  });
+
+          if (nodesForReplicaType.size() < replicaCount) {
+            throw new PlacementException(
+                "Not enough eligible nodes to place "
+                    + replicaCount
+                    + " replica(s) of type "
+                    + replicaType
+                    + " for shard "
+                    + shardName
+                    + " of collection "
+                    + solrCollection.getName());
+          }
+
+          int nodesChosen = 0;
+          while (nodesChosen < replicaCount) {
+            if (nodesForReplicaType.isEmpty()) {
+              throw new PlacementException(
+                  "There are not enough nodes to handle request to place replica");
+            }
+            WeightedNode node = nodesForReplicaType.poll();
+            while (node.hasWeightChangedSinceSort()) {
+              log.info("Out of date Node: {}", node.getNode());
+              node.addToSortedCollection(nodesForReplicaType);
+              node = nodesForReplicaType.poll();
+            }
+            log.info("Node: {}", node.getNode());
+
+            boolean needsToResort =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, node.getNode()));
+            nodesChosen += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), replicaType));
+            if (needsToResort) {
+              List<WeightedNode> nodeList = new ArrayList<>(nodesForReplicaType);
+              nodesForReplicaType.clear();
+              nodeList.forEach(n -> n.addToSortedCollection(nodesForReplicaType));
+            }
+          }
+        }
+      }
+
+      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()) {
+        // Re-sort this node and go back to find the lowest
+        lowestWeight.addToSortedCollection(orderedNodes);
+        continue;
+      }
+      log.info(
+          "Lowest 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()) {
+          // Re-sort this node and go back to find the lowest
+          highestWeight.addToSortedCollection(orderedNodes);
+          continue;
+        }
+        log.debug(
+            "Highest 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();
+          log.info(
+              "Replica: {}, lowestWith: {} ({}), highestWithout: {} ({})",
+              r.getReplicaName(),
+              lowestWeightWithReplica,
+              lowestWeight.canAddReplica(r),
+              highestWeightWithoutReplica,
+              highestWeight.canRemoveReplicas(Set.of(r)));
+
+          // 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;
+          }
+          log.info("Replica Movement Chosen!");
+          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;
+        }
+      }
+      orderedNodes.addAll(traversedHighNodes);
+      traversedHighNodes.clear();
+      if (newReplicaMovements.size() > 0) {
+        replicaMovements.putAll(newReplicaMovements);
+        // There are no replicas to move to the lowestWeight, remove it from our loop
+        orderedNodes.add(lowestWeight);
+      }
+    }
+
+    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()));
+      }
+      node.canRemoveReplicas(entry.getValue())

Review Comment:
   <picture><img alt="7% of developers fix this issue" src="https://lift.sonatype.com/api/commentimage/fixrate/7/display.svg"></picture>
   
   <b>*NULLPTR_DEREFERENCE:</b>*  `node` could be null (last assigned on line 343) and is dereferenced.
   
   ---
   
   <details><summary>ℹ️ Expand to see all <b>@sonatype-lift</b> commands</summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226896752


##########
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:
   This code really needs an overhaul anyways. It was copied and pasted from ReplaceNodeCmd. I think we can handle the cleanup/overhaul in a separate PR.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224399754


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    }
   }
 
-  static List<ZkNodeProps> getReplicasOfNode(String source, ClusterState state) {
-    List<ZkNodeProps> sourceReplicas = new ArrayList<>();
+  static List<Replica> getReplicasOfNode(String source, ClusterState state) {

Review Comment:
   Changed and moved to the utility class, since its used by ReplaceNodeCmd and DeleteNodeCmd



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] tflobbe commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "tflobbe (via GitHub)" <gi...@apache.org>.
tflobbe commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1223482384


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    }
   }
 
-  static List<ZkNodeProps> getReplicasOfNode(String source, ClusterState state) {
-    List<ZkNodeProps> sourceReplicas = new ArrayList<>();
+  static List<Replica> getReplicasOfNode(String source, ClusterState state) {

Review Comment:
   maybe rename `source` to `node` or `nodeName`?



##########
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()) {
+      ccc.getZkStateReader().removeCollectionStateWatcher(e.getKey(), e.getValue());
+    }
+    if (anyOneFailed.get()) {
+      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+      SolrCloseableLatch cleanupLatch =
+          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
+      for (ZkNodeProps createdReplica : createdReplicas) {
+        NamedList<Object> deleteResult = new NamedList<>();
+        try {
+          new DeleteReplicaCmd(ccc)
+              .deleteReplica(
+                  ccc.getZkStateReader().getClusterState(),
+                  createdReplica.plus("parallel", "true"),
+                  deleteResult,
+                  () -> {
+                    cleanupLatch.countDown();
+                    if (deleteResult.get("failure") != null) {
+                      synchronized (results) {
+                        results.add(
+                            "failure",
+                            "Could not cleanup, because of : " + deleteResult.get("failure"));
+                      }
+                    }
+                  });
+        } catch (KeeperException e) {
+          cleanupLatch.countDown();
+          log.warn("Error deleting replica ", e);
+        } catch (Exception e) {
+          log.warn("Error deleting replica ", e);
+          cleanupLatch.countDown();
+          throw e;
+        }
+      }
+      cleanupLatch.await(5, TimeUnit.MINUTES);
+      return false;
+    }
+
+    // we have reached this far, meaning all replicas should have been recreated.
+    // now cleanup the original replicas
+    return cleanupReplicas(
+        results, ccc.getZkStateReader().getClusterState(), movements.keySet(), ccc, asyncId);
+  }
+
+  static boolean cleanupReplicas(
+      NamedList<Object> results,
+      ClusterState clusterState,
+      Collection<Replica> sourceReplicas,
+      CollectionCommandContext ccc,
+      String async)
+      throws IOException, InterruptedException {
+    CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());
+    for (Replica sourceReplica : sourceReplicas) {
+      String coll = sourceReplica.getCollection();
+      String shard = sourceReplica.getShard();
+      String type = sourceReplica.getType().toString();
+      String node = sourceReplica.getNodeName();
+      log.info(
+          "Deleting replica type={} for collection={} shard={} on node={}",
+          type,
+          coll,
+          shard,
+          node);
+      NamedList<Object> deleteResult = new NamedList<>();
+      try {
+        ZkNodeProps cmdMessage = sourceReplica.toFullProps();
+        if (async != null) cmdMessage = cmdMessage.plus(ASYNC, async);
+        new DeleteReplicaCmd(ccc)
+            .deleteReplica(
+                clusterState,
+                cmdMessage.plus("parallel", "true"),
+                deleteResult,
+                () -> {
+                  cleanupLatch.countDown();
+                  if (deleteResult.get("failure") != null) {
+                    synchronized (results) {
+                      results.add(
+                          "failure",
+                          String.format(
+                              Locale.ROOT,
+                              "Failed to delete replica for collection=%s shard=%s" + " on node=%s",

Review Comment:
   No need to add strings



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java:
##########
@@ -251,1006 +231,463 @@ private AffinityPlacementPlugin(
 
       // We make things reproducible in tests by using test seed if any
       String seed = System.getProperty("tests.seed");
-      if (seed != null) {
-        replicaPlacementRandom.setSeed(seed.hashCode());
-      }
-    }
-
-    @Override
-    @SuppressForbidden(
-        reason =
-            "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-    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());
-      }
-
-      // Fetch attributes for a superset of all nodes requested amongst the placementRequests
-      AttributeFetcher attributeFetcher = placementContext.getAttributeFetcher();
-      attributeFetcher
-          .requestNodeSystemProperty(AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.NODE_TYPE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
-      attributeFetcher
-          .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
-          .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB);
-      Set<ReplicaMetric<?>> replicaMetrics = Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
-      for (SolrCollection collection : allCollections) {
-        attributeFetcher.requestCollectionMetrics(collection, replicaMetrics);
-      }
-      attributeFetcher.fetchFrom(allNodes);
-      final AttributeValues attrValues = attributeFetcher.fetchAttributes();
-      // Get the number of currently existing cores per node, so we can update as we place new cores
-      // to not end up always selecting the same node(s). This is used across placement requests
-      Map<Node, Integer> allCoresOnNodes = getCoreCountPerNode(allNodes, attrValues);
-
-      boolean doSpreadAcrossDomains = shouldSpreadAcrossDomains(allNodes, attrValues);
-
-      // Keep track with nodesWithReplicas across requests
-      Map<String, Map<String, Set<Node>>> allNodesWithReplicas = new HashMap<>();
-      for (PlacementRequest request : requests) {
-        Set<Node> nodes = request.getTargetNodes();
-        SolrCollection solrCollection = request.getCollection();
-
-        // filter out nodes that don't meet the `withCollection` constraint
-        nodes =
-            filterNodesWithCollection(placementContext.getCluster(), request, attrValues, nodes);
-        // filter out nodes that don't match the "node types" specified in the collection props
-        nodes = filterNodesByNodeType(placementContext.getCluster(), request, attrValues, nodes);
-
-        // All available zones of live nodes. Due to some nodes not being candidates for placement,
-        // and some existing replicas being one availability zones that might be offline (i.e. their
-        // nodes are not live), this set might contain zones on which it is impossible to place
-        // replicas. That's ok.
-        Set<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
-
-        // Build the replica placement decisions here
-        Set<ReplicaPlacement> replicaPlacements = new HashSet<>();
-
-        // Let's now iterate on all shards to create replicas for and start finding home sweet homes
-        // for the replicas
-        for (String shardName : request.getShardNames()) {
-          ReplicaMetrics leaderMetrics =
-              attrValues
-                  .getCollectionMetrics(solrCollection.getName())
-                  .flatMap(colMetrics -> colMetrics.getShardMetrics(shardName))
-                  .flatMap(ShardMetrics::getLeaderMetrics)
-                  .orElse(null);
-
-          // Split the set of nodes into 3 sets of nodes accepting each replica type (sets can
-          // overlap
-          // if nodes accept multiple replica types). These subsets sets are actually maps, because
-          // we
-          // capture the number of cores (of any replica type) present on each node.
-          //
-          // This also filters out nodes that will not satisfy the rules if the replica is placed
-          // there
-          EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
-              getAvailableNodesForReplicaTypes(nodes, attrValues, leaderMetrics);
-
-          // Inventory nodes (if any) that already have a replica of any type for the shard, because
-          // we can't be placing additional replicas on these. This data structure is updated after
-          // each replica to node assign and is used to make sure different replica types are not
-          // allocated to the same nodes (protecting same node assignments within a given replica
-          // type is done "by construction" in makePlacementDecisions()).
-          Set<Node> nodesWithReplicas =
-              allNodesWithReplicas
-                  .computeIfAbsent(solrCollection.getName(), col -> new HashMap<>())
-                  .computeIfAbsent(
-                      shardName,
-                      s -> {
-                        Set<Node> newNodeSet = new HashSet<>();
-                        Shard shard = solrCollection.getShard(s);
-                        if (shard != null) {
-                          // Prefill the set with the existing replicas
-                          for (Replica r : shard.replicas()) {
-                            newNodeSet.add(r.getNode());
-                          }
-                        }
-                        return newNodeSet;
-                      });
-
-          // Iterate on the replica types in the enum order. We place more strategic replicas first
-          // (NRT is more strategic than TLOG more strategic than PULL). This is in case we
-          // eventually decide that less strategic replica placement impossibility is not a problem
-          // that should lead to replica placement computation failure. Current code does fail if
-          // placement is impossible (constraint is at most one replica of a shard on any node).
-          for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-            int numReplicasToCreate = request.getCountReplicasToCreate(replicaType);
-            if (numReplicasToCreate > 0) {
-              makePlacementDecisions(
-                  solrCollection,
-                  shardName,
-                  availabilityZones,
-                  replicaType,
-                  numReplicasToCreate,
-                  attrValues,
-                  leaderMetrics,
-                  replicaTypeToNodes,
-                  nodesWithReplicas,
-                  allCoresOnNodes,
-                  placementContext.getPlacementPlanFactory(),
-                  replicaPlacements,
-                  doSpreadAcrossDomains);
-            }
-          }
-        }
-        placementPlans.add(
-            placementContext
-                .getPlacementPlanFactory()
-                .createPlacementPlan(request, replicaPlacements));
-      }
-
-      return placementPlans;
-    }
-
-    private boolean shouldSpreadAcrossDomains(Set<Node> allNodes, AttributeValues attrValues) {
-      boolean doSpreadAcrossDomains =
-          spreadAcrossDomains && spreadDomainPropPresent(allNodes, attrValues);
-      if (spreadAcrossDomains && !doSpreadAcrossDomains) {
-        log.warn(
-            "AffinityPlacementPlugin configured to spread across domains, but there are nodes in the cluster without the {} system property. Ignoring spreadAcrossDomains.",
-            AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
-      }
-      return doSpreadAcrossDomains;
-    }
-
-    private boolean spreadDomainPropPresent(Set<Node> allNodes, AttributeValues attrValues) {
-      // We can only use spread domains if all nodes have the system property
-      return allNodes.stream()
-          .noneMatch(
-              n ->
-                  attrValues
-                      .getSystemProperty(n, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                      .isEmpty());
     }
 
     @Override
-    public void verifyAllowedModification(
-        ModificationRequest modificationRequest, PlacementContext placementContext)
-        throws PlacementModificationException {
-      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);
-      }
-    }
-
-    private void verifyDeleteCollection(
+    protected void verifyDeleteCollection(
         DeleteCollectionRequest deleteCollectionRequest, PlacementContext placementContext)
         throws PlacementModificationException {
       Cluster cluster = placementContext.getCluster();
-      Set<String> colocatedCollections =
-          colocatedWith.getOrDefault(deleteCollectionRequest.getCollection().getName(), Set.of());
-      for (String primaryName : colocatedCollections) {
+      Set<String> collocatedCollections =
+          collocatedWith.getOrDefault(deleteCollectionRequest.getCollection().getName(), Set.of());
+      for (String primaryName : collocatedCollections) {
         try {
           if (cluster.getCollection(primaryName) != null) {
             // still exists
             throw new PlacementModificationException(
-                "colocated collection "
+                "collocated collection "
                     + primaryName
                     + " of "
                     + deleteCollectionRequest.getCollection().getName()
                     + " still present");
           }
         } catch (IOException e) {
           throw new PlacementModificationException(
-              "failed to retrieve colocated collection information", e);
+              "failed to retrieve collocated collection information", e);
         }
       }
     }
 
-    private void verifyDeleteReplicas(
-        DeleteReplicasRequest deleteReplicasRequest, PlacementContext placementContext)
-        throws PlacementModificationException {
-      Cluster cluster = placementContext.getCluster();
-      SolrCollection secondaryCollection = deleteReplicasRequest.getCollection();
-      Set<String> colocatedCollections = colocatedWith.get(secondaryCollection.getName());
-      if (colocatedCollections == null) {
-        return;
-      }
-      Map<Node, Map<String, AtomicInteger>> secondaryNodeShardReplicas = new HashMap<>();
-      secondaryCollection
-          .shards()
-          .forEach(
-              shard ->
-                  shard
-                      .replicas()
-                      .forEach(
-                          replica ->
-                              secondaryNodeShardReplicas
-                                  .computeIfAbsent(replica.getNode(), n -> new HashMap<>())
-                                  .computeIfAbsent(
-                                      replica.getShard().getShardName(), s -> new AtomicInteger())
-                                  .incrementAndGet()));
+    private static final class AffinityPlacementContext {
+      private final Set<String> allSpreadDomains = new HashSet<>();
+      private final Map<String, Map<String, ReplicaSpread>> spreadDomainUsage = new HashMap<>();
+      private final Set<String> allAvailabilityZones = new HashSet<>();
+      private final Map<String, Map<String, Map<Replica.ReplicaType, ReplicaSpread>>>
+          availabilityZoneUsage = new HashMap<>();
+      private boolean doSpreadAcrossDomains;
+    }
 
-      // find the colocated-with collections
-      Map<Node, Set<String>> colocatingNodes = new HashMap<>();
-      try {
-        for (String colocatedCollection : colocatedCollections) {
-          SolrCollection coll = cluster.getCollection(colocatedCollection);
-          coll.shards()
-              .forEach(
-                  shard ->
-                      shard
-                          .replicas()
-                          .forEach(
-                              replica ->
-                                  colocatingNodes
-                                      .computeIfAbsent(replica.getNode(), n -> new HashSet<>())
-                                      .add(coll.getName())));
-        }
-      } catch (IOException ioe) {
-        throw new PlacementModificationException(
-            "failed to retrieve colocated collection information", ioe);
-      }
-      PlacementModificationException exception = null;
-      for (Replica replica : deleteReplicasRequest.getReplicas()) {
-        if (!colocatingNodes.containsKey(replica.getNode())) {
-          continue;
-        }
-        // check that there will be at least one replica remaining
-        AtomicInteger secondaryCount =
-            secondaryNodeShardReplicas
-                .getOrDefault(replica.getNode(), Map.of())
-                .getOrDefault(replica.getShard().getShardName(), new AtomicInteger());
-        if (secondaryCount.get() > 1) {
-          // we can delete it - record the deletion
-          secondaryCount.decrementAndGet();
-          continue;
-        }
-        // fail - this replica cannot be removed
-        if (exception == null) {
-          exception = new PlacementModificationException("delete replica(s) rejected");
+    @Override
+    protected Map<Node, WeightedNode> getBaseWeightedNodes(
+        PlacementContext placementContext,
+        Set<Node> nodes,
+        Iterable<SolrCollection> relevantCollections,
+        boolean skipNodesWithErrors)
+        throws PlacementException {
+      // Fetch attributes for a superset of all nodes requested amongst the placementRequests
+      AttributeFetcher attributeFetcher = placementContext.getAttributeFetcher();
+      attributeFetcher
+          .requestNodeSystemProperty(AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.NODE_TYPE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
+      attributeFetcher
+          .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
+          .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB);
+      Set<ReplicaMetric<?>> replicaMetrics = Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
+      Set<String> requestedCollections = new HashSet<>();
+      for (SolrCollection collection : relevantCollections) {
+        if (requestedCollections.add(collection.getName())) {
+          attributeFetcher.requestCollectionMetrics(collection, replicaMetrics);
         }
-        exception.addRejectedModification(
-            replica.toString(),
-            "co-located with replicas of " + colocatingNodes.get(replica.getNode()));
-      }
-      if (exception != null) {
-        throw exception;
       }
-    }
+      attributeFetcher.fetchFrom(nodes);
+      final AttributeValues attrValues = attributeFetcher.fetchAttributes();
 
-    private Set<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
-      Set<String> azs = new HashSet<>();
+      AffinityPlacementContext affinityPlacementContext = new AffinityPlacementContext();
+      affinityPlacementContext.doSpreadAcrossDomains = spreadAcrossDomains;
 
-      for (Node n : nodes) {
-        azs.add(getNodeAZ(n, attrValues));
+      Map<Node, WeightedNode> affinityNodeMap = CollectionUtil.newHashMap(nodes.size());
+      for (Node node : nodes) {
+        AffinityNode affinityNode =
+            newNodeFromMetrics(node, attrValues, affinityPlacementContext, skipNodesWithErrors);
+        if (affinityNode != null) {
+          affinityNodeMap.put(node, affinityNode);
+        }
       }
 
-      return Collections.unmodifiableSet(azs);
+      return affinityNodeMap;
     }
 
-    /**
-     * Resolves the AZ of a node and takes care of nodes that have no defined AZ in system property
-     * {@link AffinityPlacementConfig#AVAILABILITY_ZONE_SYSPROP} to then return {@link
-     * AffinityPlacementConfig#UNDEFINED_AVAILABILITY_ZONE} as the AZ name.
-     */
-    private String getNodeAZ(Node n, final AttributeValues attrValues) {
-      Optional<String> nodeAz =
-          attrValues.getSystemProperty(n, AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP);
-      // All nodes with undefined AZ will be considered part of the same AZ. This also works for
-      // deployments that do not care about AZ's
-      return nodeAz.orElse(AffinityPlacementConfig.UNDEFINED_AVAILABILITY_ZONE);
-    }
-
-    /**
-     * This class captures an availability zone and the nodes that are legitimate targets for
-     * replica placement in that Availability Zone. Instances are used as values in a {@link
-     * java.util.TreeMap} in which the total number of already existing replicas in the AZ is the
-     * key. This allows easily picking the set of nodes from which to select a node for placement in
-     * order to balance the number of replicas per AZ. Picking one of the nodes from the set is done
-     * using different criteria unrelated to the Availability Zone (picking the node is based on the
-     * {@link CoresAndDiskComparator} ordering).
-     */
-    private static class AzWithNodes {
-      final String azName;
-      private final boolean useSpreadDomains;
-      private boolean listIsSorted = false;
-      private final Comparator<Node> nodeComparator;
-      private final Random random;
-      private final List<Node> availableNodesForPlacement;
-      private final AttributeValues attributeValues;
-      private final ReplicaMetrics leaderMetrics;
-      private TreeSet<SpreadDomainWithNodes> sortedSpreadDomains;
-      private final Map<String, Integer> currentSpreadDomainUsageUsage;
-      private int numNodesForPlacement;
-
-      AzWithNodes(
-          String azName,
-          List<Node> availableNodesForPlacement,
-          boolean useSpreadDomains,
-          Comparator<Node> nodeComparator,
-          Random random,
-          AttributeValues attributeValues,
-          ReplicaMetrics leaderMetrics,
-          Map<String, Integer> currentSpreadDomainUsageUsage) {
-        this.azName = azName;
-        this.availableNodesForPlacement = availableNodesForPlacement;
-        this.useSpreadDomains = useSpreadDomains;
-        this.nodeComparator = nodeComparator;
-        this.random = random;
-        this.attributeValues = attributeValues;
-        this.leaderMetrics = leaderMetrics;
-        this.currentSpreadDomainUsageUsage = currentSpreadDomainUsageUsage;
-        this.numNodesForPlacement = availableNodesForPlacement.size();
-      }
-
-      private boolean hasBeenSorted() {
-        return (useSpreadDomains && sortedSpreadDomains != null)
-            || (!useSpreadDomains && listIsSorted);
-      }
-
-      void ensureSorted() {
-        if (!hasBeenSorted()) {
-          sort();
-        }
+    AffinityNode newNodeFromMetrics(
+        Node node,
+        AttributeValues attrValues,
+        AffinityPlacementContext affinityPlacementContext,
+        boolean skipNodesWithErrors)
+        throws PlacementException {
+      Set<Replica.ReplicaType> supportedReplicaTypes =
+          attrValues.getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP).stream()
+              .flatMap(s -> Arrays.stream(s.split(",")))
+              .map(String::trim)
+              .map(s -> s.toUpperCase(Locale.ROOT))
+              .map(
+                  s -> {
+                    try {
+                      return Replica.ReplicaType.valueOf(s);
+                    } catch (IllegalArgumentException e) {
+                      log.warn(
+                          "Node {} has an invalid value for the {} systemProperty: {}",
+                          node.getName(),
+                          AffinityPlacementConfig.REPLICA_TYPE_SYSPROP,
+                          s);
+                      return null;
+                    }
+                  })
+              .collect(Collectors.toSet());
+      if (supportedReplicaTypes.isEmpty()) {
+        // If property not defined or is only whitespace on a node, assuming node can take any
+        // replica type
+        supportedReplicaTypes = Set.of(Replica.ReplicaType.values());
       }
 
-      private void sort() {
-        assert !listIsSorted && sortedSpreadDomains == null
-            : "We shouldn't be sorting this list again";
-
-        // Make sure we do not tend to use always the same nodes (within an AZ) if all
-        // conditions are identical (well, this likely is not the case since after having added
-        // a replica to a node its number of cores increases for the next placement decision,
-        // but let's be defensive here, given that multiple concurrent placement decisions might
-        // see the same initial cluster state, and we want placement to be reasonable even in
-        // that case without creating an unnecessary imbalance). For example, if all nodes have
-        // 0 cores and same amount of free disk space, ideally we want to pick a random node for
-        // placement, not always the same one due to some internal ordering.
-        Collections.shuffle(availableNodesForPlacement, random);
-
-        if (useSpreadDomains) {
-          // When we use spread domains, we don't just sort the list of nodes, instead we generate a
-          // TreeSet of SpreadDomainWithNodes,
-          // sorted by the number of times the domain has been used. Each
-          // SpreadDomainWithNodes internally contains the list of nodes that belong to that
-          // particular domain,
-          // and it's sorted internally by the comparator passed to this
-          // class (which is the same that's used when not using spread domains).
-          // Whenever a node from a particular SpreadDomainWithNodes is selected as the best
-          // candidate, the call to "removeBestNode" will:
-          // 1. Remove the SpreadDomainWithNodes instance from the TreeSet
-          // 2. Remove the best node from the list within the SpreadDomainWithNodes
-          // 3. Increment the count of times the domain has been used
-          // 4. Re-add the SpreadDomainWithNodes instance to the TreeSet if there are still nodes
-          // available
-          HashMap<String, List<Node>> spreadDomainToListOfNodesMap = new HashMap<>();
-          for (Node node : availableNodesForPlacement) {
-            spreadDomainToListOfNodesMap
-                .computeIfAbsent(
-                    attributeValues
-                        .getSystemProperty(node, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                        .get(),
-                    k -> new ArrayList<>())
-                .add(node);
-          }
-          sortedSpreadDomains =
-              new TreeSet<>(new SpreadDomainComparator(currentSpreadDomainUsageUsage));
-
-          int i = 0;
-          for (Map.Entry<String, List<Node>> entry : spreadDomainToListOfNodesMap.entrySet()) {
-            // Sort the nodes within the spread domain by the provided comparator
-            entry.getValue().sort(nodeComparator);
-            sortedSpreadDomains.add(
-                new SpreadDomainWithNodes(entry.getKey(), entry.getValue(), i++, nodeComparator));
+      Set<String> nodeType;
+      Optional<String> nodePropOpt =
+          attrValues.getSystemProperty(node, AffinityPlacementConfig.NODE_TYPE_SYSPROP);
+      if (nodePropOpt.isEmpty()) {
+        nodeType = Collections.emptySet();
+      } else {
+        nodeType = new HashSet<>(StrUtils.splitSmart(nodePropOpt.get(), ','));
+      }
+
+      Optional<Double> nodeFreeDiskGB =
+          attrValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
+      Optional<Integer> nodeNumCores =
+          attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES);
+      String az =
+          attrValues
+              .getSystemProperty(node, AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
+              .orElse(AffinityPlacementConfig.UNDEFINED_AVAILABILITY_ZONE);
+      affinityPlacementContext.allAvailabilityZones.add(az);
+      String spreadDomain;
+      if (affinityPlacementContext.doSpreadAcrossDomains) {
+        spreadDomain =
+            attrValues
+                .getSystemProperty(node, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
+                .orElse(null);
+        if (spreadDomain == null) {
+          if (log.isWarnEnabled()) {
+            log.warn(
+                "AffinityPlacementPlugin configured to spread across domains, but node {} does not have the {} system property. Ignoring spreadAcrossDomains.",
+                node.getName(),
+                AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
           }
+          affinityPlacementContext.doSpreadAcrossDomains = false;
+          affinityPlacementContext.allSpreadDomains.clear();
         } else {
-          availableNodesForPlacement.sort(nodeComparator);
-          listIsSorted = true;
+          affinityPlacementContext.allSpreadDomains.add(spreadDomain);
         }
+      } else {
+        spreadDomain = null;
       }
-
-      Node getBestNode() {
-        assert hasBeenSorted();
-        if (useSpreadDomains) {
-          return sortedSpreadDomains.first().sortedNodesForPlacement.get(0);
-        } else {
-          return availableNodesForPlacement.get(0);
+      if (nodeFreeDiskGB.isEmpty() && skipNodesWithErrors) {
+        if (log.isWarnEnabled()) {
+          log.warn(
+              "Unknown free disk on node {}, excluding it from placement decisions.",
+              node.getName());
         }
-      }
-
-      public Node removeBestNode() {
-        assert hasBeenSorted();
-        this.numNodesForPlacement--;
-        Node n;
-        if (useSpreadDomains) {
-          // Since this SpreadDomainWithNodes needs to be re-sorted in the sortedSpreadDomains, we
-          // remove it and then re-add it, once the best node has been removed.
-          SpreadDomainWithNodes group = sortedSpreadDomains.pollFirst();
-          n = group.sortedNodesForPlacement.remove(0);
-          this.currentSpreadDomainUsageUsage.merge(group.spreadDomainName, 1, Integer::sum);
-          if (!group.sortedNodesForPlacement.isEmpty()) {
-            sortedSpreadDomains.add(group);
-          }
-        } else {
-          n = availableNodesForPlacement.remove(0);
+        return null;
+      } else if (nodeNumCores.isEmpty() && skipNodesWithErrors) {
+        if (log.isWarnEnabled()) {
+          log.warn(
+              "Unknown number of cores on node {}, excluding it from placement decisions.",
+              node.getName());
         }
-        Optional.ofNullable(leaderMetrics)
-            .flatMap(lrm -> lrm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
-            .ifPresent(
-                indexSize ->
-                    attributeValues.decreaseNodeMetric(
-                        n, BuiltInMetrics.NODE_FREE_DISK_GB, indexSize));
-        attributeValues.increaseNodeMetric(n, BuiltInMetrics.NODE_NUM_CORES, 1);
-        return n;
-      }
-
-      public int numNodes() {
-        return this.numNodesForPlacement;
+        return null;
+      } else {
+        return new AffinityNode(
+            node,
+            attrValues,
+            affinityPlacementContext,
+            supportedReplicaTypes,
+            nodeType,
+            nodeNumCores.orElse(0),
+            nodeFreeDiskGB.orElse(0D),
+            az,
+            spreadDomain);
       }
     }
 
-    /**
-     * This class represents group of nodes with the same {@link
-     * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label.
-     */
-    static class SpreadDomainWithNodes implements Comparable<SpreadDomainWithNodes> {
+    private class AffinityNode extends WeightedNode {
 
-      /**
-       * This is the label that all nodes in this group have in {@link
-       * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label.
-       */
-      final String spreadDomainName;
+      private final AttributeValues attrValues;
 
-      /**
-       * The list of all nodes that contain the same {@link
-       * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label. They must be sorted before creating
-       * this class.
-       */
-      private final List<Node> sortedNodesForPlacement;
+      private final AffinityPlacementContext affinityPlacementContext;
 
-      /**
-       * This is used for tie breaking the sort of {@link SpreadDomainWithNodes}, when the
-       * nodeComparator between the top nodes of each group return 0.
-       */
-      private final int tieBreaker;
+      private final Set<Replica.ReplicaType> supportedReplicaTypes;
+      private final Set<String> nodeType;
 
-      /**
-       * This is the comparator that is used to compare the top nodes in the {@link
-       * #sortedNodesForPlacement} lists. Must be the same that was used to sort {@link
-       * #sortedNodesForPlacement}.
-       */
-      private final Comparator<Node> nodeComparator;
+      private int coresOnNode;
+      private double nodeFreeDiskGB;
 
-      public SpreadDomainWithNodes(
-          String spreadDomainName,
-          List<Node> sortedNodesForPlacement,
-          int tieBreaker,
-          Comparator<Node> nodeComparator) {
-        this.spreadDomainName = spreadDomainName;
-        this.sortedNodesForPlacement = sortedNodesForPlacement;
-        this.tieBreaker = tieBreaker;
-        this.nodeComparator = nodeComparator;
+      private final String availabilityZone;
+      private final String spreadDomain;
+
+      AffinityNode(
+          Node node,
+          AttributeValues attrValues,
+          AffinityPlacementContext affinityPlacementContext,
+          Set<Replica.ReplicaType> supportedReplicaTypes,
+          Set<String> nodeType,
+          int coresOnNode,
+          double nodeFreeDiskGB,
+          String az,
+          String spreadDomain) {
+        super(node);
+        this.attrValues = attrValues;
+        this.affinityPlacementContext = affinityPlacementContext;
+        this.supportedReplicaTypes = supportedReplicaTypes;
+        this.nodeType = nodeType;
+        this.coresOnNode = coresOnNode;
+        this.nodeFreeDiskGB = nodeFreeDiskGB;
+        this.availabilityZone = az;
+        this.spreadDomain = spreadDomain;
       }
 
       @Override
-      public int compareTo(SpreadDomainWithNodes o) {
-        if (o == this) {
-          return 0;
-        }
-        int result =
-            nodeComparator.compare(
-                this.sortedNodesForPlacement.get(0), o.sortedNodesForPlacement.get(0));
-        if (result == 0) {
-          return Integer.compare(this.tieBreaker, o.tieBreaker);
-        }
-        return result;
+      public int calcWeight() {
+        return coresOnNode
+            + 100 * (prioritizedFreeDiskGB > 0 && nodeFreeDiskGB < prioritizedFreeDiskGB ? 1 : 0)
+            + 10000 * getSpreadDomainWeight()
+            + 1000000 * getAZWeight();
       }
-    }
-
-    /**
-     * Builds the number of existing cores on each node returned in the attrValues. Nodes for which
-     * the number of cores is not available for whatever reason are excluded from acceptable
-     * candidate nodes as it would not be possible to make any meaningful placement decisions.
-     *
-     * @param nodes all nodes on which this plugin should compute placement
-     * @param attrValues attributes fetched for the nodes. This method uses system property {@link
-     *     AffinityPlacementConfig#REPLICA_TYPE_SYSPROP} as well as the number of cores on each
-     *     node.
-     */
-    private Map<Node, Integer> getCoreCountPerNode(
-        Set<Node> nodes, final AttributeValues attrValues) {
-      Map<Node, Integer> coresOnNodes = new HashMap<>();
 
-      for (Node node : nodes) {
-        attrValues
-            .getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES)
-            .ifPresent(count -> coresOnNodes.put(node, count));
+      @Override
+      public int calcRelevantWeightWithReplica(Replica replica) {
+        return coresOnNode
+            + 100
+                * (prioritizedFreeDiskGB > 0
+                        && nodeFreeDiskGB - getProjectedSizeOfReplica(replica)
+                            < prioritizedFreeDiskGB
+                    ? 1
+                    : 0)
+            + 10000 * projectReplicaSpreadWeight(replica)
+            + 1000000 * projectAZWeight(replica);
       }
 
-      return coresOnNodes;
-    }
-
-    /**
-     * Given the set of all nodes on which to do placement and fetched attributes, builds the sets
-     * representing candidate nodes for placement of replicas of each replica type. These sets are
-     * packaged and returned in an EnumMap keyed by replica type. Nodes for which the number of
-     * cores is not available for whatever reason are excluded from acceptable candidate nodes as it
-     * would not be possible to make any meaningful placement decisions.
-     *
-     * @param nodes all nodes on which this plugin should compute placement
-     * @param attrValues attributes fetched for the nodes. This method uses system property {@link
-     *     AffinityPlacementConfig#REPLICA_TYPE_SYSPROP} as well as the number of cores on each
-     *     node.
-     */
-    private EnumMap<Replica.ReplicaType, Set<Node>> getAvailableNodesForReplicaTypes(
-        Set<Node> nodes, final AttributeValues attrValues, final ReplicaMetrics leaderMetrics) {
-      EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
-          new EnumMap<>(Replica.ReplicaType.class);
-
-      for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-        replicaTypeToNodes.put(replicaType, new HashSet<>());
+      @Override
+      public boolean canAddReplica(Replica replica) {
+        String collection = replica.getShard().getCollection().getName();
+        return
+        // By default, do not allow two replicas of the same shard on a node
+        super.canAddReplica(replica)
+            && supportedReplicaTypes.contains(replica.getType())
+            && Optional.ofNullable(nodeTypes.get(collection))
+                .map(s -> s.stream().anyMatch(nodeType::contains))
+                .orElse(true)
+            && Optional.ofNullable(withCollections.get(collection))
+                .map(this::hasCollectionOnNode)
+                .orElse(true)
+            && (minimalFreeDiskGB <= 0
+                || nodeFreeDiskGB - getProjectedSizeOfReplica(replica) > minimalFreeDiskGB);
       }
 
-      for (Node node : nodes) {
-        // Exclude nodes with unknown or too small disk free space
-        Optional<Double> nodeFreeDiskGB =
-            attrValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
-        if (nodeFreeDiskGB.isEmpty()) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Unknown free disk on node {}, excluding it from placement decisions.",
-                node.getName());
-          }
-          // We rely later on the fact that the free disk optional is present (see
-          // CoresAndDiskComparator), be careful it you change anything here.
-          continue;
-        }
-        double replicaIndexSize =
-            Optional.ofNullable(leaderMetrics)
-                .flatMap(lm -> lm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
-                .orElse(0D);
-        double projectedFreeDiskIfPlaced =
-            BuiltInMetrics.NODE_FREE_DISK_GB.decrease(nodeFreeDiskGB.get(), replicaIndexSize);
-        if (projectedFreeDiskIfPlaced < minimalFreeDiskGB) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Node {} free disk ({}GB) minus the projected replica size ({}GB) is lower than configured"
-                    + " minimum {}GB, excluding it from placement decisions.",
-                node.getName(),
-                nodeFreeDiskGB.get(),
-                replicaIndexSize,
-                minimalFreeDiskGB);
-          }
-          continue;
-        }
-
-        if (attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES).isEmpty()) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Unknown number of cores on node {}, excluding it from placement decisions.",
-                node.getName());
+      @Override
+      public Map<Replica, String> canRemoveReplicas(Collection<Replica> replicas) {
+        Map<Replica, String> replicaRemovalExceptions = new HashMap<>();
+        Map<String, Map<String, Set<Replica>>> removals = new HashMap<>();
+        for (Replica replica : replicas) {
+          SolrCollection collection = replica.getShard().getCollection();
+          Set<String> collocatedCollections = new HashSet<>();
+          Optional.ofNullable(collocatedWith.get(collection.getName()))
+              .ifPresent(collocatedCollections::addAll);
+          collocatedCollections.retainAll(getCollectionsOnNode());
+          if (collocatedCollections.isEmpty()) {
+            continue;
           }
-          // We rely later on the fact that the number of cores optional is present (see
-          // CoresAndDiskComparator), be careful it you change anything here.
-          continue;
-        }
 
-        String supportedReplicaTypes =
-            attrValues
-                    .getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-                    .isPresent()
-                ? attrValues
-                    .getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-                    .get()
-                : null;
-        // If property not defined or is only whitespace on a node, assuming node can take any
-        // replica type
-        if (supportedReplicaTypes == null || supportedReplicaTypes.isBlank()) {
-          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-            replicaTypeToNodes.get(rt).add(node);
-          }
-        } else {
-          Set<String> acceptedTypes =
-              Arrays.stream(supportedReplicaTypes.split(","))
-                  .map(String::trim)
-                  .map(s -> s.toLowerCase(Locale.ROOT))
-                  .collect(Collectors.toSet());
-          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-            if (acceptedTypes.contains(rt.name().toLowerCase(Locale.ROOT))) {
-              replicaTypeToNodes.get(rt).add(node);
-            }
+          // There are collocatedCollections for this shard, so make sure there is a replica of this
+          // shard left on the node after it is removed
+          Set<Replica> replicasRemovedForShard =
+              removals
+                  .computeIfAbsent(
+                      replica.getShard().getCollection().getName(), k -> new HashMap<>())
+                  .computeIfAbsent(replica.getShard().getShardName(), k -> new HashSet<>());
+          replicasRemovedForShard.add(replica);
+
+          if (replicasRemovedForShard.size()
+              >= getReplicasForShardOnNode(replica.getShard()).size()) {
+            replicaRemovalExceptions.put(
+                replica, "co-located with replicas of " + collocatedCollections);
           }
         }
+        return replicaRemovalExceptions;
       }
-      return replicaTypeToNodes;
-    }
 
-    /**
-     * Picks nodes from {@code targetNodes} for placing {@code numReplicas} replicas.
-     *
-     * <p>The criteria used in this method are, in this order:
-     *
-     * <ol>
-     *   <li>No more than one replica of a given shard on a given node (strictly enforced)
-     *   <li>Balance as much as possible replicas of a given {@link
-     *       org.apache.solr.cluster.Replica.ReplicaType} over available AZ's. This balancing takes
-     *       into account existing replicas <b>of the corresponding replica type</b>, if any.
-     *   <li>Place replicas if possible on nodes having more than a certain amount of free disk
-     *       space (note that nodes with a too small amount of free disk space were eliminated as
-     *       placement targets earlier, in {@link #getAvailableNodesForReplicaTypes(Set,
-     *       AttributeValues, ReplicaMetrics)}). There's a threshold here rather than sorting on the
-     *       amount of free disk space, because sorting on that value would in practice lead to
-     *       never considering the number of cores on a node.
-     *   <li>Place replicas on nodes having a smaller number of cores (the number of cores
-     *       considered for this decision includes previous placement decisions made during the
-     *       processing of the placement request)
-     * </ol>
-     */
-    @SuppressForbidden(
-        reason =
-            "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-    private void makePlacementDecisions(
-        SolrCollection solrCollection,
-        String shardName,
-        Set<String> availabilityZones,
-        Replica.ReplicaType replicaType,
-        int numReplicas,
-        final AttributeValues attrValues,
-        final ReplicaMetrics leaderMetrics,
-        EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes,
-        Set<Node> nodesWithReplicas,
-        Map<Node, Integer> coresOnNodes,
-        PlacementPlanFactory placementPlanFactory,
-        Set<ReplicaPlacement> replicaPlacements,
-        boolean doSpreadAcrossDomains)
-        throws PlacementException {
-      // Count existing replicas per AZ. We count only instances of the type of replica for which we
-      // need to do placement. If we ever want to balance replicas of any type across AZ's (and not
-      // each replica type balanced independently), we'd have to move this data structure to the
-      // caller of this method so it can be reused across different replica type placements for a
-      // given shard. Note then that this change would be risky. For example all NRT's and PULL
-      // replicas for a shard my be correctly balanced over three AZ's, but then all NRT can end up
-      // in the same AZ...
-      Map<String, Integer> azToNumReplicas = new HashMap<>();
-      for (String az : availabilityZones) {
-        azToNumReplicas.put(az, 0);
+      @Override
+      protected boolean addProjectedReplicaWeights(Replica replica) {
+        nodeFreeDiskGB -= getProjectedSizeOfReplica(replica);
+        coresOnNode += 1;
+        return addReplicaToAzAndSpread(replica);
       }
 
-      // Build the set of candidate nodes for the placement, i.e. nodes that can accept the replica
-      // type
-      Set<Node> candidateNodes = new HashSet<>(replicaTypeToNodes.get(replicaType));
-      // Remove nodes that already have a replica for the shard (no two replicas of same shard can
-      // be put on same node)
-      candidateNodes.removeAll(nodesWithReplicas);
+      @Override
+      protected void initReplicaWeights(Replica replica) {
+        addReplicaToAzAndSpread(replica);
+      }
 
-      // This Map will include the affinity labels for the nodes that are currently hosting replicas
-      // of this shard. It will be modified with new placement decisions.
-      Map<String, Integer> spreadDomainsInUse = new HashMap<>();
-      Shard shard = solrCollection.getShard(shardName);
-      if (shard != null) {
-        // shard is non null if we're adding replicas to an already existing collection.
-        // If we're creating the collection, the shards do not exist yet.
-        for (Replica replica : shard.replicas()) {
-          // The node's AZ is counted as having a replica if it has a replica of the same type as
-          // the one we need to place here.
-          if (replica.getType() == replicaType) {
-            final String az = getNodeAZ(replica.getNode(), attrValues);
-            if (azToNumReplicas.containsKey(az)) {
-              // We do not count replicas on AZ's for which we don't have any node to place on
-              // because it would not help the placement decision. If we did want to do that, note
-              // the dereferencing below can't be assumed as the entry will not exist in the map.
-              azToNumReplicas.put(az, azToNumReplicas.get(az) + 1);
-            }
-            if (doSpreadAcrossDomains) {
-              attrValues
-                  .getSystemProperty(
-                      replica.getNode(), AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                  .ifPresent(nodeDomain -> spreadDomainsInUse.merge(nodeDomain, 1, Integer::sum));
-            }
-          }
+      private boolean addReplicaToAzAndSpread(Replica replica) {
+        boolean needsResort = false;
+        needsResort |=

Review Comment:
   you can remove the default value in the first line



##########
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();

Review Comment:
   Should this happen at the end of the method instead of the beginning? Otherwise there is a race condition with the `anyOneFailed` I think?



##########
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)) {

Review Comment:
   Do we need to wait for this if `anyOneFailed` is `true`?



##########
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()) {
+      ccc.getZkStateReader().removeCollectionStateWatcher(e.getKey(), e.getValue());
+    }
+    if (anyOneFailed.get()) {
+      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+      SolrCloseableLatch cleanupLatch =
+          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
+      for (ZkNodeProps createdReplica : createdReplicas) {
+        NamedList<Object> deleteResult = new NamedList<>();
+        try {
+          new DeleteReplicaCmd(ccc)
+              .deleteReplica(
+                  ccc.getZkStateReader().getClusterState(),
+                  createdReplica.plus("parallel", "true"),
+                  deleteResult,
+                  () -> {
+                    cleanupLatch.countDown();
+                    if (deleteResult.get("failure") != null) {
+                      synchronized (results) {
+                        results.add(
+                            "failure",
+                            "Could not cleanup, because of : " + deleteResult.get("failure"));
+                      }
+                    }
+                  });
+        } catch (KeeperException e) {
+          cleanupLatch.countDown();
+          log.warn("Error deleting replica ", e);
+        } catch (Exception e) {
+          log.warn("Error deleting replica ", e);
+          cleanupLatch.countDown();
+          throw e;
+        }
+      }
+      cleanupLatch.await(5, TimeUnit.MINUTES);
+      return false;
+    }
+
+    // we have reached this far, meaning all replicas should have been recreated.
+    // now cleanup the original replicas
+    return cleanupReplicas(
+        results, ccc.getZkStateReader().getClusterState(), movements.keySet(), ccc, asyncId);
+  }
+
+  static boolean cleanupReplicas(
+      NamedList<Object> results,
+      ClusterState clusterState,
+      Collection<Replica> sourceReplicas,
+      CollectionCommandContext ccc,
+      String async)
+      throws IOException, InterruptedException {
+    CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());

Review Comment:
   We don't use `SolrCloseableLatch` here?



##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java:
##########
@@ -251,1006 +231,463 @@ private AffinityPlacementPlugin(
 
       // We make things reproducible in tests by using test seed if any
       String seed = System.getProperty("tests.seed");

Review Comment:
   Leftover?



##########
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",

Review Comment:
   No need to have two strings here



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);

Review Comment:
   I know it's not new, but this log line is misleading in the case of async (and maybe in waitForFinalState=false) too. Should we at least include also the values for async? or maybe have something like:
   ```
   "REPLACENODE action " + (async == null ? "completed": "submitted") + " successfully from : "
   ```



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1582636541

   @radu-gheorghe if you are interested!


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1557713764

   The logic for moving replicas is now shared between ReplaceNodeCmd and BalanceReplicasCmd.
   
   This introduced more stuff to review, but the logic is the exact same, so the best path forward is to have both classes share it.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224420594


##########
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)) {

Review Comment:
   We really only need to ensure that there is another active replica to take leadership if we are deleting a leader in `cleanupReplicas()`, so we should probably just create another latch and loop for it after the `countDownLatch` is complete. (if waitForFinalState=false)



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226892527


##########
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:
   I moved this up to OrderedNodePlacementPlugin, since it really should only be used there, and I added javadocs.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1589689966

   I've added tests and docs for the new API. Will probably merge in a day or two.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] radu-gheorghe commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "radu-gheorghe (via GitHub)" <gi...@apache.org>.
radu-gheorghe commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1582772911

   > @radu-gheorghe if you are interested!
   
   Very interested, thanks for tagging me! I'll check it out and get back to you.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226874002


##########
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:
   This formula kind of existed beforehand.
   
   Basically there didn't use to be a "node weight", but this one class in particular had a "node weight for replica". It was basically:
   `(the number of replicas in the collection on the node) * 5 + (total replicas on node)`.
   
   As a part of this PR I made this a bit better with:
   `(the number of replicas in the collection on the node) * 5 + (the number of replicas in the shard on the node) * 100 + (total replicas on node)`
   
   When trying to convert this to a full node weight, a node with 4 replicas of the same collection will be weighted the same as a node with 2 replicas of one collection, and 2 replicas of another collection (because `2*5 + 2*5 = 4*5`). This isn't really in the spirit of the logic, we want to end up with less replicas of the same collections on nodes. So it makes more sense to square the number of replicas in the same (collection|shard). That way`12^2*5 + 2^2*5 < 4^2* 5`. (And then I subtracted 1, because there's no reason to penalize the first replica of each shard/collection.)



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226897201


##########
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:
   Again, we can address this in the separate ReplicaMigration cleanup JIRA/PR.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman merged pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman merged PR #1650:
URL: https://github.com/apache/solr/pull/1650


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1584658161

   I also want to point out that the balanceReplicas logic is _decent_, it's not perfect.
   
   In the affinityPlugin, adding/removing replicas from a Node will change the weights of other nodes. This is why we needed the `WeightedNode.addReplica()` to return a boolean if other nodes might have lower weights after the addition. The `computePlacement()` logic is perfectly fine for other nodes to have their weights increased silently, since we check if their weight is out-of-date when picking the lowest weight. However nodes that silently have their weights decreased will not be at the front of the priorityQueue, to have their weights re-checked and computed. So the "entire" list needs to be re-sorted.
   
   However, this is ok in `computePlacements()` because it is only looking for nodes with the lowest weights. For `computeBalancing()` it's using a TreeSet to find nodes with the lowest and highest weights. Therefore, any other nodes having their weights silently change (via a different node using `addReplica()` or `removeReplica()`) need to be resorted if their weight goes up or down, for optimal placement. Currently we don't do this, because it would likely slow down the computation massively for the affinityPlugin when using `spreadDomain` or `availabilityZone`. Since for every placement, `x` or `n - x` nodes would have to be resorted.
   
   So in the meantime the balancing will be good, but it won't be optimal (likely). The `computeBalancing()` loop will continue until there are no replicas that can be moved around to make weighting more optimized, so maybe I'm over thinking this anyways...
   
   Anyways, I think this is a fine concession to get the feature in Solr. And it's something we can make incremental improvements on in the future. (This is also why I left a `@lucene.experimental` tag on `WeightedNode`. The improvements we make might need different method signatures, like I had to make `addReplica()` return a boolean to make `computePlacements` optimal.)
   
   I'll add comments to the `computeBalancing()` method to make this clear to future people wanting to work on the code.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224379842


##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/AffinityPlacementFactory.java:
##########
@@ -251,1006 +231,463 @@ private AffinityPlacementPlugin(
 
       // We make things reproducible in tests by using test seed if any
       String seed = System.getProperty("tests.seed");
-      if (seed != null) {
-        replicaPlacementRandom.setSeed(seed.hashCode());
-      }
-    }
-
-    @Override
-    @SuppressForbidden(
-        reason =
-            "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-    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());
-      }
-
-      // Fetch attributes for a superset of all nodes requested amongst the placementRequests
-      AttributeFetcher attributeFetcher = placementContext.getAttributeFetcher();
-      attributeFetcher
-          .requestNodeSystemProperty(AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.NODE_TYPE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-          .requestNodeSystemProperty(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
-      attributeFetcher
-          .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
-          .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB);
-      Set<ReplicaMetric<?>> replicaMetrics = Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
-      for (SolrCollection collection : allCollections) {
-        attributeFetcher.requestCollectionMetrics(collection, replicaMetrics);
-      }
-      attributeFetcher.fetchFrom(allNodes);
-      final AttributeValues attrValues = attributeFetcher.fetchAttributes();
-      // Get the number of currently existing cores per node, so we can update as we place new cores
-      // to not end up always selecting the same node(s). This is used across placement requests
-      Map<Node, Integer> allCoresOnNodes = getCoreCountPerNode(allNodes, attrValues);
-
-      boolean doSpreadAcrossDomains = shouldSpreadAcrossDomains(allNodes, attrValues);
-
-      // Keep track with nodesWithReplicas across requests
-      Map<String, Map<String, Set<Node>>> allNodesWithReplicas = new HashMap<>();
-      for (PlacementRequest request : requests) {
-        Set<Node> nodes = request.getTargetNodes();
-        SolrCollection solrCollection = request.getCollection();
-
-        // filter out nodes that don't meet the `withCollection` constraint
-        nodes =
-            filterNodesWithCollection(placementContext.getCluster(), request, attrValues, nodes);
-        // filter out nodes that don't match the "node types" specified in the collection props
-        nodes = filterNodesByNodeType(placementContext.getCluster(), request, attrValues, nodes);
-
-        // All available zones of live nodes. Due to some nodes not being candidates for placement,
-        // and some existing replicas being one availability zones that might be offline (i.e. their
-        // nodes are not live), this set might contain zones on which it is impossible to place
-        // replicas. That's ok.
-        Set<String> availabilityZones = getZonesFromNodes(nodes, attrValues);
-
-        // Build the replica placement decisions here
-        Set<ReplicaPlacement> replicaPlacements = new HashSet<>();
-
-        // Let's now iterate on all shards to create replicas for and start finding home sweet homes
-        // for the replicas
-        for (String shardName : request.getShardNames()) {
-          ReplicaMetrics leaderMetrics =
-              attrValues
-                  .getCollectionMetrics(solrCollection.getName())
-                  .flatMap(colMetrics -> colMetrics.getShardMetrics(shardName))
-                  .flatMap(ShardMetrics::getLeaderMetrics)
-                  .orElse(null);
-
-          // Split the set of nodes into 3 sets of nodes accepting each replica type (sets can
-          // overlap
-          // if nodes accept multiple replica types). These subsets sets are actually maps, because
-          // we
-          // capture the number of cores (of any replica type) present on each node.
-          //
-          // This also filters out nodes that will not satisfy the rules if the replica is placed
-          // there
-          EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
-              getAvailableNodesForReplicaTypes(nodes, attrValues, leaderMetrics);
-
-          // Inventory nodes (if any) that already have a replica of any type for the shard, because
-          // we can't be placing additional replicas on these. This data structure is updated after
-          // each replica to node assign and is used to make sure different replica types are not
-          // allocated to the same nodes (protecting same node assignments within a given replica
-          // type is done "by construction" in makePlacementDecisions()).
-          Set<Node> nodesWithReplicas =
-              allNodesWithReplicas
-                  .computeIfAbsent(solrCollection.getName(), col -> new HashMap<>())
-                  .computeIfAbsent(
-                      shardName,
-                      s -> {
-                        Set<Node> newNodeSet = new HashSet<>();
-                        Shard shard = solrCollection.getShard(s);
-                        if (shard != null) {
-                          // Prefill the set with the existing replicas
-                          for (Replica r : shard.replicas()) {
-                            newNodeSet.add(r.getNode());
-                          }
-                        }
-                        return newNodeSet;
-                      });
-
-          // Iterate on the replica types in the enum order. We place more strategic replicas first
-          // (NRT is more strategic than TLOG more strategic than PULL). This is in case we
-          // eventually decide that less strategic replica placement impossibility is not a problem
-          // that should lead to replica placement computation failure. Current code does fail if
-          // placement is impossible (constraint is at most one replica of a shard on any node).
-          for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-            int numReplicasToCreate = request.getCountReplicasToCreate(replicaType);
-            if (numReplicasToCreate > 0) {
-              makePlacementDecisions(
-                  solrCollection,
-                  shardName,
-                  availabilityZones,
-                  replicaType,
-                  numReplicasToCreate,
-                  attrValues,
-                  leaderMetrics,
-                  replicaTypeToNodes,
-                  nodesWithReplicas,
-                  allCoresOnNodes,
-                  placementContext.getPlacementPlanFactory(),
-                  replicaPlacements,
-                  doSpreadAcrossDomains);
-            }
-          }
-        }
-        placementPlans.add(
-            placementContext
-                .getPlacementPlanFactory()
-                .createPlacementPlan(request, replicaPlacements));
-      }
-
-      return placementPlans;
-    }
-
-    private boolean shouldSpreadAcrossDomains(Set<Node> allNodes, AttributeValues attrValues) {
-      boolean doSpreadAcrossDomains =
-          spreadAcrossDomains && spreadDomainPropPresent(allNodes, attrValues);
-      if (spreadAcrossDomains && !doSpreadAcrossDomains) {
-        log.warn(
-            "AffinityPlacementPlugin configured to spread across domains, but there are nodes in the cluster without the {} system property. Ignoring spreadAcrossDomains.",
-            AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
-      }
-      return doSpreadAcrossDomains;
-    }
-
-    private boolean spreadDomainPropPresent(Set<Node> allNodes, AttributeValues attrValues) {
-      // We can only use spread domains if all nodes have the system property
-      return allNodes.stream()
-          .noneMatch(
-              n ->
-                  attrValues
-                      .getSystemProperty(n, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                      .isEmpty());
     }
 
     @Override
-    public void verifyAllowedModification(
-        ModificationRequest modificationRequest, PlacementContext placementContext)
-        throws PlacementModificationException {
-      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);
-      }
-    }
-
-    private void verifyDeleteCollection(
+    protected void verifyDeleteCollection(
         DeleteCollectionRequest deleteCollectionRequest, PlacementContext placementContext)
         throws PlacementModificationException {
       Cluster cluster = placementContext.getCluster();
-      Set<String> colocatedCollections =
-          colocatedWith.getOrDefault(deleteCollectionRequest.getCollection().getName(), Set.of());
-      for (String primaryName : colocatedCollections) {
+      Set<String> collocatedCollections =
+          collocatedWith.getOrDefault(deleteCollectionRequest.getCollection().getName(), Set.of());
+      for (String primaryName : collocatedCollections) {
         try {
           if (cluster.getCollection(primaryName) != null) {
             // still exists
             throw new PlacementModificationException(
-                "colocated collection "
+                "collocated collection "
                     + primaryName
                     + " of "
                     + deleteCollectionRequest.getCollection().getName()
                     + " still present");
           }
         } catch (IOException e) {
           throw new PlacementModificationException(
-              "failed to retrieve colocated collection information", e);
+              "failed to retrieve collocated collection information", e);
         }
       }
     }
 
-    private void verifyDeleteReplicas(
-        DeleteReplicasRequest deleteReplicasRequest, PlacementContext placementContext)
-        throws PlacementModificationException {
-      Cluster cluster = placementContext.getCluster();
-      SolrCollection secondaryCollection = deleteReplicasRequest.getCollection();
-      Set<String> colocatedCollections = colocatedWith.get(secondaryCollection.getName());
-      if (colocatedCollections == null) {
-        return;
-      }
-      Map<Node, Map<String, AtomicInteger>> secondaryNodeShardReplicas = new HashMap<>();
-      secondaryCollection
-          .shards()
-          .forEach(
-              shard ->
-                  shard
-                      .replicas()
-                      .forEach(
-                          replica ->
-                              secondaryNodeShardReplicas
-                                  .computeIfAbsent(replica.getNode(), n -> new HashMap<>())
-                                  .computeIfAbsent(
-                                      replica.getShard().getShardName(), s -> new AtomicInteger())
-                                  .incrementAndGet()));
+    private static final class AffinityPlacementContext {
+      private final Set<String> allSpreadDomains = new HashSet<>();
+      private final Map<String, Map<String, ReplicaSpread>> spreadDomainUsage = new HashMap<>();
+      private final Set<String> allAvailabilityZones = new HashSet<>();
+      private final Map<String, Map<String, Map<Replica.ReplicaType, ReplicaSpread>>>
+          availabilityZoneUsage = new HashMap<>();
+      private boolean doSpreadAcrossDomains;
+    }
 
-      // find the colocated-with collections
-      Map<Node, Set<String>> colocatingNodes = new HashMap<>();
-      try {
-        for (String colocatedCollection : colocatedCollections) {
-          SolrCollection coll = cluster.getCollection(colocatedCollection);
-          coll.shards()
-              .forEach(
-                  shard ->
-                      shard
-                          .replicas()
-                          .forEach(
-                              replica ->
-                                  colocatingNodes
-                                      .computeIfAbsent(replica.getNode(), n -> new HashSet<>())
-                                      .add(coll.getName())));
-        }
-      } catch (IOException ioe) {
-        throw new PlacementModificationException(
-            "failed to retrieve colocated collection information", ioe);
-      }
-      PlacementModificationException exception = null;
-      for (Replica replica : deleteReplicasRequest.getReplicas()) {
-        if (!colocatingNodes.containsKey(replica.getNode())) {
-          continue;
-        }
-        // check that there will be at least one replica remaining
-        AtomicInteger secondaryCount =
-            secondaryNodeShardReplicas
-                .getOrDefault(replica.getNode(), Map.of())
-                .getOrDefault(replica.getShard().getShardName(), new AtomicInteger());
-        if (secondaryCount.get() > 1) {
-          // we can delete it - record the deletion
-          secondaryCount.decrementAndGet();
-          continue;
-        }
-        // fail - this replica cannot be removed
-        if (exception == null) {
-          exception = new PlacementModificationException("delete replica(s) rejected");
+    @Override
+    protected Map<Node, WeightedNode> getBaseWeightedNodes(
+        PlacementContext placementContext,
+        Set<Node> nodes,
+        Iterable<SolrCollection> relevantCollections,
+        boolean skipNodesWithErrors)
+        throws PlacementException {
+      // Fetch attributes for a superset of all nodes requested amongst the placementRequests
+      AttributeFetcher attributeFetcher = placementContext.getAttributeFetcher();
+      attributeFetcher
+          .requestNodeSystemProperty(AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.NODE_TYPE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
+          .requestNodeSystemProperty(AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
+      attributeFetcher
+          .requestNodeMetric(BuiltInMetrics.NODE_NUM_CORES)
+          .requestNodeMetric(BuiltInMetrics.NODE_FREE_DISK_GB);
+      Set<ReplicaMetric<?>> replicaMetrics = Set.of(BuiltInMetrics.REPLICA_INDEX_SIZE_GB);
+      Set<String> requestedCollections = new HashSet<>();
+      for (SolrCollection collection : relevantCollections) {
+        if (requestedCollections.add(collection.getName())) {
+          attributeFetcher.requestCollectionMetrics(collection, replicaMetrics);
         }
-        exception.addRejectedModification(
-            replica.toString(),
-            "co-located with replicas of " + colocatingNodes.get(replica.getNode()));
-      }
-      if (exception != null) {
-        throw exception;
       }
-    }
+      attributeFetcher.fetchFrom(nodes);
+      final AttributeValues attrValues = attributeFetcher.fetchAttributes();
 
-    private Set<String> getZonesFromNodes(Set<Node> nodes, final AttributeValues attrValues) {
-      Set<String> azs = new HashSet<>();
+      AffinityPlacementContext affinityPlacementContext = new AffinityPlacementContext();
+      affinityPlacementContext.doSpreadAcrossDomains = spreadAcrossDomains;
 
-      for (Node n : nodes) {
-        azs.add(getNodeAZ(n, attrValues));
+      Map<Node, WeightedNode> affinityNodeMap = CollectionUtil.newHashMap(nodes.size());
+      for (Node node : nodes) {
+        AffinityNode affinityNode =
+            newNodeFromMetrics(node, attrValues, affinityPlacementContext, skipNodesWithErrors);
+        if (affinityNode != null) {
+          affinityNodeMap.put(node, affinityNode);
+        }
       }
 
-      return Collections.unmodifiableSet(azs);
+      return affinityNodeMap;
     }
 
-    /**
-     * Resolves the AZ of a node and takes care of nodes that have no defined AZ in system property
-     * {@link AffinityPlacementConfig#AVAILABILITY_ZONE_SYSPROP} to then return {@link
-     * AffinityPlacementConfig#UNDEFINED_AVAILABILITY_ZONE} as the AZ name.
-     */
-    private String getNodeAZ(Node n, final AttributeValues attrValues) {
-      Optional<String> nodeAz =
-          attrValues.getSystemProperty(n, AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP);
-      // All nodes with undefined AZ will be considered part of the same AZ. This also works for
-      // deployments that do not care about AZ's
-      return nodeAz.orElse(AffinityPlacementConfig.UNDEFINED_AVAILABILITY_ZONE);
-    }
-
-    /**
-     * This class captures an availability zone and the nodes that are legitimate targets for
-     * replica placement in that Availability Zone. Instances are used as values in a {@link
-     * java.util.TreeMap} in which the total number of already existing replicas in the AZ is the
-     * key. This allows easily picking the set of nodes from which to select a node for placement in
-     * order to balance the number of replicas per AZ. Picking one of the nodes from the set is done
-     * using different criteria unrelated to the Availability Zone (picking the node is based on the
-     * {@link CoresAndDiskComparator} ordering).
-     */
-    private static class AzWithNodes {
-      final String azName;
-      private final boolean useSpreadDomains;
-      private boolean listIsSorted = false;
-      private final Comparator<Node> nodeComparator;
-      private final Random random;
-      private final List<Node> availableNodesForPlacement;
-      private final AttributeValues attributeValues;
-      private final ReplicaMetrics leaderMetrics;
-      private TreeSet<SpreadDomainWithNodes> sortedSpreadDomains;
-      private final Map<String, Integer> currentSpreadDomainUsageUsage;
-      private int numNodesForPlacement;
-
-      AzWithNodes(
-          String azName,
-          List<Node> availableNodesForPlacement,
-          boolean useSpreadDomains,
-          Comparator<Node> nodeComparator,
-          Random random,
-          AttributeValues attributeValues,
-          ReplicaMetrics leaderMetrics,
-          Map<String, Integer> currentSpreadDomainUsageUsage) {
-        this.azName = azName;
-        this.availableNodesForPlacement = availableNodesForPlacement;
-        this.useSpreadDomains = useSpreadDomains;
-        this.nodeComparator = nodeComparator;
-        this.random = random;
-        this.attributeValues = attributeValues;
-        this.leaderMetrics = leaderMetrics;
-        this.currentSpreadDomainUsageUsage = currentSpreadDomainUsageUsage;
-        this.numNodesForPlacement = availableNodesForPlacement.size();
-      }
-
-      private boolean hasBeenSorted() {
-        return (useSpreadDomains && sortedSpreadDomains != null)
-            || (!useSpreadDomains && listIsSorted);
-      }
-
-      void ensureSorted() {
-        if (!hasBeenSorted()) {
-          sort();
-        }
+    AffinityNode newNodeFromMetrics(
+        Node node,
+        AttributeValues attrValues,
+        AffinityPlacementContext affinityPlacementContext,
+        boolean skipNodesWithErrors)
+        throws PlacementException {
+      Set<Replica.ReplicaType> supportedReplicaTypes =
+          attrValues.getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP).stream()
+              .flatMap(s -> Arrays.stream(s.split(",")))
+              .map(String::trim)
+              .map(s -> s.toUpperCase(Locale.ROOT))
+              .map(
+                  s -> {
+                    try {
+                      return Replica.ReplicaType.valueOf(s);
+                    } catch (IllegalArgumentException e) {
+                      log.warn(
+                          "Node {} has an invalid value for the {} systemProperty: {}",
+                          node.getName(),
+                          AffinityPlacementConfig.REPLICA_TYPE_SYSPROP,
+                          s);
+                      return null;
+                    }
+                  })
+              .collect(Collectors.toSet());
+      if (supportedReplicaTypes.isEmpty()) {
+        // If property not defined or is only whitespace on a node, assuming node can take any
+        // replica type
+        supportedReplicaTypes = Set.of(Replica.ReplicaType.values());
       }
 
-      private void sort() {
-        assert !listIsSorted && sortedSpreadDomains == null
-            : "We shouldn't be sorting this list again";
-
-        // Make sure we do not tend to use always the same nodes (within an AZ) if all
-        // conditions are identical (well, this likely is not the case since after having added
-        // a replica to a node its number of cores increases for the next placement decision,
-        // but let's be defensive here, given that multiple concurrent placement decisions might
-        // see the same initial cluster state, and we want placement to be reasonable even in
-        // that case without creating an unnecessary imbalance). For example, if all nodes have
-        // 0 cores and same amount of free disk space, ideally we want to pick a random node for
-        // placement, not always the same one due to some internal ordering.
-        Collections.shuffle(availableNodesForPlacement, random);
-
-        if (useSpreadDomains) {
-          // When we use spread domains, we don't just sort the list of nodes, instead we generate a
-          // TreeSet of SpreadDomainWithNodes,
-          // sorted by the number of times the domain has been used. Each
-          // SpreadDomainWithNodes internally contains the list of nodes that belong to that
-          // particular domain,
-          // and it's sorted internally by the comparator passed to this
-          // class (which is the same that's used when not using spread domains).
-          // Whenever a node from a particular SpreadDomainWithNodes is selected as the best
-          // candidate, the call to "removeBestNode" will:
-          // 1. Remove the SpreadDomainWithNodes instance from the TreeSet
-          // 2. Remove the best node from the list within the SpreadDomainWithNodes
-          // 3. Increment the count of times the domain has been used
-          // 4. Re-add the SpreadDomainWithNodes instance to the TreeSet if there are still nodes
-          // available
-          HashMap<String, List<Node>> spreadDomainToListOfNodesMap = new HashMap<>();
-          for (Node node : availableNodesForPlacement) {
-            spreadDomainToListOfNodesMap
-                .computeIfAbsent(
-                    attributeValues
-                        .getSystemProperty(node, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                        .get(),
-                    k -> new ArrayList<>())
-                .add(node);
-          }
-          sortedSpreadDomains =
-              new TreeSet<>(new SpreadDomainComparator(currentSpreadDomainUsageUsage));
-
-          int i = 0;
-          for (Map.Entry<String, List<Node>> entry : spreadDomainToListOfNodesMap.entrySet()) {
-            // Sort the nodes within the spread domain by the provided comparator
-            entry.getValue().sort(nodeComparator);
-            sortedSpreadDomains.add(
-                new SpreadDomainWithNodes(entry.getKey(), entry.getValue(), i++, nodeComparator));
+      Set<String> nodeType;
+      Optional<String> nodePropOpt =
+          attrValues.getSystemProperty(node, AffinityPlacementConfig.NODE_TYPE_SYSPROP);
+      if (nodePropOpt.isEmpty()) {
+        nodeType = Collections.emptySet();
+      } else {
+        nodeType = new HashSet<>(StrUtils.splitSmart(nodePropOpt.get(), ','));
+      }
+
+      Optional<Double> nodeFreeDiskGB =
+          attrValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
+      Optional<Integer> nodeNumCores =
+          attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES);
+      String az =
+          attrValues
+              .getSystemProperty(node, AffinityPlacementConfig.AVAILABILITY_ZONE_SYSPROP)
+              .orElse(AffinityPlacementConfig.UNDEFINED_AVAILABILITY_ZONE);
+      affinityPlacementContext.allAvailabilityZones.add(az);
+      String spreadDomain;
+      if (affinityPlacementContext.doSpreadAcrossDomains) {
+        spreadDomain =
+            attrValues
+                .getSystemProperty(node, AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
+                .orElse(null);
+        if (spreadDomain == null) {
+          if (log.isWarnEnabled()) {
+            log.warn(
+                "AffinityPlacementPlugin configured to spread across domains, but node {} does not have the {} system property. Ignoring spreadAcrossDomains.",
+                node.getName(),
+                AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP);
           }
+          affinityPlacementContext.doSpreadAcrossDomains = false;
+          affinityPlacementContext.allSpreadDomains.clear();
         } else {
-          availableNodesForPlacement.sort(nodeComparator);
-          listIsSorted = true;
+          affinityPlacementContext.allSpreadDomains.add(spreadDomain);
         }
+      } else {
+        spreadDomain = null;
       }
-
-      Node getBestNode() {
-        assert hasBeenSorted();
-        if (useSpreadDomains) {
-          return sortedSpreadDomains.first().sortedNodesForPlacement.get(0);
-        } else {
-          return availableNodesForPlacement.get(0);
+      if (nodeFreeDiskGB.isEmpty() && skipNodesWithErrors) {
+        if (log.isWarnEnabled()) {
+          log.warn(
+              "Unknown free disk on node {}, excluding it from placement decisions.",
+              node.getName());
         }
-      }
-
-      public Node removeBestNode() {
-        assert hasBeenSorted();
-        this.numNodesForPlacement--;
-        Node n;
-        if (useSpreadDomains) {
-          // Since this SpreadDomainWithNodes needs to be re-sorted in the sortedSpreadDomains, we
-          // remove it and then re-add it, once the best node has been removed.
-          SpreadDomainWithNodes group = sortedSpreadDomains.pollFirst();
-          n = group.sortedNodesForPlacement.remove(0);
-          this.currentSpreadDomainUsageUsage.merge(group.spreadDomainName, 1, Integer::sum);
-          if (!group.sortedNodesForPlacement.isEmpty()) {
-            sortedSpreadDomains.add(group);
-          }
-        } else {
-          n = availableNodesForPlacement.remove(0);
+        return null;
+      } else if (nodeNumCores.isEmpty() && skipNodesWithErrors) {
+        if (log.isWarnEnabled()) {
+          log.warn(
+              "Unknown number of cores on node {}, excluding it from placement decisions.",
+              node.getName());
         }
-        Optional.ofNullable(leaderMetrics)
-            .flatMap(lrm -> lrm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
-            .ifPresent(
-                indexSize ->
-                    attributeValues.decreaseNodeMetric(
-                        n, BuiltInMetrics.NODE_FREE_DISK_GB, indexSize));
-        attributeValues.increaseNodeMetric(n, BuiltInMetrics.NODE_NUM_CORES, 1);
-        return n;
-      }
-
-      public int numNodes() {
-        return this.numNodesForPlacement;
+        return null;
+      } else {
+        return new AffinityNode(
+            node,
+            attrValues,
+            affinityPlacementContext,
+            supportedReplicaTypes,
+            nodeType,
+            nodeNumCores.orElse(0),
+            nodeFreeDiskGB.orElse(0D),
+            az,
+            spreadDomain);
       }
     }
 
-    /**
-     * This class represents group of nodes with the same {@link
-     * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label.
-     */
-    static class SpreadDomainWithNodes implements Comparable<SpreadDomainWithNodes> {
+    private class AffinityNode extends WeightedNode {
 
-      /**
-       * This is the label that all nodes in this group have in {@link
-       * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label.
-       */
-      final String spreadDomainName;
+      private final AttributeValues attrValues;
 
-      /**
-       * The list of all nodes that contain the same {@link
-       * AffinityPlacementConfig#SPREAD_DOMAIN_SYSPROP} label. They must be sorted before creating
-       * this class.
-       */
-      private final List<Node> sortedNodesForPlacement;
+      private final AffinityPlacementContext affinityPlacementContext;
 
-      /**
-       * This is used for tie breaking the sort of {@link SpreadDomainWithNodes}, when the
-       * nodeComparator between the top nodes of each group return 0.
-       */
-      private final int tieBreaker;
+      private final Set<Replica.ReplicaType> supportedReplicaTypes;
+      private final Set<String> nodeType;
 
-      /**
-       * This is the comparator that is used to compare the top nodes in the {@link
-       * #sortedNodesForPlacement} lists. Must be the same that was used to sort {@link
-       * #sortedNodesForPlacement}.
-       */
-      private final Comparator<Node> nodeComparator;
+      private int coresOnNode;
+      private double nodeFreeDiskGB;
 
-      public SpreadDomainWithNodes(
-          String spreadDomainName,
-          List<Node> sortedNodesForPlacement,
-          int tieBreaker,
-          Comparator<Node> nodeComparator) {
-        this.spreadDomainName = spreadDomainName;
-        this.sortedNodesForPlacement = sortedNodesForPlacement;
-        this.tieBreaker = tieBreaker;
-        this.nodeComparator = nodeComparator;
+      private final String availabilityZone;
+      private final String spreadDomain;
+
+      AffinityNode(
+          Node node,
+          AttributeValues attrValues,
+          AffinityPlacementContext affinityPlacementContext,
+          Set<Replica.ReplicaType> supportedReplicaTypes,
+          Set<String> nodeType,
+          int coresOnNode,
+          double nodeFreeDiskGB,
+          String az,
+          String spreadDomain) {
+        super(node);
+        this.attrValues = attrValues;
+        this.affinityPlacementContext = affinityPlacementContext;
+        this.supportedReplicaTypes = supportedReplicaTypes;
+        this.nodeType = nodeType;
+        this.coresOnNode = coresOnNode;
+        this.nodeFreeDiskGB = nodeFreeDiskGB;
+        this.availabilityZone = az;
+        this.spreadDomain = spreadDomain;
       }
 
       @Override
-      public int compareTo(SpreadDomainWithNodes o) {
-        if (o == this) {
-          return 0;
-        }
-        int result =
-            nodeComparator.compare(
-                this.sortedNodesForPlacement.get(0), o.sortedNodesForPlacement.get(0));
-        if (result == 0) {
-          return Integer.compare(this.tieBreaker, o.tieBreaker);
-        }
-        return result;
+      public int calcWeight() {
+        return coresOnNode
+            + 100 * (prioritizedFreeDiskGB > 0 && nodeFreeDiskGB < prioritizedFreeDiskGB ? 1 : 0)
+            + 10000 * getSpreadDomainWeight()
+            + 1000000 * getAZWeight();
       }
-    }
-
-    /**
-     * Builds the number of existing cores on each node returned in the attrValues. Nodes for which
-     * the number of cores is not available for whatever reason are excluded from acceptable
-     * candidate nodes as it would not be possible to make any meaningful placement decisions.
-     *
-     * @param nodes all nodes on which this plugin should compute placement
-     * @param attrValues attributes fetched for the nodes. This method uses system property {@link
-     *     AffinityPlacementConfig#REPLICA_TYPE_SYSPROP} as well as the number of cores on each
-     *     node.
-     */
-    private Map<Node, Integer> getCoreCountPerNode(
-        Set<Node> nodes, final AttributeValues attrValues) {
-      Map<Node, Integer> coresOnNodes = new HashMap<>();
 
-      for (Node node : nodes) {
-        attrValues
-            .getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES)
-            .ifPresent(count -> coresOnNodes.put(node, count));
+      @Override
+      public int calcRelevantWeightWithReplica(Replica replica) {
+        return coresOnNode
+            + 100
+                * (prioritizedFreeDiskGB > 0
+                        && nodeFreeDiskGB - getProjectedSizeOfReplica(replica)
+                            < prioritizedFreeDiskGB
+                    ? 1
+                    : 0)
+            + 10000 * projectReplicaSpreadWeight(replica)
+            + 1000000 * projectAZWeight(replica);
       }
 
-      return coresOnNodes;
-    }
-
-    /**
-     * Given the set of all nodes on which to do placement and fetched attributes, builds the sets
-     * representing candidate nodes for placement of replicas of each replica type. These sets are
-     * packaged and returned in an EnumMap keyed by replica type. Nodes for which the number of
-     * cores is not available for whatever reason are excluded from acceptable candidate nodes as it
-     * would not be possible to make any meaningful placement decisions.
-     *
-     * @param nodes all nodes on which this plugin should compute placement
-     * @param attrValues attributes fetched for the nodes. This method uses system property {@link
-     *     AffinityPlacementConfig#REPLICA_TYPE_SYSPROP} as well as the number of cores on each
-     *     node.
-     */
-    private EnumMap<Replica.ReplicaType, Set<Node>> getAvailableNodesForReplicaTypes(
-        Set<Node> nodes, final AttributeValues attrValues, final ReplicaMetrics leaderMetrics) {
-      EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes =
-          new EnumMap<>(Replica.ReplicaType.class);
-
-      for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
-        replicaTypeToNodes.put(replicaType, new HashSet<>());
+      @Override
+      public boolean canAddReplica(Replica replica) {
+        String collection = replica.getShard().getCollection().getName();
+        return
+        // By default, do not allow two replicas of the same shard on a node
+        super.canAddReplica(replica)
+            && supportedReplicaTypes.contains(replica.getType())
+            && Optional.ofNullable(nodeTypes.get(collection))
+                .map(s -> s.stream().anyMatch(nodeType::contains))
+                .orElse(true)
+            && Optional.ofNullable(withCollections.get(collection))
+                .map(this::hasCollectionOnNode)
+                .orElse(true)
+            && (minimalFreeDiskGB <= 0
+                || nodeFreeDiskGB - getProjectedSizeOfReplica(replica) > minimalFreeDiskGB);
       }
 
-      for (Node node : nodes) {
-        // Exclude nodes with unknown or too small disk free space
-        Optional<Double> nodeFreeDiskGB =
-            attrValues.getNodeMetric(node, BuiltInMetrics.NODE_FREE_DISK_GB);
-        if (nodeFreeDiskGB.isEmpty()) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Unknown free disk on node {}, excluding it from placement decisions.",
-                node.getName());
-          }
-          // We rely later on the fact that the free disk optional is present (see
-          // CoresAndDiskComparator), be careful it you change anything here.
-          continue;
-        }
-        double replicaIndexSize =
-            Optional.ofNullable(leaderMetrics)
-                .flatMap(lm -> lm.getReplicaMetric(BuiltInMetrics.REPLICA_INDEX_SIZE_GB))
-                .orElse(0D);
-        double projectedFreeDiskIfPlaced =
-            BuiltInMetrics.NODE_FREE_DISK_GB.decrease(nodeFreeDiskGB.get(), replicaIndexSize);
-        if (projectedFreeDiskIfPlaced < minimalFreeDiskGB) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Node {} free disk ({}GB) minus the projected replica size ({}GB) is lower than configured"
-                    + " minimum {}GB, excluding it from placement decisions.",
-                node.getName(),
-                nodeFreeDiskGB.get(),
-                replicaIndexSize,
-                minimalFreeDiskGB);
-          }
-          continue;
-        }
-
-        if (attrValues.getNodeMetric(node, BuiltInMetrics.NODE_NUM_CORES).isEmpty()) {
-          if (log.isWarnEnabled()) {
-            log.warn(
-                "Unknown number of cores on node {}, excluding it from placement decisions.",
-                node.getName());
+      @Override
+      public Map<Replica, String> canRemoveReplicas(Collection<Replica> replicas) {
+        Map<Replica, String> replicaRemovalExceptions = new HashMap<>();
+        Map<String, Map<String, Set<Replica>>> removals = new HashMap<>();
+        for (Replica replica : replicas) {
+          SolrCollection collection = replica.getShard().getCollection();
+          Set<String> collocatedCollections = new HashSet<>();
+          Optional.ofNullable(collocatedWith.get(collection.getName()))
+              .ifPresent(collocatedCollections::addAll);
+          collocatedCollections.retainAll(getCollectionsOnNode());
+          if (collocatedCollections.isEmpty()) {
+            continue;
           }
-          // We rely later on the fact that the number of cores optional is present (see
-          // CoresAndDiskComparator), be careful it you change anything here.
-          continue;
-        }
 
-        String supportedReplicaTypes =
-            attrValues
-                    .getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-                    .isPresent()
-                ? attrValues
-                    .getSystemProperty(node, AffinityPlacementConfig.REPLICA_TYPE_SYSPROP)
-                    .get()
-                : null;
-        // If property not defined or is only whitespace on a node, assuming node can take any
-        // replica type
-        if (supportedReplicaTypes == null || supportedReplicaTypes.isBlank()) {
-          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-            replicaTypeToNodes.get(rt).add(node);
-          }
-        } else {
-          Set<String> acceptedTypes =
-              Arrays.stream(supportedReplicaTypes.split(","))
-                  .map(String::trim)
-                  .map(s -> s.toLowerCase(Locale.ROOT))
-                  .collect(Collectors.toSet());
-          for (Replica.ReplicaType rt : Replica.ReplicaType.values()) {
-            if (acceptedTypes.contains(rt.name().toLowerCase(Locale.ROOT))) {
-              replicaTypeToNodes.get(rt).add(node);
-            }
+          // There are collocatedCollections for this shard, so make sure there is a replica of this
+          // shard left on the node after it is removed
+          Set<Replica> replicasRemovedForShard =
+              removals
+                  .computeIfAbsent(
+                      replica.getShard().getCollection().getName(), k -> new HashMap<>())
+                  .computeIfAbsent(replica.getShard().getShardName(), k -> new HashSet<>());
+          replicasRemovedForShard.add(replica);
+
+          if (replicasRemovedForShard.size()
+              >= getReplicasForShardOnNode(replica.getShard()).size()) {
+            replicaRemovalExceptions.put(
+                replica, "co-located with replicas of " + collocatedCollections);
           }
         }
+        return replicaRemovalExceptions;
       }
-      return replicaTypeToNodes;
-    }
 
-    /**
-     * Picks nodes from {@code targetNodes} for placing {@code numReplicas} replicas.
-     *
-     * <p>The criteria used in this method are, in this order:
-     *
-     * <ol>
-     *   <li>No more than one replica of a given shard on a given node (strictly enforced)
-     *   <li>Balance as much as possible replicas of a given {@link
-     *       org.apache.solr.cluster.Replica.ReplicaType} over available AZ's. This balancing takes
-     *       into account existing replicas <b>of the corresponding replica type</b>, if any.
-     *   <li>Place replicas if possible on nodes having more than a certain amount of free disk
-     *       space (note that nodes with a too small amount of free disk space were eliminated as
-     *       placement targets earlier, in {@link #getAvailableNodesForReplicaTypes(Set,
-     *       AttributeValues, ReplicaMetrics)}). There's a threshold here rather than sorting on the
-     *       amount of free disk space, because sorting on that value would in practice lead to
-     *       never considering the number of cores on a node.
-     *   <li>Place replicas on nodes having a smaller number of cores (the number of cores
-     *       considered for this decision includes previous placement decisions made during the
-     *       processing of the placement request)
-     * </ol>
-     */
-    @SuppressForbidden(
-        reason =
-            "Ordering.arbitrary() has no equivalent in Comparator class. Rather reuse than copy.")
-    private void makePlacementDecisions(
-        SolrCollection solrCollection,
-        String shardName,
-        Set<String> availabilityZones,
-        Replica.ReplicaType replicaType,
-        int numReplicas,
-        final AttributeValues attrValues,
-        final ReplicaMetrics leaderMetrics,
-        EnumMap<Replica.ReplicaType, Set<Node>> replicaTypeToNodes,
-        Set<Node> nodesWithReplicas,
-        Map<Node, Integer> coresOnNodes,
-        PlacementPlanFactory placementPlanFactory,
-        Set<ReplicaPlacement> replicaPlacements,
-        boolean doSpreadAcrossDomains)
-        throws PlacementException {
-      // Count existing replicas per AZ. We count only instances of the type of replica for which we
-      // need to do placement. If we ever want to balance replicas of any type across AZ's (and not
-      // each replica type balanced independently), we'd have to move this data structure to the
-      // caller of this method so it can be reused across different replica type placements for a
-      // given shard. Note then that this change would be risky. For example all NRT's and PULL
-      // replicas for a shard my be correctly balanced over three AZ's, but then all NRT can end up
-      // in the same AZ...
-      Map<String, Integer> azToNumReplicas = new HashMap<>();
-      for (String az : availabilityZones) {
-        azToNumReplicas.put(az, 0);
+      @Override
+      protected boolean addProjectedReplicaWeights(Replica replica) {
+        nodeFreeDiskGB -= getProjectedSizeOfReplica(replica);
+        coresOnNode += 1;
+        return addReplicaToAzAndSpread(replica);
       }
 
-      // Build the set of candidate nodes for the placement, i.e. nodes that can accept the replica
-      // type
-      Set<Node> candidateNodes = new HashSet<>(replicaTypeToNodes.get(replicaType));
-      // Remove nodes that already have a replica for the shard (no two replicas of same shard can
-      // be put on same node)
-      candidateNodes.removeAll(nodesWithReplicas);
+      @Override
+      protected void initReplicaWeights(Replica replica) {
+        addReplicaToAzAndSpread(replica);
+      }
 
-      // This Map will include the affinity labels for the nodes that are currently hosting replicas
-      // of this shard. It will be modified with new placement decisions.
-      Map<String, Integer> spreadDomainsInUse = new HashMap<>();
-      Shard shard = solrCollection.getShard(shardName);
-      if (shard != null) {
-        // shard is non null if we're adding replicas to an already existing collection.
-        // If we're creating the collection, the shards do not exist yet.
-        for (Replica replica : shard.replicas()) {
-          // The node's AZ is counted as having a replica if it has a replica of the same type as
-          // the one we need to place here.
-          if (replica.getType() == replicaType) {
-            final String az = getNodeAZ(replica.getNode(), attrValues);
-            if (azToNumReplicas.containsKey(az)) {
-              // We do not count replicas on AZ's for which we don't have any node to place on
-              // because it would not help the placement decision. If we did want to do that, note
-              // the dereferencing below can't be assumed as the entry will not exist in the map.
-              azToNumReplicas.put(az, azToNumReplicas.get(az) + 1);
-            }
-            if (doSpreadAcrossDomains) {
-              attrValues
-                  .getSystemProperty(
-                      replica.getNode(), AffinityPlacementConfig.SPREAD_DOMAIN_SYSPROP)
-                  .ifPresent(nodeDomain -> spreadDomainsInUse.merge(nodeDomain, 1, Integer::sum));
-            }
-          }
+      private boolean addReplicaToAzAndSpread(Replica replica) {
+        boolean needsResort = false;
+        needsResort |=

Review Comment:
   Instead, I only calculated the azSpread if there are more than one availabilityZone.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224394581


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);

Review Comment:
   So I think this is confusing. The asyncId is passed to the child commands (addReplica and cleanupReplicas), but the commands aren't async themselves. They would have to be added to the overseer queue for that to be the case (right?).
   
   I think the asyncId is just used for tracking. So when it hits this point, the migration is actually successful.
   
   As for the waitForFinalState=false, I think "completed" is fine, since the user didn't ask for the final state to be healthy at the end of the migration. And it's much closer to "completed" than "submitted" IMO.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] tflobbe commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "tflobbe (via GitHub)" <gi...@apache.org>.
tflobbe commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224557307


##########
solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java:
##########
@@ -135,190 +100,35 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
         assignRequests.add(assignRequest);
       }
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
-      replicaPositions = assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
-    }
-    int replicaPositionIdx = 0;
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-      if (log.isInfoEnabled()) {
-        log.info(
-            "Going to create replica for collection={} shard={} on node={}",
-            sourceCollection,
-            sourceReplica.getStr(SHARD_ID_PROP),
-            target);
-      }
-      String targetNode;
-      // Use the assigned replica positions, if target is null or empty (checked above)
-      if (replicaPositions != null) {
-        targetNode = replicaPositions.get(replicaPositionIdx).node;
-        replicaPositionIdx++;
-      } else {
-        targetNode = target;
-      }
-      ZkNodeProps msg =
-          sourceReplica
-              .plus("parallel", String.valueOf(parallel))
-              .plus(CoreAdminParams.NODE, targetNode);
-      if (async != null) msg.getProperties().put(ASYNC, async);
-      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.getStr(SHARD_ID_PROP),
-                              target);
-                      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.getStr(SHARD_ID_PROP),
-                            target);
-                      }
-                    }
-                  })
-              .get(0);
-
-      if (addedReplica != null) {
-        createdReplicas.add(addedReplica);
-        if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
-          String shardName = sourceReplica.getStr(SHARD_ID_PROP);
-          String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
-          String collectionName = sourceCollection;
-          String key = collectionName + "_" + replicaName;
-          CollectionStateWatcher watcher;
-          if (waitForFinalState) {
-            watcher =
-                new ActiveReplicaWatcher(
-                    collectionName,
-                    null,
-                    Collections.singletonList(addedReplica.getStr(ZkStateReader.CORE_NAME_PROP)),
-                    replicasToRecover);
-          } else {
-            watcher =
-                new LeaderRecoveryWatcher(
-                    collectionName,
-                    shardName,
-                    replicaName,
-                    addedReplica.getStr(ZkStateReader.CORE_NAME_PROP),
-                    replicasToRecover);
-          }
-          watchers.put(key, watcher);
-          log.debug("--- adding {}, {}", key, watcher);
-          zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
-        } else {
-          log.debug("--- not waiting for {}", addedReplica);
-        }
+      List<ReplicaPosition> replicaPositions =
+          assignStrategy.assign(ccc.getSolrCloudManager(), assignRequests);
+      int position = 0;
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, replicaPositions.get(position++).node);
       }
-    }
-
-    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()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch =
-          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        NamedList<Object> deleteResult = new NamedList<>();
-        try {
-          new DeleteReplicaCmd(ccc)
-              .deleteReplica(
-                  zkStateReader.getClusterState(),
-                  createdReplica.plus("parallel", "true"),
-                  deleteResult,
-                  () -> {
-                    cleanupLatch.countDown();
-                    if (deleteResult.get("failure") != null) {
-                      synchronized (results) {
-                        results.add(
-                            "failure",
-                            "Could not cleanup, because of : " + deleteResult.get("failure"));
-                      }
-                    }
-                  });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
-        }
+      for (Replica sourceReplica : sourceReplicas) {
+        replicaMovements.put(sourceReplica, target);
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return;
     }
 
-    // we have reached this far means all replicas could be recreated
-    // now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ccc, source, async);
-    results.add(
-        "success",
-        "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+    boolean migrationSuccessful =
+        ReplicaMigrationUtils.migrateReplicas(
+            ccc, replicaMovements, parallel, waitForFinalState, timeout, async, results);
+    if (migrationSuccessful) {
+      results.add(
+          "success",
+          "REPLACENODE action completed successfully from  : " + source + " to : " + target);

Review Comment:
   AH, ok, thanks for exaplaining



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226878640


##########
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:
   Yeah we should call that out, basically low weight is going to be chosen.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] tflobbe commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "tflobbe (via GitHub)" <gi...@apache.org>.
tflobbe commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224558856


##########
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)) {

Review Comment:
   > Honestly I'm not really sure why we are waiting for leaders to recover, since no nodes are being deleted here, only added. Why would the leaders go into recovery? And if so, we should wait to check for recovery until after the new replicas have been added, as I mentioned above.
   
   I was thinking the same thing, I'm not sure I understand this special handling of the leaders
   
   > We really only need to ensure that there is another active replica to take leadership if we are deleting a leader in cleanupReplicas(), so we should probably just create another latch and loop for it after the countDownLatch is complete. (if waitForFinalState=false)
   
   Agree, I think this would be better and also make the code cleaner



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] sonatype-lift[bot] commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "sonatype-lift[bot] (via GitHub)" <gi...@apache.org>.
sonatype-lift[bot] commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1222129466


##########
solr/core/src/java/org/apache/solr/cluster/placement/plugins/OrderedNodePlacementPlugin.java:
##########
@@ -0,0 +1,543 @@
+/*
+ * 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.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).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()) {
+        log.info("Collection: {}, shard: {}", solrCollection.getName(), shardName);
+
+        for (Replica.ReplicaType replicaType : Replica.ReplicaType.values()) {
+          log.info("ReplicaType: {}", replicaType);
+          int replicaCount = request.getCountReplicasToCreate(replicaType);
+          if (replicaCount == 0) {
+            continue;
+          }
+          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);
+                  });
+
+          if (nodesForReplicaType.size() < replicaCount) {
+            throw new PlacementException(
+                "Not enough eligible nodes to place "
+                    + replicaCount
+                    + " replica(s) of type "
+                    + replicaType
+                    + " for shard "
+                    + shardName
+                    + " of collection "
+                    + solrCollection.getName());
+          }
+
+          int nodesChosen = 0;
+          while (nodesChosen < replicaCount) {
+            if (nodesForReplicaType.isEmpty()) {
+              throw new PlacementException(
+                  "There are not enough nodes to handle request to place replica");
+            }
+            WeightedNode node = nodesForReplicaType.poll();
+            while (node.hasWeightChangedSinceSort()) {
+              log.info("Out of date Node: {}", node.getNode());
+              node.addToSortedCollection(nodesForReplicaType);
+              node = nodesForReplicaType.poll();
+            }
+            log.info("Node: {}", node.getNode());
+
+            boolean needsToResort =
+                node.addReplica(
+                    PlacementPlugin.createProjectedReplica(
+                        solrCollection, shardName, replicaType, node.getNode()));
+            nodesChosen += 1;
+            replicaPlacements.add(
+                placementContext
+                    .getPlacementPlanFactory()
+                    .createReplicaPlacement(
+                        solrCollection, shardName, node.getNode(), replicaType));
+            if (needsToResort) {
+              List<WeightedNode> nodeList = new ArrayList<>(nodesForReplicaType);
+              nodesForReplicaType.clear();
+              nodeList.forEach(n -> n.addToSortedCollection(nodesForReplicaType));
+            }
+          }
+        }
+      }
+
+      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())
+            .values();
+    // This is critical to store the last sort weight for this node
+    weightedNodes.forEach(node -> 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()) {
+        // Re-sort this node and go back to find the lowest
+        lowestWeight.addToSortedCollection(orderedNodes);
+        continue;
+      }
+      log.debug(
+          "Lowest 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()) {
+          // Re-sort this node and go back to find the lowest
+          highestWeight.addToSortedCollection(orderedNodes);
+          continue;
+        }
+        log.debug(
+            "Highest 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();
+          log.debug(
+              "Replica: {}, lowestWith: {} ({}), highestWithout: {} ({})",
+              r.getReplicaName(),
+              lowestWeightWithReplica,
+              lowestWeight.canAddReplica(r),
+              highestWeightWithoutReplica,
+              highestWeight.canRemoveReplicas(Set.of(r)));
+
+          // 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;
+          }
+          log.debug("Replica Movement Chosen!");
+          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;
+        }
+      }
+      orderedNodes.addAll(traversedHighNodes);
+      traversedHighNodes.clear();
+      if (newReplicaMovements.size() > 0) {
+        replicaMovements.putAll(newReplicaMovements);
+        // There are no replicas to move to the lowestWeight, remove it from our loop
+        orderedNodes.add(lowestWeight);
+      }
+    }
+
+    return placementContext
+        .getBalancePlanFactory()
+        .createBalancePlan(balanceRequest, replicaMovements);
+  }
+
+  protected Map<Node, WeightedNode> getWeightedNodes(
+      PlacementContext placementContext,
+      Set<Node> nodes,
+      Iterable<SolrCollection> relevantCollections)
+      throws PlacementException {
+    Map<Node, WeightedNode> weightedNodes =
+        getBaseWeightedNodes(placementContext, nodes, relevantCollections);
+
+    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)
+      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());
+
+    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()));
+      }
+      node.canRemoveReplicas(entry.getValue())
+          .forEach(
+              (replica, reason) ->
+                  placementModificationException.addRejectedModification(
+                      replica.toString(), reason));
+    }
+    if (!placementModificationException.getRejectedModifications().isEmpty()) {

Review Comment:
   <picture><img alt="16% of developers fix this issue" src="https://lift.sonatype.com/api/commentimage/fixrate/16/display.svg"></picture>
   
   <b>*NULL_DEREFERENCE:</b>*  object returned by `getRejectedModifications(placementModificationException)` could be null and is dereferenced at line 352.
   
   ---
   
   <details><summary>ℹ️ Expand to see all <b>@sonatype-lift</b> commands</summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   



##########
solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java:
##########
@@ -53,40 +40,33 @@ public void call(ClusterState state, ZkNodeProps message, NamedList<Object> resu
       throws Exception {
     CollectionHandlingUtils.checkRequired(message, "node");
     String node = message.getStr("node");
-    List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
+    List<Replica> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
     List<String> singleReplicas = verifyReplicaAvailability(sourceReplicas, state);
     if (!singleReplicas.isEmpty()) {
       results.add(
           "failure",
           "Can't delete the only existing non-PULL replica(s) on node "
               + node
               + ": "
-              + singleReplicas.toString());
+              + singleReplicas);
     } else {
-      cleanupReplicas(results, state, sourceReplicas, ccc, node, message.getStr(ASYNC));
+      ReplicaMigrationUtils.cleanupReplicas(
+          results, state, sourceReplicas, ccc, message.getStr(ASYNC));
     }
   }
 
   // collect names of replicas that cannot be deleted
-  static List<String> verifyReplicaAvailability(
-      List<ZkNodeProps> sourceReplicas, ClusterState state) {
+  static List<String> verifyReplicaAvailability(List<Replica> sourceReplicas, ClusterState state) {
     List<String> res = new ArrayList<>();
-    for (ZkNodeProps sourceReplica : sourceReplicas) {
-      String coll = sourceReplica.getStr(COLLECTION_PROP);
-      String shard = sourceReplica.getStr(SHARD_ID_PROP);
-      String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
+    for (Replica sourceReplica : sourceReplicas) {
+      String coll = sourceReplica.getCollection();
+      String shard = sourceReplica.getShard();
+      String replicaName = sourceReplica.getName();
       DocCollection collection = state.getCollection(coll);
       Slice slice = collection.getSlice(shard);
       if (slice.getReplicas().size() < 2) {

Review Comment:
   <picture><img alt="7% of developers fix this issue" src="https://lift.sonatype.com/api/commentimage/fixrate/7/display.svg"></picture>
   
   <b>*NULLPTR_DEREFERENCE:</b>*  `slice` could be null (from the call to `DocCollection.getSlice(...)` on line 66) and is dereferenced.
   
   ---
   
   <details><summary>ℹ️ Expand to see all <b>@sonatype-lift</b> commands</summary>
   
   You can reply with the following commands. For example, reply with ***@sonatype-lift ignoreall*** to leave out all findings.
   | **Command** | **Usage** |
   | ------------- | ------------- |
   | `@sonatype-lift ignore` | Leave out the above finding from this PR |
   | `@sonatype-lift ignoreall` | Leave out all the existing findings from this PR |
   | `@sonatype-lift exclude <file\|issue\|path\|tool>` | Exclude specified `file\|issue\|path\|tool` from Lift findings by updating your config.toml file |
   
   **Note:** When talking to LiftBot, you need to **refresh** the page to see its response.
   <sub>[Click here](https://github.com/apps/sonatype-lift/installations/new) to add LiftBot to another repo.</sub></details>
   
   



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224416272


##########
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)) {

Review Comment:
   I guess so, but I also think the logic is flawed. The replicasToRecover watch is created for the leader right after submitting the AddReplica, which can be async. So if it is async, the leader watch could count down before the replica is actually added. In general this logic is a bit iffy.
   
   Honestly I'm not really sure why we are waiting for leaders to recover, since no nodes are being deleted here, only added. Why would the leaders go into recovery? And if so, we should wait to check for recovery until after the new replicas have been added, as I mentioned above.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224381083


##########
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()) {
+      ccc.getZkStateReader().removeCollectionStateWatcher(e.getKey(), e.getValue());
+    }
+    if (anyOneFailed.get()) {
+      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+      SolrCloseableLatch cleanupLatch =
+          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
+      for (ZkNodeProps createdReplica : createdReplicas) {
+        NamedList<Object> deleteResult = new NamedList<>();
+        try {
+          new DeleteReplicaCmd(ccc)
+              .deleteReplica(
+                  ccc.getZkStateReader().getClusterState(),
+                  createdReplica.plus("parallel", "true"),
+                  deleteResult,
+                  () -> {
+                    cleanupLatch.countDown();
+                    if (deleteResult.get("failure") != null) {
+                      synchronized (results) {
+                        results.add(
+                            "failure",
+                            "Could not cleanup, because of : " + deleteResult.get("failure"));
+                      }
+                    }
+                  });
+        } catch (KeeperException e) {
+          cleanupLatch.countDown();
+          log.warn("Error deleting replica ", e);
+        } catch (Exception e) {
+          log.warn("Error deleting replica ", e);
+          cleanupLatch.countDown();
+          throw e;
+        }
+      }
+      cleanupLatch.await(5, TimeUnit.MINUTES);
+      return false;
+    }
+
+    // we have reached this far, meaning all replicas should have been recreated.
+    // now cleanup the original replicas
+    return cleanupReplicas(
+        results, ccc.getZkStateReader().getClusterState(), movements.keySet(), ccc, asyncId);
+  }
+
+  static boolean cleanupReplicas(
+      NamedList<Object> results,
+      ClusterState clusterState,
+      Collection<Replica> sourceReplicas,
+      CollectionCommandContext ccc,
+      String async)
+      throws IOException, InterruptedException {
+    CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());

Review Comment:
   I just moved the existing code here to share for multiple commands , but happy to make the change if you think we should.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1591495743

   I reverted [SOLR-16816](https://issues.apache.org/jira/browse/SOLR-16816), since the functionality was taken care of in this PR. I included the added tests and testing changes from that PR in the latest `merge branch 'main'` commit.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1590005050

   Ok each plugin should have a good amount of docs to explain what is going on.
   
   This should be ready to go IMO.


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1584636842

   Awesome, I'm so glad this weighted approach is a good path to go down. Should be even easier to add new PlacementPlugins, and we get balanceReplicas for "free". I can't even imagine trying to implement the balance API using the previous affinityPlugin logic...
   
   >  I was thinking a cool feature in the future we could have some way of simulating a change (via API maybe, like a dry-run) and get an explain back of why a node was chosen.
   
   A dry run would be great. (And very much mimicks something I have done for a Solr replica management system in the past, it was very useful). Explanations would also be great, but I think even harder given the weighted nature of these moves. "Why" is hard to elaborate on when we just have a single number for a weight. But maybe a weighted node could give explanations for the weight... Would be an interesting addition for sure.
   
   > The AffinityPlacement could use some more comments
   
   Absolutely, will do much more commenting, now that I know I'm not going down a rabbit hole.
   
   > Have you looked at the "scalability" test after the changes? When working on the spread domain changes I noticed some of the implementations I attempted were causing a bit impact in performance there.
   
   Yes, and I love that its there to help keep us in line performance-wise. 
   
   I ran the test 20 times via IntelliJ, and got the following numbers:
   - 20.5 seconds - With this patch
   - 31.75 seconds - Without the patch (main)
   
   So it looks like a 50% improvement in speed, which I'm honestly very surprised by. I thought it would have been slower...


-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] tflobbe commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "tflobbe (via GitHub)" <gi...@apache.org>.
tflobbe commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224559229


##########
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()) {
+      ccc.getZkStateReader().removeCollectionStateWatcher(e.getKey(), e.getValue());
+    }
+    if (anyOneFailed.get()) {
+      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+      SolrCloseableLatch cleanupLatch =
+          new SolrCloseableLatch(createdReplicas.size(), ccc.getCloseableToLatchOn());
+      for (ZkNodeProps createdReplica : createdReplicas) {
+        NamedList<Object> deleteResult = new NamedList<>();
+        try {
+          new DeleteReplicaCmd(ccc)
+              .deleteReplica(
+                  ccc.getZkStateReader().getClusterState(),
+                  createdReplica.plus("parallel", "true"),
+                  deleteResult,
+                  () -> {
+                    cleanupLatch.countDown();
+                    if (deleteResult.get("failure") != null) {
+                      synchronized (results) {
+                        results.add(
+                            "failure",
+                            "Could not cleanup, because of : " + deleteResult.get("failure"));
+                      }
+                    }
+                  });
+        } catch (KeeperException e) {
+          cleanupLatch.countDown();
+          log.warn("Error deleting replica ", e);
+        } catch (Exception e) {
+          log.warn("Error deleting replica ", e);
+          cleanupLatch.countDown();
+          throw e;
+        }
+      }
+      cleanupLatch.await(5, TimeUnit.MINUTES);
+      return false;
+    }
+
+    // we have reached this far, meaning all replicas should have been recreated.
+    // now cleanup the original replicas
+    return cleanupReplicas(
+        results, ccc.getZkStateReader().getClusterState(), movements.keySet(), ccc, asyncId);
+  }
+
+  static boolean cleanupReplicas(
+      NamedList<Object> results,
+      ClusterState clusterState,
+      Collection<Replica> sourceReplicas,
+      CollectionCommandContext ccc,
+      String async)
+      throws IOException, InterruptedException {
+    CountDownLatch cleanupLatch = new CountDownLatch(sourceReplicas.size());

Review Comment:
   I think we probably should? This helps with shutdowns, right?



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1224560199


##########
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)) {

Review Comment:
   Cool, will create a different JIRA for this.



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on a diff in pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on code in PR #1650:
URL: https://github.com/apache/solr/pull/1650#discussion_r1226878099


##########
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:
   Unfortunately not, because `Collection` is the lowest common ancestor of `TreeSet` and `PriorityQueue`. 



-- 
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-unsubscribe@solr.apache.org

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


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


[GitHub] [solr] HoustonPutman commented on pull request #1650: SOLR-16806: Create a BalanceReplicas API

Posted by "HoustonPutman (via GitHub)" <gi...@apache.org>.
HoustonPutman commented on PR #1650:
URL: https://github.com/apache/solr/pull/1650#issuecomment-1568534050

   Yeah not ready for a review yet, still deciding on whether this is the right way to go or not. Thanks for checking though!


-- 
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-unsubscribe@solr.apache.org

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


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