You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by il...@apache.org on 2020/11/18 00:56:33 UTC

[lucene-solr] branch jira/solr-15004 updated: solr-15004: initial example (WIP and buggy) of plugin test

This is an automated email from the ASF dual-hosted git repository.

ilan pushed a commit to branch jira/solr-15004
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/solr-15004 by this push:
     new 2267f3d  solr-15004: initial example (WIP and buggy) of plugin test
2267f3d is described below

commit 2267f3d1640f636fe4933a430bf291403eb2d892
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Wed Nov 18 01:46:04 2020 +0100

    solr-15004: initial example (WIP and buggy) of plugin test
---
 .../impl/PlacementPluginAssignStrategy.java        |   4 +-
 .../placement/impl/PlacementRequestImpl.java       |  14 +-
 .../placement/impl/AttributeFetcherForTest.java    |  93 +++++++
 .../placement/impl/ClusterAbstractionsForTest.java | 292 +++++++++++++++++++++
 .../cluster/placement/impl/PluginTestHelper.java   |  71 +++++
 .../SamplePluginAffinityReplicaPlacementTest.java  |  92 +++++++
 .../impl/SimpleClusterAbstractionsTest.java        |   1 +
 7 files changed, 558 insertions(+), 9 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
index 0bbf4e0..c4c5667 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementPluginAssignStrategy.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.SolrCollection;
 import org.apache.solr.cluster.placement.PlacementException;
 import org.apache.solr.cluster.placement.PlacementPlugin;
 import org.apache.solr.cluster.placement.PlacementPlan;
@@ -53,8 +54,9 @@ public class PlacementPluginAssignStrategy implements Assign.AssignStrategy {
       throws Assign.AssignmentException, IOException, InterruptedException {
 
     Cluster cluster = new SimpleClusterAbstractionsImpl.ClusterImpl(solrCloudManager);
+    SolrCollection solrCollection = new SimpleClusterAbstractionsImpl.SolrCollectionImpl(collection);
 
-    PlacementRequestImpl placementRequest = PlacementRequestImpl.toPlacementRequest(cluster, collection, assignRequest);
+    PlacementRequestImpl placementRequest = PlacementRequestImpl.toPlacementRequest(cluster, solrCollection, assignRequest);
 
     final PlacementPlan placementPlan;
     try {
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
index 80cf6c5..82a26d1 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/PlacementRequestImpl.java
@@ -26,8 +26,7 @@ import org.apache.solr.cluster.Cluster;
 import org.apache.solr.cluster.Node;
 import org.apache.solr.cluster.Replica;
 import org.apache.solr.cluster.SolrCollection;
-import org.apache.solr.cluster.placement.*;
-import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.cluster.placement.PlacementRequest;
 
 public class PlacementRequestImpl implements PlacementRequest {
   private final SolrCollection solrCollection;
@@ -35,7 +34,7 @@ public class PlacementRequestImpl implements PlacementRequest {
   private final Set<Node> targetNodes;
   private final EnumMap<Replica.ReplicaType, Integer> countReplicas = new EnumMap<>(Replica.ReplicaType.class);
 
-  private PlacementRequestImpl(SolrCollection solrCollection,
+  PlacementRequestImpl(SolrCollection solrCollection,
                                Set<String> shardNames, Set<Node> targetNodes,
                                int countNrtReplicas, int countTlogReplicas, int countPullReplicas) {
     this.solrCollection = solrCollection;
@@ -72,12 +71,11 @@ public class PlacementRequestImpl implements PlacementRequest {
    * Returns a {@link PlacementRequest} that can be consumed by a plugin based on an internal Assign.AssignRequest
    * for adding replicas + additional info (upon creation of a new collection or adding replicas to an existing one).
    */
-  static PlacementRequestImpl toPlacementRequest(Cluster cluster, DocCollection docCollection,
+  static PlacementRequestImpl toPlacementRequest(Cluster cluster, SolrCollection solrCollection,
                                                  Assign.AssignRequest assignRequest) throws Assign.AssignmentException {
-    SolrCollection solrCollection = new SimpleClusterAbstractionsImpl.SolrCollectionImpl(docCollection);
     Set<String> shardNames = new HashSet<>(assignRequest.shardNames);
     if (shardNames.size() < 1) {
-      throw new Assign.AssignmentException("Bad assign request: no shards specified for collection " + docCollection.getName());
+      throw new Assign.AssignmentException("Bad assign request: no shards specified for collection " + solrCollection.getName());
     }
 
     final Set<Node> nodes;
@@ -85,12 +83,12 @@ public class PlacementRequestImpl implements PlacementRequest {
     if (assignRequest.nodes != null) {
       nodes = SimpleClusterAbstractionsImpl.NodeImpl.getNodes(assignRequest.nodes);
       if (nodes.isEmpty()) {
-        throw new Assign.AssignmentException("Bad assign request: empty list of nodes for collection " + docCollection.getName());
+        throw new Assign.AssignmentException("Bad assign request: empty list of nodes for collection " + solrCollection.getName());
       }
     } else {
       nodes = cluster.getLiveNodes();
       if (nodes.isEmpty()) {
-        throw new Assign.AssignmentException("Impossible assign request: no live nodes for collection " + docCollection.getName());
+        throw new Assign.AssignmentException("Impossible assign request: no live nodes for collection " + solrCollection.getName());
       }
     }
 
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
new file mode 100644
index 0000000..58005f7
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/AttributeFetcherForTest.java
@@ -0,0 +1,93 @@
+/*
+ * 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 org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.placement.AttributeFetcher;
+import org.apache.solr.cluster.placement.AttributeValues;
+
+import java.util.Set;
+
+public class AttributeFetcherForTest implements AttributeFetcher {
+
+    private final AttributeValues attributeValues;
+
+    AttributeFetcherForTest(AttributeValues attributeValues) {
+        this.attributeValues = attributeValues;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeCoreCount() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeDiskType() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeFreeDisk() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeTotalDisk() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeHeapUsage() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeSystemLoadAverage() {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeSystemProperty(String name) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestNodeEnvironmentVariable(String name) {
+        throw new UnsupportedOperationException("Not yet implemented...");
+    }
+
+    @Override
+    public AttributeFetcher requestNodeMetric(String metricName, NodeMetricRegistry registry) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher fetchFrom(Set<Node> nodes) {
+        return this;
+    }
+
+    @Override
+    public AttributeFetcher requestMetric(String scope, String metricName) {
+        throw new UnsupportedOperationException("Not yet implemented...");
+    }
+
+    @Override
+    public AttributeValues fetchAttributes() {
+        return attributeValues;
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
new file mode 100644
index 0000000..188e3c3
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/ClusterAbstractionsForTest.java
@@ -0,0 +1,292 @@
+/*
+ * 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 org.apache.solr.cluster.*;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.*;
+import java.util.stream.Collectors;
+
+/**
+ * Cluster abstractions independent of any internal SolrCloud abstractions to use in tests (of plugin code).
+ */
+class ClusterAbstractionsForTest {
+
+    static class ClusterImpl implements Cluster {
+        private final Set<Node> liveNodes;
+        private final Map<String, SolrCollection> collections;
+
+        ClusterImpl(Set<Node> liveNodes, Map<String, SolrCollection> collections) throws IOException {
+            this.liveNodes = liveNodes;
+            this.collections = collections;
+        }
+
+        @Override
+        public Set<Node> getLiveNodes() {
+            return liveNodes;
+        }
+
+        @Override
+        public SolrCollection getCollection(String collectionName) {
+            return collections.get(collectionName);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<SolrCollection> iterator() {
+            return collections.values().iterator();
+        }
+
+        @Override
+        public Iterable<SolrCollection> collections() {
+            return ClusterImpl.this::iterator;
+        }
+    }
+
+
+    static class NodeImpl implements Node {
+        public final String nodeName;
+
+        /**
+         * Transforms a collection of node names into a set of {@link Node} instances.
+         */
+        static Set<Node> getNodes(Collection<String> nodeNames) {
+            return nodeNames.stream().map(NodeImpl::new).collect(Collectors.toSet());
+        }
+
+        NodeImpl(String nodeName) {
+            this.nodeName = nodeName;
+        }
+
+        @Override
+        public String getName() {
+            return nodeName;
+        }
+
+        @Override
+        public String toString() {
+            return getClass().getSimpleName() + "(" + getName() + ")";
+        }
+
+        /**
+         * This class ends up as a key in Maps in {@link org.apache.solr.cluster.placement.AttributeValues}.
+         * It is important to implement this method comparing node names given that new instances of {@link Node} are created
+         * with names equal to existing instances (See {@link ReplicaImpl} constructor).
+         */
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            NodeImpl other = (NodeImpl) obj;
+            return Objects.equals(this.nodeName, other.nodeName);
+        }
+
+        public int hashCode() {
+            return Objects.hashCode(nodeName);
+        }
+    }
+
+
+    static class SolrCollectionImpl implements SolrCollection {
+        private final String collectionName;
+        /** Map from {@link Shard#getShardName()} to {@link Shard} */
+        private Map<String, Shard> shards;
+        private final Map<String, String> customProperties;
+
+        SolrCollectionImpl(String collectionName, Map<String, String> customProperties) {
+            this.collectionName = collectionName;
+            this.customProperties = customProperties;
+        }
+
+        /**
+         * Setting the shards has to happen (in tests) after creating the collection because shards reference the collection
+         */
+        void setShards(Map<String, Shard> shards) {
+            this.shards = shards;
+        }
+
+        @Override
+        public String getName() {
+            return collectionName;
+        }
+
+        @Override
+        public Shard getShard(String name) {
+            return shards.get(name);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<Shard> iterator() {
+            return shards.values().iterator();
+        }
+
+        @Override
+        public Iterable<Shard> shards() {
+            return SolrCollectionImpl.this::iterator;
+        }
+
+        @Override
+        public String getCustomProperty(String customPropertyName) {
+            return customProperties.get(customPropertyName);
+        }
+    }
+
+
+    static class ShardImpl implements Shard {
+        private final String shardName;
+        private final SolrCollection collection;
+        private final ShardState shardState;
+        private Map<String, Replica> replicas;
+        private Replica leader;
+
+        ShardImpl(String shardName, SolrCollection collection, ShardState shardState) {
+            this.shardName = shardName;
+            this.collection = collection;
+            this.shardState = shardState;
+        }
+
+        /**
+         * Setting the replicas has to happen (in tests) after creating the shard because replicas reference the shard
+         */
+        void setReplicas(Map<String, Replica> replicas, Replica leader) {
+            this.replicas = replicas;
+            this.leader = leader;
+        }
+
+        @Override
+        public String getShardName() {
+            return shardName;
+        }
+
+        @Override
+        public SolrCollection getCollection() {
+            return collection;
+        }
+
+        @Override
+        public Replica getReplica(String name) {
+            return replicas.get(name);
+        }
+
+        @Override
+        @Nonnull
+        public Iterator<Replica> iterator() {
+            return replicas.values().iterator();
+        }
+
+        @Override
+        public Iterable<Replica> replicas() {
+            return ShardImpl.this::iterator;
+        }
+
+        @Override
+        public Replica getLeader() {
+            return leader;
+        }
+
+        @Override
+        public ShardState getState() {
+            return shardState;
+        }
+
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            ShardImpl other = (ShardImpl) obj;
+            return Objects.equals(this.shardName, other.shardName)
+                    && Objects.equals(this.collection, other.collection)
+                    && Objects.equals(this.shardState, other.shardState)
+                    && Objects.equals(this.replicas, other.replicas)
+                    && Objects.equals(this.leader, other.leader);
+        }
+
+        public int hashCode() {
+            return Objects.hash(shardName, collection, shardState);
+        }
+    }
+
+
+    static class ReplicaImpl implements Replica {
+        private final String replicaName;
+        private final String coreName;
+        private final Shard shard;
+        private final ReplicaType replicaType;
+        private final ReplicaState replicaState;
+        private final Node node;
+
+        ReplicaImpl(String replicaName, String coreName, Shard shard, ReplicaType replicaType, ReplicaState replicaState, Node node) {
+            this.replicaName = replicaName;
+            this.coreName = coreName;
+            this.shard = shard;
+            this.replicaType = replicaType;
+            this.replicaState = replicaState;
+            this.node = node;
+        }
+
+        @Override
+        public Shard getShard() {
+            return shard;
+        }
+
+        @Override
+        public ReplicaType getType() {
+            return replicaType;
+        }
+
+        @Override
+        public ReplicaState getState() {
+            return replicaState;
+        }
+
+        @Override
+        public String getReplicaName() {
+            return replicaName;
+        }
+
+        @Override
+        public String getCoreName() {
+            return coreName;
+        }
+
+        @Override
+        public Node getNode() {
+            return node;
+        }
+
+        public boolean equals(Object obj) {
+            if (obj == null) { return false; }
+            if (obj == this) { return true; }
+            if (obj.getClass() != getClass()) { return false; }
+            ReplicaImpl other = (ReplicaImpl) obj;
+            return Objects.equals(this.replicaName, other.replicaName)
+                    && Objects.equals(this.coreName, other.coreName)
+                    && Objects.equals(this.shard, other.shard)
+                    && Objects.equals(this.replicaType, other.replicaType)
+                    && Objects.equals(this.replicaState, other.replicaState)
+                    && Objects.equals(this.node, other.node);
+        }
+
+        public int hashCode() {
+            return Objects.hash(replicaName, coreName, shard, replicaType, replicaState, node);
+        }
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
new file mode 100644
index 0000000..13ef70d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/PluginTestHelper.java
@@ -0,0 +1,71 @@
+/*
+ * 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 org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Replica;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.SolrCollection;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+
+public class PluginTestHelper {
+
+    /**
+     * Builds the representation of shards for a collection, based on the number of shards and replicas for each to create.
+     * The replicas are allocated to the provided nodes in a round robin way. The leader is set to the last replica of each shard.
+     */
+    static Map<String, Shard> createShardsAndReplicas(SolrCollection collection, int numShards, int numNrtReplicas, Set<Node> nodes) {
+        Iterator<Node> nodeIterator = nodes.iterator();
+
+        Map<String, Shard> shards = new HashMap<>();
+
+        for (int s = 0; s < numShards; s++) {
+            String shardName = collection.getName() + "_s" + s;
+
+            ClusterAbstractionsForTest.ShardImpl shard = new ClusterAbstractionsForTest.ShardImpl(shardName, collection, Shard.ShardState.ACTIVE);
+
+            Map<String, Replica> replicas = new HashMap<>();
+            Replica leader = null;
+            for (int r = 0; r < numNrtReplicas; r++) {
+                String replicaName = shardName + "_r" + r;
+                String coreName = replicaName + "_c";
+                final Node node;
+                if (!nodeIterator.hasNext()) {
+                    nodeIterator = nodes.iterator();
+                }
+                // If the nodes set is empty, this call will fail
+                node = nodeIterator.next();
+
+                Replica replica = new ClusterAbstractionsForTest.ReplicaImpl(replicaName, coreName, shard, Replica.ReplicaType.NRT, Replica.ReplicaState.ACTIVE, node);
+
+                replicas.put(replica.getReplicaName(), replica);
+                leader = replica;
+            }
+
+            shard.setReplicas(replicas, leader);
+
+            shards.put(shard.getShardName(), shard);
+        }
+
+        return shards;
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
new file mode 100644
index 0000000..7c7dee5
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SamplePluginAffinityReplicaPlacementTest.java
@@ -0,0 +1,92 @@
+/*
+ * 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 org.apache.solr.cluster.Cluster;
+import org.apache.solr.cluster.Node;
+import org.apache.solr.cluster.Shard;
+import org.apache.solr.cluster.placement.*;
+import org.apache.solr.cluster.placement.plugins.SamplePluginAffinityReplicaPlacement;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Unit test for {@link SamplePluginAffinityReplicaPlacement}
+ */
+public class SamplePluginAffinityReplicaPlacementTest extends Assert {
+
+    private static PlacementPlugin plugin;
+
+    @BeforeClass
+    public static void setupCluster() {
+        PlacementPluginConfig config = PlacementPluginConfigImpl.createConfigFromProperties(
+                Map.of("minimalFreeDiskGB", 10L, "deprioritizedFreeDiskGB", 50L));
+        plugin = new SamplePluginAffinityReplicaPlacement.Factory().createPluginInstance(config);
+    }
+
+    // FIXME IG: this test fails (expecting placement on node1 but placement happens on node2)
+    @Test
+    public void testBasicPlacementNewCollection() throws Exception {
+        testBasicPlacementInternal(false);
+    }
+
+    @Test
+    public void testBasicPlacementExistingCollection() throws Exception {
+        testBasicPlacementInternal(true);
+    }
+
+    /**
+     * When this test places a replica for a new collection, it should pick the node with less cores.<p>
+     *
+     * When it places a replica for an existing collection, it should pick the node with more cores that doesn't already have a replica for the shard.
+     */
+    private void testBasicPlacementInternal(boolean hasExistingCollection) throws Exception {
+        String collectionName = "testCollection";
+
+        Node node1 = new ClusterAbstractionsForTest.NodeImpl("node1");
+        Node node2 = new ClusterAbstractionsForTest.NodeImpl("node2");
+//        Set<Node> liveNodes = ClusterAbstractionsForTest.NodeImpl.getNodes(Set.of("node1", "node2", "node3"));
+        Set<Node> liveNodes = Set.of(node1, node2);
+        ClusterAbstractionsForTest.SolrCollectionImpl solrCollection = new ClusterAbstractionsForTest.SolrCollectionImpl(collectionName, Map.of());
+        // An existing collection with a single replica on node 1
+        Map<String, Shard> shards = PluginTestHelper.createShardsAndReplicas(solrCollection, 1, 1, Set.of(node1));
+        solrCollection.setShards(shards);
+
+        Cluster cluster = new ClusterAbstractionsForTest.ClusterImpl(liveNodes, hasExistingCollection ? Map.of(solrCollection.getName(), solrCollection) : Map.of());
+        // Place a new replica for the (only) existing shard of the collection
+        PlacementRequestImpl placementRequest = new PlacementRequestImpl(solrCollection, Set.of(shards.keySet().iterator().next()), liveNodes, 1, 0, 0);
+        // More cores on node2
+        Map<Node, Integer> nodeToCoreCount = Map.of(node1, 1, node2, 10);
+        // A lot of free disk on the two nodes
+        final Map<Node, Long> nodeToFreeDisk = Map.of(node1, 100L, node2, 100L);
+        AttributeValues attributeValues = new AttributeValuesImpl(nodeToCoreCount, Map.of(), nodeToFreeDisk, Map.of(), Map.of(), Map.of(), Map.of(), Map.of());
+        AttributeFetcher attributeFetcher = new AttributeFetcherForTest(attributeValues);
+        PlacementPlanFactory placementPlanFactory = new PlacementPlanFactoryImpl();
+
+        PlacementPlan pp = plugin.computePlacement(cluster, placementRequest, attributeFetcher, placementPlanFactory);
+
+
+        assertEquals(1, pp.getReplicaPlacements().size());
+        ReplicaPlacement rp = pp.getReplicaPlacements().iterator().next();
+        assertEquals(hasExistingCollection ? node2 : node1, rp.getNode());
+    }
+}
diff --git a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
index eab2785..5e58779 100644
--- a/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
+++ b/solr/core/src/test/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsTest.java
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.solr.cluster.placement.impl;
 
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;