You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2020/07/09 16:21:10 UTC

[lucene-solr] 01/01: SOLR-14244: Remove ReplicaInfo, part 1.

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

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

commit 413c50c59b66aa1b112506ecbb5222828364a581
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Thu Jul 9 18:20:31 2020 +0200

    SOLR-14244: Remove ReplicaInfo, part 1.
---
 .../stream/AnalyticsShardRequestManager.java       |   2 +-
 .../apache/solr/cloud/ExclusiveSliceProperty.java  |   4 +-
 .../solr/cloud/ShardLeaderElectionContext.java     |   2 +-
 .../java/org/apache/solr/cloud/ZkController.java   |   6 +-
 .../apache/solr/cloud/api/collections/Assign.java  |   6 +-
 .../solr/cloud/api/collections/BackupCmd.java      |   2 +-
 .../cloud/api/collections/CreateSnapshotCmd.java   |   2 +-
 .../cloud/api/collections/DeleteReplicaCmd.java    |   4 +-
 .../solr/cloud/api/collections/DeleteShardCmd.java |   2 +-
 .../cloud/api/collections/DeleteSnapshotCmd.java   |   2 +-
 .../solr/cloud/api/collections/MigrateCmd.java     |  34 +--
 .../solr/cloud/api/collections/MoveReplicaCmd.java |  20 +-
 .../solr/cloud/api/collections/ReplaceNodeCmd.java |   2 +-
 .../solr/cloud/api/collections/RestoreCmd.java     |   2 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |  31 ++-
 .../solr/cloud/autoscaling/IndexSizeTrigger.java   |  91 ++++----
 .../solr/cloud/autoscaling/MetricTrigger.java      |   4 +-
 .../solr/cloud/autoscaling/SearchRateTrigger.java  |  55 +++--
 .../cloud/autoscaling/sim/SimCloudManager.java     |   7 +-
 .../autoscaling/sim/SimClusterStateProvider.java   | 240 +++++++++++----------
 .../autoscaling/sim/SimNodeStateProvider.java      |  22 +-
 .../solr/cloud/autoscaling/sim/SimScenario.java    |   8 +-
 .../solr/cloud/autoscaling/sim/SimUtils.java       |  25 ++-
 .../autoscaling/sim/SnapshotCloudManager.java      |   4 +-
 .../autoscaling/sim/SnapshotNodeStateProvider.java |  58 ++---
 .../apache/solr/cloud/overseer/NodeMutator.java    |   2 +-
 .../apache/solr/cloud/overseer/ReplicaMutator.java |   4 +-
 .../apache/solr/cloud/rule/ReplicaAssigner.java    |   4 +-
 .../src/java/org/apache/solr/core/CoreSorter.java  |   4 +-
 .../java/org/apache/solr/handler/IndexFetcher.java |   2 +-
 .../org/apache/solr/handler/SolrConfigHandler.java |   2 +-
 .../solr/handler/admin/CollectionsHandler.java     |   4 +-
 .../solr/handler/admin/MetricsHistoryHandler.java  |   5 +-
 .../org/apache/solr/schema/ManagedIndexSchema.java |   2 +-
 .../solr/search/join/ScoreJoinQParserPlugin.java   |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   4 +-
 .../processor/DistributedZkUpdateProcessor.java    |   6 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |   7 +-
 .../solr/cloud/ChaosMonkeyShardSplitTest.java      |   2 +-
 .../solr/cloud/ClusterStateMockUtilTest.java       |   2 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |   4 +-
 .../apache/solr/cloud/CreateRoutedAliasTest.java   |   2 +-
 .../test/org/apache/solr/cloud/DeleteNodeTest.java |   2 +-
 .../org/apache/solr/cloud/DeleteReplicaTest.java   |   6 +-
 .../DistribDocExpirationUpdateProcessorTest.java   |   4 +-
 .../solr/cloud/DistribJoinFromCollectionTest.java  |   2 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java     |   4 +-
 .../org/apache/solr/cloud/HttpPartitionTest.java   |   4 +-
 .../cloud/LeaderFailoverAfterPartitionTest.java    |   6 +-
 .../cloud/LeaderFailureAfterFreshStartTest.java    |   2 +-
 .../apache/solr/cloud/LeaderTragicEventTest.java   |   4 +-
 .../solr/cloud/LeaderVoteWaitTimeoutTest.java      |   6 +-
 .../solr/cloud/MoveReplicaHDFSFailoverTest.java    |   6 +-
 .../org/apache/solr/cloud/MoveReplicaTest.java     |  20 +-
 .../apache/solr/cloud/PeerSyncReplicationTest.java |   2 +-
 .../cloud/RoutingToNodesWithPropertiesTest.java    |   4 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java     |   6 +-
 .../test/org/apache/solr/cloud/SyncSliceTest.java  |   2 +-
 .../apache/solr/cloud/TestCloudConsistency.java    |   2 +-
 .../apache/solr/cloud/TestCloudDeleteByQuery.java  |  10 +-
 .../org/apache/solr/cloud/TestCloudRecovery2.java  |   2 +-
 .../solr/cloud/TestCloudSearcherWarming.java       |   4 +-
 .../org/apache/solr/cloud/TestPullReplica.java     |   4 +-
 .../apache/solr/cloud/TestRebalanceLeaders.java    |   4 +-
 .../solr/cloud/TestShortCircuitedRequests.java     |   2 +-
 .../solr/cloud/TestStressInPlaceUpdates.java       |   2 +-
 .../solr/cloud/TestTlogReplayVsRecovery.java       |   2 +-
 .../org/apache/solr/cloud/TestTlogReplica.java     |   6 +-
 .../cloud/TestTolerantUpdateProcessorCloud.java    |  10 +-
 .../org/apache/solr/cloud/TestUtilizeNode.java     |   2 +-
 .../org/apache/solr/cloud/TestWithCollection.java  |  30 +--
 .../AbstractCloudBackupRestoreTestCase.java        |   2 +-
 .../collections/CollectionTooManyReplicasTest.java |   4 +-
 .../CollectionsAPIAsyncDistributedZkTest.java      |   2 +-
 .../CollectionsAPIDistributedZkTest.java           |   2 +-
 .../api/collections/ReplicaPropertiesBase.java     |  10 +-
 .../solr/cloud/api/collections/ShardSplitTest.java |   2 +-
 .../cloud/api/collections/TestCollectionAPI.java   |   4 +-
 .../TestCollectionsAPIViaSolrCloudCluster.java     |   8 +-
 .../AutoAddReplicasIntegrationTest.java            |   2 +-
 .../cloud/autoscaling/AutoScalingHandlerTest.java  |   6 +-
 .../IndexSizeTriggerSizeEstimationTest.java        |   4 +-
 .../SearchRateTriggerIntegrationTest.java          |   4 +-
 .../cloud/autoscaling/SearchRateTriggerTest.java   |   2 +-
 .../cloud/autoscaling/SystemLogListenerTest.java   |   2 +-
 .../solr/cloud/autoscaling/TestPolicyCloud.java    |  30 +--
 .../autoscaling/sim/SimSolrCloudTestCase.java      |   2 +-
 .../cloud/autoscaling/sim/TestSimLargeCluster.java |   4 +-
 .../cloud/autoscaling/sim/TestSimPolicyCloud.java  |  20 +-
 .../autoscaling/sim/TestSimTriggerIntegration.java |   4 +-
 .../autoscaling/sim/TestSnapshotCloudManager.java  |   2 +-
 .../solr/cloud/cdcr/BaseCdcrDistributedZkTest.java |   4 +-
 .../org/apache/solr/cloud/cdcr/CdcrTestsUtil.java  |   2 +-
 .../org/apache/solr/cloud/rule/RuleEngineTest.java |   3 +-
 .../test/org/apache/solr/cloud/rule/RulesTest.java |   8 +-
 .../apache/solr/core/BlobRepositoryCloudTest.java  |   2 +-
 .../test/org/apache/solr/core/CoreSorterTest.java  |   4 +-
 .../admin/AutoscalingHistoryHandlerTest.java       |  10 +-
 .../solr/update/TestInPlaceUpdatesDistrib.java     |   4 +-
 .../solr/client/solrj/cloud/NodeStateProvider.java |   5 +-
 .../client/solrj/cloud/autoscaling/Clause.java     |   2 +-
 .../solrj/cloud/autoscaling/CoresVariable.java     |   6 +-
 .../autoscaling/DelegatingNodeStateProvider.java   |   3 +-
 .../solrj/cloud/autoscaling/FreeDiskVariable.java  |  29 +--
 .../cloud/autoscaling/MoveReplicaSuggester.java    |  11 +-
 .../solrj/cloud/autoscaling/PolicyHelper.java      |   8 +-
 .../solrj/cloud/autoscaling/ReplicaCount.java      |   6 +-
 .../{ReplicaInfo.java => ReplicaInfoXX.java}       |  24 +--
 .../solr/client/solrj/cloud/autoscaling/Row.java   |  46 ++--
 .../client/solrj/cloud/autoscaling/Suggester.java  |  12 +-
 .../client/solrj/cloud/autoscaling/Variable.java   |   9 +-
 .../client/solrj/cloud/autoscaling/Violation.java  |   5 +-
 .../cloud/autoscaling/WithCollectionVariable.java  |   6 +-
 .../client/solrj/impl/BaseCloudSolrClient.java     |   4 +-
 .../solrj/impl/SolrClientNodeStateProvider.java    |  30 +--
 .../solrj/io/stream/FeaturesSelectionStream.java   |   2 +-
 .../client/solrj/io/stream/TextLogitStream.java    |   2 +-
 .../solr/client/solrj/io/stream/TopicStream.java   |   6 +-
 .../solr/client/solrj/io/stream/TupleStream.java   |   2 +-
 .../routing/NodePreferenceRulesComparator.java     |   2 +-
 .../apache/solr/common/cloud/ClusterStateUtil.java |   8 +-
 .../apache/solr/common/cloud/DocCollection.java    |  10 +-
 .../java/org/apache/solr/common/cloud/Replica.java | 175 ++++++++++++---
 .../apache/solr/common/cloud/ZkStateReader.java    |   4 +-
 .../client/solrj/cloud/autoscaling/TestPolicy.java |  79 +++----
 .../solrj/cloud/autoscaling/TestPolicy2.java       |  13 +-
 .../client/solrj/io/stream/MathExpressionTest.java |   2 +-
 .../solrj/io/stream/StreamDecoratorTest.java       |   2 +-
 .../solr/client/solrj/request/TestV2Request.java   |   2 +-
 .../routing/NodePreferenceRulesComparatorTest.java |  20 +-
 ...RequestReplicaListTransformerGeneratorTest.java |  44 ++--
 .../solr/cloud/AbstractDistribZkTestBase.java      |   2 +-
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |  10 +-
 .../java/org/apache/solr/cloud/ChaosMonkey.java    |   2 +-
 134 files changed, 859 insertions(+), 740 deletions(-)

diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
index 4ce5eb6..59c3305 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/stream/AnalyticsShardRequestManager.java
@@ -114,7 +114,7 @@ public class AnalyticsShardRequestManager {
         Collection<Replica> replicas = slice.getReplicas();
         List<Replica> shuffler = new ArrayList<>();
         for(Replica replica : replicas) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode()))
           shuffler.add(replica);
         }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java b/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java
index 448f455..20f37ae 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ExclusiveSliceProperty.java
@@ -133,8 +133,8 @@ class ExclusiveSliceProperty {
           }
           continue;
         }
-        allHosts.add(replica.getNodeName());
-        String nodeName = replica.getNodeName();
+        allHosts.add(replica.getNode());
+        String nodeName = replica.getNode();
         if (StringUtils.isNotBlank(replica.getStr(property))) {
           if (sliceHasProp) {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
index f6c96ca..b7eddc6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -352,7 +352,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
     for (Replica replica : slices.getReplicas()) {
       if (replica.getName().equals(coreNodeName)) continue;
 
-      if (clusterState.getLiveNodes().contains(replica.getNodeName())) {
+      if (clusterState.getLiveNodes().contains(replica.getNode())) {
         long otherTerm = zkShardTerms.getTerm(replica.getName());
         boolean isOtherReplicaRecovering = zkShardTerms.isRecovering(replica.getName());
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index c50581c..cc44704 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -730,7 +730,7 @@ public class ZkController implements Closeable {
     int numActiveReplicas = shard.getReplicas(
         rep -> rep.getState() == Replica.State.ACTIVE
             && rep.getType() != Type.PULL
-            && getClusterState().getLiveNodes().contains(rep.getNodeName())
+            && getClusterState().getLiveNodes().contains(rep.getNode())
     ).size();
 
     // at least the leader still be able to search, we should give up leadership if other replicas can take over
@@ -1355,7 +1355,7 @@ public class ZkController implements Closeable {
 
     Replica replica = slice.getReplica(coreNodeName);
     if (replica == null) return null;
-    if (!getNodeName().equals(replica.getNodeName())) return null;
+    if (!getNodeName().equals(replica.getNode())) return null;
 
     return replica;
   }
@@ -1857,7 +1857,7 @@ public class ZkController implements Closeable {
       Replica replica = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName())
           .getSlice(cloudDesc.getShardId())
           .getReplica(cloudDesc.getCoreNodeName());
-      return !replica.getNodeName().equals(getNodeName());
+      return !replica.getNode().equals(getNodeName());
   }
 
   private void checkStateInZk(CoreDescriptor cd) throws InterruptedException, NotInClusterStateException {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
index 98a399a..debd65d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/Assign.java
@@ -421,7 +421,7 @@ public class Assign {
       for (Slice slice : c.getSlices()) {
         Collection<Replica> replicas = slice.getReplicas();
         for (Replica replica : replicas) {
-          ReplicaCount count = nodeNameVsShardCount.get(replica.getNodeName());
+          ReplicaCount count = nodeNameVsShardCount.get(replica.getNode());
           if (count != null) {
             count.totalNodes++; // Used to "weigh" whether this node should be used later.
             if (entry.getKey().equals(collectionName)) {
@@ -614,9 +614,9 @@ public class Assign {
           LinkedHashMap<String, Integer> n = new LinkedHashMap<>();
           shardVsNodes.put(slice.getName(), n);
           for (Replica replica : slice.getReplicas()) {
-            Integer count = n.get(replica.getNodeName());
+            Integer count = n.get(replica.getNode());
             if (count == null) count = 0;
-            n.put(replica.getNodeName(), ++count);
+            n.put(replica.getNode(), ++count);
           }
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
index 6ff3797..bd93539 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
@@ -226,7 +226,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
         params.set(CoreAdminParams.COMMIT_NAME, snapshotMeta.get().getName());
       }
 
-      shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(replica.getNode(), params, shardHandler);
       log.debug("Sent backup request to core={} for backupName={}", coreName, backupName);
     }
     log.debug("Sent backup requests to all shard leaders for backupName={}", backupName);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
index e3d8ab5..f0697fa 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateSnapshotCmd.java
@@ -116,7 +116,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
         params.set(CORE_NAME_PROP, coreName);
         params.set(CoreAdminParams.COMMIT_NAME, commitName);
 
-        shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+        shardRequestTracker.sendShardRequest(replica.getNode(), params, shardHandler);
         log.debug("Sent createsnapshot request to core={} with commitName={}", coreName, commitName);
 
         shardByCoreName.put(coreName, slice);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
index ff168c4..0696878 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteReplicaCmd.java
@@ -245,10 +245,10 @@ public class DeleteReplicaCmd implements Cmd {
     params.set(CoreAdminParams.DELETE_DATA_DIR, message.getBool(CoreAdminParams.DELETE_DATA_DIR, true));
     params.set(CoreAdminParams.DELETE_METRICS_HISTORY, message.getBool(CoreAdminParams.DELETE_METRICS_HISTORY, true));
 
-    boolean isLive = ocmh.zkStateReader.getClusterState().getLiveNodes().contains(replica.getNodeName());
+    boolean isLive = ocmh.zkStateReader.getClusterState().getLiveNodes().contains(replica.getNode());
     final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
     if (isLive) {
-      shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(replica.getNode(), params, shardHandler);
     }
 
     Callable<Boolean> callable = () -> {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
index ff7edfa..fb654eb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteShardCmd.java
@@ -165,7 +165,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
           SHARD_ID_PROP, slice.getName(),
           ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
           ZkStateReader.REPLICA_PROP, replica.getName(),
-          CoreAdminParams.NODE, replica.getNodeName());
+          CoreAdminParams.NODE, replica.getNode());
       sourceReplicas.add(props);
     }
     return sourceReplicas;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
index 9e4388b..7601258 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteSnapshotCmd.java
@@ -122,7 +122,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
           params.set(CoreAdminParams.COMMIT_NAME, commitName);
 
           log.info("Sending deletesnapshot request to core={} with commitName={}", coreName, commitName);
-          shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+          shardRequestTracker.sendShardRequest(replica.getNode(), params, shardHandler);
         }
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
index 85bac4b..4b7ea6c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MigrateCmd.java
@@ -180,7 +180,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     if (log.isInfoEnabled()) {
       log.info("Asking target leader node: {} core: {} to buffer updates"
-          , targetLeader.getNodeName(), targetLeader.getStr("core"));
+          , targetLeader.getNode(), targetLeader.getStr("core"));
     }
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTBUFFERUPDATES.toString());
@@ -188,7 +188,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     {
       final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
-      shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(targetLeader.getNode(), params, shardHandler);
 
       shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to request node to buffer updates");
     }
@@ -237,7 +237,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
         NRT_REPLICAS, 1,
         OverseerCollectionMessageHandler.NUM_SLICES, 1,
         CollectionAdminParams.COLL_CONF, configName,
-        OverseerCollectionMessageHandler.CREATE_NODE_SET, sourceLeader.getNodeName());
+        OverseerCollectionMessageHandler.CREATE_NODE_SET, sourceLeader.getNode());
     if (asyncId != null) {
       String internalAsyncId = asyncId + Math.abs(System.nanoTime());
       props.put(ASYNC, internalAsyncId);
@@ -252,14 +252,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     String tempCollectionReplica1 = tempSourceLeader.getCoreName();
     String coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
-        sourceLeader.getNodeName(), tempCollectionReplica1);
+        sourceLeader.getNode(), tempCollectionReplica1);
     // wait for the replicas to be seen as active on temp source leader
     if (log.isInfoEnabled()) {
-      log.info("Asking source leader to wait for: {} to be alive on: {}", tempCollectionReplica1, sourceLeader.getNodeName());
+      log.info("Asking source leader to wait for: {} to be alive on: {}", tempCollectionReplica1, sourceLeader.getNode());
     }
     CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
     cmd.setCoreName(tempCollectionReplica1);
-    cmd.setNodeName(sourceLeader.getNodeName());
+    cmd.setNodeName(sourceLeader.getNode());
     cmd.setCoreNodeName(coreNodeName);
     cmd.setState(Replica.State.ACTIVE);
     cmd.setCheckLive(true);
@@ -267,7 +267,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     {
       final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
       // we don't want this to happen asynchronously
-      syncRequestTracker.sendShardRequest(tempSourceLeader.getNodeName(), new ModifiableSolrParams(cmd.getParams()),
+      syncRequestTracker.sendShardRequest(tempSourceLeader.getNode(), new ModifiableSolrParams(cmd.getParams()),
           shardHandler);
 
       syncRequestTracker.processResponses(results, shardHandler, true,
@@ -283,7 +283,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     params.set(CoreAdminParams.RANGES, splitRange.toString());
     params.set("split.key", splitKey);
 
-    String tempNodeName = sourceLeader.getNodeName();
+    String tempNodeName = sourceLeader.getNode();
 
     {
       final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
@@ -292,7 +292,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     }
     if (log.isInfoEnabled()) {
       log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
-          tempSourceCollectionName, targetLeader.getNodeName());
+          tempSourceCollectionName, targetLeader.getNode());
     }
     String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
         zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
@@ -300,7 +300,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
     props.put(COLLECTION_PROP, tempSourceCollectionName);
     props.put(SHARD_ID_PROP, tempSourceSlice.getName());
-    props.put("node", targetLeader.getNodeName());
+    props.put("node", targetLeader.getNode());
     props.put(CoreAdminParams.NAME, tempCollectionReplica2);
     // copy over property params:
     for (String key : message.keySet()) {
@@ -320,14 +320,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
         "temporary collection in target leader node.");
     }
     coreNodeName = ocmh.waitForCoreNodeName(tempSourceCollectionName,
-        targetLeader.getNodeName(), tempCollectionReplica2);
+        targetLeader.getNode(), tempCollectionReplica2);
     // wait for the replicas to be seen as active on temp source leader
     if (log.isInfoEnabled()) {
-      log.info("Asking temp source leader to wait for: {} to be alive on: {}", tempCollectionReplica2, targetLeader.getNodeName());
+      log.info("Asking temp source leader to wait for: {} to be alive on: {}", tempCollectionReplica2, targetLeader.getNode());
     }
     cmd = new CoreAdminRequest.WaitForState();
     cmd.setCoreName(tempSourceLeader.getStr("core"));
-    cmd.setNodeName(targetLeader.getNodeName());
+    cmd.setNodeName(targetLeader.getNode());
     cmd.setCoreNodeName(coreNodeName);
     cmd.setState(Replica.State.ACTIVE);
     cmd.setCheckLive(true);
@@ -336,7 +336,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     {
       final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
-      shardRequestTracker.sendShardRequest(tempSourceLeader.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(tempSourceLeader.getNode(), params, shardHandler);
 
       shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to create temp collection" +
         " replica or timed out waiting for them to come up");
@@ -352,9 +352,9 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     {
       final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
     
-      shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(targetLeader.getNode(), params, shardHandler);
     String msg = "MIGRATE failed to merge " + tempCollectionReplica2 + " to "
-        + targetLeader.getStr("core") + " on node: " + targetLeader.getNodeName();
+        + targetLeader.getStr("core") + " on node: " + targetLeader.getNode();
     shardRequestTracker.processResponses(results, shardHandler, true, msg);
     }
     log.info("Asking target leader to apply buffered updates");
@@ -364,7 +364,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
     {
       final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
-      shardRequestTracker.sendShardRequest(targetLeader.getNodeName(), params, shardHandler);
+      shardRequestTracker.sendShardRequest(targetLeader.getNode(), params, shardHandler);
       shardRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to request node to apply buffered updates");
     }
     try {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
index f567b2e..4d886f1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MoveReplicaCmd.java
@@ -117,7 +117,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'" + SHARD_ID_PROP + "' is a required param");
       }
       Slice slice = coll.getSlice(shardId);
-      List<Replica> sliceReplicas = new ArrayList<>(slice.getReplicas(r -> sourceNode.equals(r.getNodeName())));
+      List<Replica> sliceReplicas = new ArrayList<>(slice.getReplicas(r -> sourceNode.equals(r.getNode())));
       if (sliceReplicas.isEmpty()) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
             "Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
@@ -128,18 +128,18 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
     if (coll.getStr(CollectionAdminParams.COLOCATED_WITH) != null) {
       // we must ensure that moving this replica does not cause the co-location to break
-      String sourceNode = replica.getNodeName();
+      String sourceNode = replica.getNode();
       String colocatedCollectionName = coll.getStr(CollectionAdminParams.COLOCATED_WITH);
       DocCollection colocatedCollection = clusterState.getCollectionOrNull(colocatedCollectionName);
       if (colocatedCollection != null) {
-        if (colocatedCollection.getReplica((s, r) -> sourceNode.equals(r.getNodeName())) != null) {
+        if (colocatedCollection.getReplica((s, r) -> sourceNode.equals(r.getNode())) != null) {
           // check if we have at least two replicas of the collection on the source node
           // only then it is okay to move one out to another node
-          List<Replica> replicasOnSourceNode = coll.getReplicas(replica.getNodeName());
+          List<Replica> replicasOnSourceNode = coll.getReplicas(replica.getNode());
           if (replicasOnSourceNode == null || replicasOnSourceNode.size() < 2) {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
                 "Collection: " + collection + " is co-located with collection: " + colocatedCollectionName
-                    + " and has a single replica: " + replica.getName() + " on node: " + replica.getNodeName()
+                    + " and has a single replica: " + replica.getName() + " on node: " + replica.getNode()
                     + " so it is not possible to move it to another node");
           }
         }
@@ -170,7 +170,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
   private void moveHdfsReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String dataDir, String targetNode, String async,
                                  DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
     String skipCreateReplicaInClusterState = "true";
-    if (clusterState.getLiveNodes().contains(replica.getNodeName())) {
+    if (clusterState.getLiveNodes().contains(replica.getNode())) {
       skipCreateReplicaInClusterState = "false";
       ZkNodeProps removeReplicasProps = new ZkNodeProps(
           COLLECTION_PROP, coll.getName(),
@@ -236,7 +236,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
           " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
       results.add("failure", errorString);
       log.warn("Error adding replica {} - trying to roll back...",  addReplicasProps, e);
-      addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
+      addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNode());
       @SuppressWarnings({"rawtypes"})
       NamedList rollback = new NamedList();
       ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
@@ -253,7 +253,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       results.add("failure", errorString);
       log.debug("--- trying to roll back...");
       // try to roll back
-      addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
+      addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNode());
       @SuppressWarnings({"rawtypes"})
       NamedList rollback = new NamedList();
       try {
@@ -269,7 +269,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       return;
     } else {
       String successString = String.format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " +
-          "to replica=%s at node=%s", replica.getCoreName(), replica.getNodeName(), replica.getCoreName(), targetNode);
+          "to replica=%s at node=%s", replica.getCoreName(), replica.getNode(), replica.getCoreName(), targetNode);
       results.add("success", successString);
     }
   }
@@ -346,7 +346,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       results.add("failure", errorString);
     } else {
       String successString = String.format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " +
-          "to replica=%s at node=%s", replica.getCoreName(), replica.getNodeName(), newCoreName, targetNode);
+          "to replica=%s at node=%s", replica.getCoreName(), replica.getNode(), newCoreName, targetNode);
       results.add("success", successString);
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
index f1c1f8c..77a0f5c 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReplaceNodeCmd.java
@@ -242,7 +242,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
     for (Map.Entry<String, DocCollection> e : state.getCollectionsMap().entrySet()) {
       for (Slice slice : e.getValue().getSlices()) {
         for (Replica replica : slice.getReplicas()) {
-          if (source.equals(replica.getNodeName())) {
+          if (source.equals(replica.getNode())) {
             ZkNodeProps props = new ZkNodeProps(
                 COLLECTION_PROP, e.getKey(),
                 SHARD_ID_PROP, slice.getName(),
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index f800979..3923e59 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -330,7 +330,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
 
         for (Slice s : restoreCollection.getSlices()) {
           for (Replica r : s.getReplicas()) {
-            String nodeName = r.getNodeName();
+            String nodeName = r.getNode();
             String coreNodeName = r.getCoreName();
             Replica.State stateRep = r.getState();
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
index 3c1480f..e14c5ef 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SplitShardCmd.java
@@ -35,7 +35,6 @@ import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
 import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
@@ -158,10 +157,10 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
     t.stop();
 
     // let's record the ephemeralOwner of the parent leader node
-    Stat leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null, true);
+    Stat leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNode(), null, true);
     if (leaderZnodeStat == null)  {
       // we just got to know the leader but its live node is gone already!
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNode() + " is not live anymore!");
     }
 
     List<DocRouter.Range> subRanges = new ArrayList<>();
@@ -225,7 +224,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
         {
           final ShardRequestTracker shardRequestTracker = ocmh.syncRequestTracker();
-          shardRequestTracker.sendShardRequest(parentShardLeader.getNodeName(), params, shardHandler);
+          shardRequestTracker.sendShardRequest(parentShardLeader.getNode(), params, shardHandler);
           SimpleOrderedMap<Object> getRangesResults = new SimpleOrderedMap<>();
           String msgOnError = "SPLITSHARD failed to invoke SPLIT.getRanges core admin command";
           shardRequestTracker.processResponses(getRangesResults, shardHandler, true, msgOnError);
@@ -288,7 +287,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         collection = clusterState.getCollection(collectionName);
       }
 
-      String nodeName = parentShardLeader.getNodeName();
+      String nodeName = parentShardLeader.getNode();
 
       t = timings.sub("createSubSlicesAndLeadersInState");
       for (int i = 0; i < subRanges.size(); i++) {
@@ -390,7 +389,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       t = timings.sub("splitParentCore");
       {
         final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId);
-        shardRequestTracker.sendShardRequest(parentShardLeader.getNodeName(), params, shardHandler);
+        shardRequestTracker.sendShardRequest(parentShardLeader.getNode(), params, shardHandler);
 
         String msgOnError = "SPLITSHARD failed to invoke SPLIT core admin command";
         shardRequestTracker.processResponses(results, shardHandler, true, msgOnError);
@@ -515,7 +514,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
       long ephemeralOwner = leaderZnodeStat.getEphemeralOwner();
       // compare against the ephemeralOwner of the parent leader node
-      leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null, true);
+      leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNode(), null, true);
       if (leaderZnodeStat == null || ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
         // put sub-shards in recovery_failed state
 
@@ -530,11 +529,11 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
         if (leaderZnodeStat == null)  {
           // the leader is not live anymore, fail the split!
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNodeName() + " is not live anymore!");
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard leader node: " + parentShardLeader.getNode() + " is not live anymore!");
         } else if (ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
           // there's a new leader, fail the split!
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              "The zk session id for the shard leader node: " + parentShardLeader.getNodeName() + " has changed from "
+              "The zk session id for the shard leader node: " + parentShardLeader.getNode() + " has changed from "
                   + ephemeralOwner + " to " + leaderZnodeStat.getEphemeralOwner() + ". This can cause data loss so we must abort the split");
         }
       }
@@ -636,19 +635,19 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
     // check that enough disk space is available on the parent leader node
     // otherwise the actual index splitting will always fail
     NodeStateProvider nodeStateProvider = cloudManager.getNodeStateProvider();
-    Map<String, Object> nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNodeName(),
+    Map<String, Object> nodeValues = nodeStateProvider.getNodeValues(parentShardLeader.getNode(),
         Collections.singletonList(ImplicitSnitch.DISK));
-    Map<String, Map<String, List<ReplicaInfo>>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNodeName(),
+    Map<String, Map<String, List<Replica>>> infos = nodeStateProvider.getReplicaInfo(parentShardLeader.getNode(),
         Collections.singletonList(Type.CORE_IDX.metricsAttribute));
     if (infos.get(collection) == null || infos.get(collection).get(shard) == null) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing replica information for parent shard leader");
     }
     // find the leader
-    List<ReplicaInfo> lst = infos.get(collection).get(shard);
+    List<Replica> lst = infos.get(collection).get(shard);
     Double indexSize = null;
-    for (ReplicaInfo info : lst) {
-      if (info.getCore().equals(parentShardLeader.getCoreName())) {
-        Number size = (Number)info.getVariable(Type.CORE_IDX.metricsAttribute);
+    for (Replica info : lst) {
+      if (info.getCoreName().equals(parentShardLeader.getCoreName())) {
+        Number size = (Number)info.get(Type.CORE_IDX.metricsAttribute);
         if (size == null) {
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "missing index size information for parent shard leader");
         }
@@ -667,7 +666,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
     double neededSpace = method == SolrIndexSplitter.SplitMethod.REWRITE ? 2.0 * indexSize : 1.05 * indexSize;
     if (freeSize.doubleValue() < neededSpace) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "not enough free disk space to perform index split on node " +
-          parentShardLeader.getNodeName() + ", required: " + neededSpace + ", available: " + freeSize);
+          parentShardLeader.getNode() + ", required: " + neededSpace + ", available: " + freeSize);
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
index da40366..b276036 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/IndexSizeTrigger.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.Locale;
 
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
@@ -263,14 +262,14 @@ public class IndexSizeTrigger extends TriggerBase {
     }
 
     // replica name / info + size, retrieved from leaders only
-    Map<String, ReplicaInfo> currentSizes = new HashMap<>();
+    Map<String, Replica> currentSizes = new HashMap<>();
 
     try {
       ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
       for (String node : clusterState.getLiveNodes()) {
-        Map<String, ReplicaInfo> metricTags = new HashMap<>();
+        Map<String, Replica> metricTags = new HashMap<>();
         // coll, shard, replica
-        Map<String, Map<String, List<ReplicaInfo>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
+        Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
         infos.forEach((coll, shards) -> {
           if (!collections.isEmpty() && !collections.contains(coll)) {
             return;
@@ -289,13 +288,13 @@ public class IndexSizeTrigger extends TriggerBase {
               return;
             }
             // not on this node
-            if (!r.getNodeName().equals(node)) {
+            if (!r.getNode().equals(node)) {
               return;
             }
             // find ReplicaInfo
-            ReplicaInfo info = null;
-            for (ReplicaInfo ri : replicas) {
-              if (r.getCoreName().equals(ri.getCore())) {
+            Replica info = null;
+            for (Replica ri : replicas) {
+              if (r.getCoreName().equals(ri.getCoreName())) {
                 info = ri;
                 break;
               }
@@ -305,7 +304,7 @@ public class IndexSizeTrigger extends TriggerBase {
               return;
             }
             // we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
-            String replicaName = Utils.parseMetricsReplicaName(coll, info.getCore());
+            String replicaName = Utils.parseMetricsReplicaName(coll, info.getCoreName());
             if (replicaName == null) { // should never happen???
               replicaName = info.getName(); // which is actually coreNode name...
             }
@@ -325,7 +324,7 @@ public class IndexSizeTrigger extends TriggerBase {
         }
         Map<String, Object> sizes = cloudManager.getNodeStateProvider().getNodeValues(node, metricTags.keySet());
         sizes.forEach((tag, size) -> {
-          final ReplicaInfo info = metricTags.get(tag);
+          final Replica info = metricTags.get(tag);
           if (info == null) {
             log.warn("Missing replica info for response tag {}", tag);
           } else {
@@ -335,15 +334,15 @@ public class IndexSizeTrigger extends TriggerBase {
               return;
             }
 
-            ReplicaInfo currentInfo = currentSizes.computeIfAbsent(info.getCore(), k -> (ReplicaInfo)info.clone());
+            Replica currentInfo = currentSizes.computeIfAbsent(info.getCoreName(), k -> (Replica) info.clone());
             if (tag.contains("INDEX")) {
-              currentInfo.getVariables().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
+              currentInfo.getProperties().put(TOTAL_BYTES_SIZE_KEY, ((Number) size).longValue());
             } else if (tag.endsWith("SEARCHER.searcher.numDocs")) {
-              currentInfo.getVariables().put(DOCS_SIZE_KEY, ((Number) size).longValue());
+              currentInfo.getProperties().put(DOCS_SIZE_KEY, ((Number) size).longValue());
             } else if (tag.endsWith("SEARCHER.searcher.maxDoc")) {
-              currentInfo.getVariables().put(MAX_DOC_KEY, ((Number) size).longValue());
+              currentInfo.getProperties().put(MAX_DOC_KEY, ((Number) size).longValue());
             } else if (tag.endsWith("SEARCHER.searcher.indexCommitSize")) {
-              currentInfo.getVariables().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
+              currentInfo.getProperties().put(COMMIT_SIZE_KEY, ((Number) size).longValue());
             }
           }
         });
@@ -358,31 +357,31 @@ public class IndexSizeTrigger extends TriggerBase {
     // now check thresholds
 
     // collection / list(info)
-    Map<String, List<ReplicaInfo>> aboveSize = new HashMap<>();
+    Map<String, List<Replica>> aboveSize = new HashMap<>();
 
     Set<String> splittable = new HashSet<>();
 
     currentSizes.forEach((coreName, info) -> {
       // calculate estimated bytes
-      long maxDoc = (Long)info.getVariable(MAX_DOC_KEY);
-      long numDocs = (Long)info.getVariable(DOCS_SIZE_KEY);
-      long commitSize = (Long)info.getVariable(COMMIT_SIZE_KEY, 0L);
+      long maxDoc = (Long)info.get(MAX_DOC_KEY);
+      long numDocs = (Long)info.get(DOCS_SIZE_KEY);
+      long commitSize = (Long)info.get(COMMIT_SIZE_KEY, 0L);
       if (commitSize <= 0) {
-        commitSize = (Long)info.getVariable(TOTAL_BYTES_SIZE_KEY);
+        commitSize = (Long)info.get(TOTAL_BYTES_SIZE_KEY);
       }
       // calculate estimated size as a side-effect
       commitSize = estimatedSize(maxDoc, numDocs, commitSize);
-      info.getVariables().put(BYTES_SIZE_KEY, commitSize);
+      info.getProperties().put(BYTES_SIZE_KEY, commitSize);
 
-      if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes ||
-          (Long)info.getVariable(DOCS_SIZE_KEY) > aboveDocs) {
+      if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes ||
+          (Long)info.get(DOCS_SIZE_KEY) > aboveDocs) {
         if (waitForElapsed(coreName, now, lastAboveEventMap)) {
-          List<ReplicaInfo> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
+          List<Replica> infos = aboveSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
           if (!infos.contains(info)) {
-            if ((Long)info.getVariable(BYTES_SIZE_KEY) > aboveBytes) {
-              info.getVariables().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
+            if ((Long)info.get(BYTES_SIZE_KEY) > aboveBytes) {
+              info.getProperties().put(VIOLATION_KEY, ABOVE_BYTES_PROP);
             } else {
-              info.getVariables().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
+              info.getProperties().put(VIOLATION_KEY, ABOVE_DOCS_PROP);
             }
             infos.add(info);
             splittable.add(info.getName());
@@ -395,20 +394,20 @@ public class IndexSizeTrigger extends TriggerBase {
     });
 
     // collection / list(info)
-    Map<String, List<ReplicaInfo>> belowSize = new HashMap<>();
+    Map<String, List<Replica>> belowSize = new HashMap<>();
 
     currentSizes.forEach((coreName, info) -> {
-      if (((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes ||
-          (Long)info.getVariable(DOCS_SIZE_KEY) < belowDocs) &&
+      if (((Long)info.get(BYTES_SIZE_KEY) < belowBytes ||
+          (Long)info.get(DOCS_SIZE_KEY) < belowDocs) &&
           // make sure we don't produce conflicting ops
           !splittable.contains(info.getName())) {
         if (waitForElapsed(coreName, now, lastBelowEventMap)) {
-          List<ReplicaInfo> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
+          List<Replica> infos = belowSize.computeIfAbsent(info.getCollection(), c -> new ArrayList<>());
           if (!infos.contains(info)) {
-            if ((Long)info.getVariable(BYTES_SIZE_KEY) < belowBytes) {
-              info.getVariables().put(VIOLATION_KEY, BELOW_BYTES_PROP);
+            if ((Long)info.get(BYTES_SIZE_KEY) < belowBytes) {
+              info.getProperties().put(VIOLATION_KEY, BELOW_BYTES_PROP);
             } else {
-              info.getVariables().put(VIOLATION_KEY, BELOW_DOCS_PROP);
+              info.getProperties().put(VIOLATION_KEY, BELOW_DOCS_PROP);
             }
             infos.add(info);
           }
@@ -435,7 +434,7 @@ public class IndexSizeTrigger extends TriggerBase {
       // sort by decreasing size to first split the largest ones
       // XXX see the comment below about using DOCS_SIZE_PROP in lieu of BYTES_SIZE_PROP
       replicas.sort((r1, r2) -> {
-        long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
+        long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
         if (delta > 0) {
           return -1;
         } else if (delta < 0) {
@@ -458,7 +457,7 @@ public class IndexSizeTrigger extends TriggerBase {
         params.put(SPLIT_BY_PREFIX, splitByPrefix);
         op.addHint(Suggester.Hint.PARAMS, params);
         ops.add(op);
-        Long time = lastAboveEventMap.get(r.getCore());
+        Long time = lastAboveEventMap.get(r.getCoreName());
         if (time != null && eventTime.get() > time) {
           eventTime.set(time);
         }
@@ -477,7 +476,7 @@ public class IndexSizeTrigger extends TriggerBase {
         // then we should be sorting by BYTES_SIZE_PROP. However, since DOCS and BYTES are
         // loosely correlated it's simpler to sort just by docs (which better reflects the "too small"
         // condition than index size, due to possibly existing deleted docs that still occupy space)
-        long delta = (Long) r1.getVariable(DOCS_SIZE_KEY) - (Long) r2.getVariable(DOCS_SIZE_KEY);
+        long delta = (Long) r1.get(DOCS_SIZE_KEY) - (Long) r2.get(DOCS_SIZE_KEY);
         if (delta > 0) {
           return 1;
         } else if (delta < 0) {
@@ -495,11 +494,11 @@ public class IndexSizeTrigger extends TriggerBase {
       op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(0).getShard()));
       op.addHint(Suggester.Hint.COLL_SHARD, new Pair(coll, replicas.get(1).getShard()));
       ops.add(op);
-      Long time = lastBelowEventMap.get(replicas.get(0).getCore());
+      Long time = lastBelowEventMap.get(replicas.get(0).getCoreName());
       if (time != null && eventTime.get() > time) {
         eventTime.set(time);
       }
-      time = lastBelowEventMap.get(replicas.get(1).getCore());
+      time = lastBelowEventMap.get(replicas.get(1).getCoreName());
       if (time != null && eventTime.get() > time) {
         eventTime.set(time);
       }
@@ -511,14 +510,14 @@ public class IndexSizeTrigger extends TriggerBase {
     if (processor.process(new IndexSizeEvent(getName(), eventTime.get(), ops, aboveSize, belowSize))) {
       // update last event times
       aboveSize.forEach((coll, replicas) -> {
-        replicas.forEach(r -> lastAboveEventMap.put(r.getCore(), now));
+        replicas.forEach(r -> lastAboveEventMap.put(r.getCoreName(), now));
       });
       belowSize.forEach((coll, replicas) -> {
         if (replicas.size() < 2) {
           return;
         }
-        lastBelowEventMap.put(replicas.get(0).getCore(), now);
-        lastBelowEventMap.put(replicas.get(1).getCore(), now);
+        lastBelowEventMap.put(replicas.get(0).getCoreName(), now);
+        lastBelowEventMap.put(replicas.get(1).getCoreName(), now);
       });
     }
   }
@@ -544,18 +543,18 @@ public class IndexSizeTrigger extends TriggerBase {
   }
 
   public static class IndexSizeEvent extends TriggerEvent {
-    public IndexSizeEvent(String source, long eventTime, List<Op> ops, Map<String, List<ReplicaInfo>> aboveSize,
-                          Map<String, List<ReplicaInfo>> belowSize) {
+    public IndexSizeEvent(String source, long eventTime, List<Op> ops, Map<String, List<Replica>> aboveSize,
+                          Map<String, List<Replica>> belowSize) {
       super(TriggerEventType.INDEXSIZE, source, eventTime, null);
       properties.put(TriggerEvent.REQUESTED_OPS, ops);
       // avoid passing very large amounts of data here - just use replica names
       TreeMap<String, String> above = new TreeMap<>();
       aboveSize.forEach((coll, replicas) ->
-          replicas.forEach(r -> above.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
+          replicas.forEach(r -> above.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
       properties.put(ABOVE_SIZE_KEY, above);
       TreeMap<String, String> below = new TreeMap<>();
       belowSize.forEach((coll, replicas) ->
-          replicas.forEach(r -> below.put(r.getCore(), "docs=" + r.getVariable(DOCS_SIZE_KEY) + ", bytes=" + r.getVariable(BYTES_SIZE_KEY))));
+          replicas.forEach(r -> below.put(r.getCoreName(), "docs=" + r.get(DOCS_SIZE_KEY) + ", bytes=" + r.get(BYTES_SIZE_KEY))));
       properties.put(BELOW_SIZE_KEY, below);
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
index 573ac77..0b9b055 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/MetricTrigger.java
@@ -133,7 +133,7 @@ public class MetricTrigger extends TriggerBase {
         }
         if (shard.equals(Policy.ANY)) {
           docCollection.getReplicas().forEach(replica -> {
-            nodes.add(replica.getNodeName());
+            nodes.add(replica.getNode());
           });
         } else {
           Slice slice = docCollection.getSlice(shard);
@@ -141,7 +141,7 @@ public class MetricTrigger extends TriggerBase {
             log.warn("MetricTrigger could not find collection: {} shard: {}", collection, shard);
             return;
           }
-          slice.getReplicas().forEach(replica -> nodes.add(replica.getNodeName()));
+          slice.getReplicas().forEach(replica -> nodes.add(replica.getNode()));
         }
         liveNodes = nodes;
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
index efd5b24..505c511 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/SearchRateTrigger.java
@@ -33,7 +33,6 @@ import java.util.concurrent.atomic.AtomicLong;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.AtomicDouble;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
@@ -331,7 +330,7 @@ public class SearchRateTrigger extends TriggerBase {
     }
 
     // collection, shard, list(replica + rate)
-    Map<String, Map<String, List<ReplicaInfo>>> collectionRates = new HashMap<>();
+    Map<String, Map<String, List<Replica>>> collectionRates = new HashMap<>();
     // node, rate
     Map<String, AtomicDouble> nodeRates = new HashMap<>();
     // this replication factor only considers replica types that are searchable
@@ -346,9 +345,9 @@ public class SearchRateTrigger extends TriggerBase {
       return;
     }
     for (String node : cloudManager.getClusterStateProvider().getLiveNodes()) {
-      Map<String, ReplicaInfo> metricTags = new HashMap<>();
+      Map<String, Replica> metricTags = new HashMap<>();
       // coll, shard, replica
-      Map<String, Map<String, List<ReplicaInfo>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
+      Map<String, Map<String, List<Replica>>> infos = cloudManager.getNodeStateProvider().getReplicaInfo(node, Collections.emptyList());
       infos.forEach((coll, shards) -> {
         Map<String, AtomicInteger> replPerShard = searchableReplicationFactors.computeIfAbsent(coll, c -> new HashMap<>());
         shards.forEach((sh, replicas) -> {
@@ -360,7 +359,7 @@ public class SearchRateTrigger extends TriggerBase {
             }
             repl.incrementAndGet();
             // we have to translate to the metrics registry name, which uses "_replica_nN" as suffix
-            String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCore());
+            String replicaName = Utils.parseMetricsReplicaName(coll, replica.getCoreName());
             if (replicaName == null) { // should never happen???
               replicaName = replica.getName(); // which is actually coreNode name...
             }
@@ -379,14 +378,14 @@ public class SearchRateTrigger extends TriggerBase {
         rates.forEach((tag, rate) -> log.debug("###  " + tag + "\t" + rate)); // logOk
       }
       rates.forEach((tag, rate) -> {
-        ReplicaInfo info = metricTags.get(tag);
+        Replica info = metricTags.get(tag);
         if (info == null) {
           log.warn("Missing replica info for response tag {}", tag);
         } else {
-          Map<String, List<ReplicaInfo>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
-          List<ReplicaInfo> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
-          info = (ReplicaInfo)info.clone();
-          info.getVariables().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
+          Map<String, List<Replica>> perCollection = collectionRates.computeIfAbsent(info.getCollection(), s -> new HashMap<>());
+          List<Replica> perShard = perCollection.computeIfAbsent(info.getShard(), s -> new ArrayList<>());
+          info = (Replica)info.clone();
+          info.getProperties().put(AutoScalingParams.RATE, ((Number)rate).doubleValue());
           perShard.add(info);
           AtomicDouble perNode = nodeRates.computeIfAbsent(node, s -> new AtomicDouble());
           perNode.addAndGet(((Number)rate).doubleValue());
@@ -399,7 +398,7 @@ public class SearchRateTrigger extends TriggerBase {
         log.debug("## Collection: {}", coll);
         collRates.forEach((s, replicas) -> {
           log.debug("##  - {}", s);
-          replicas.forEach(ri -> log.debug("##     {}  {}", ri.getCore(), ri.getVariable(AutoScalingParams.RATE))); //logOk
+          replicas.forEach(ri -> log.debug("##     {}  {}", ri.getCoreName(), ri.get(AutoScalingParams.RATE))); //logOk
         });
       });
     }
@@ -428,18 +427,18 @@ public class SearchRateTrigger extends TriggerBase {
 
     Map<String, Map<String, Double>> hotShards = new HashMap<>();
     Map<String, Map<String, Double>> coldShards = new HashMap<>();
-    List<ReplicaInfo> hotReplicas = new ArrayList<>();
-    List<ReplicaInfo> coldReplicas = new ArrayList<>();
+    List<Replica> hotReplicas = new ArrayList<>();
+    List<Replica> coldReplicas = new ArrayList<>();
     collectionRates.forEach((coll, shardRates) -> {
       shardRates.forEach((sh, replicaRates) -> {
         double totalShardRate = replicaRates.stream()
             .map(r -> {
-              String elapsedKey = r.getCollection() + "." + r.getCore();
-              if ((Double)r.getVariable(AutoScalingParams.RATE) > aboveRate) {
+              String elapsedKey = r.getCollection() + "." + r.getCoreName();
+              if ((Double)r.get(AutoScalingParams.RATE) > aboveRate) {
                 if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
                   hotReplicas.add(r);
                 }
-              } else if ((Double)r.getVariable(AutoScalingParams.RATE) < belowRate) {
+              } else if ((Double)r.get(AutoScalingParams.RATE) < belowRate) {
                 if (waitForElapsed(elapsedKey, now, lastReplicaEvent)) {
                   coldReplicas.add(r);
                 }
@@ -449,7 +448,7 @@ public class SearchRateTrigger extends TriggerBase {
               }
               return r;
             })
-            .mapToDouble(r -> (Double)r.getVariable(AutoScalingParams.RATE)).sum();
+            .mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum();
         // calculate average shard rate over all searchable replicas (see SOLR-12470)
         double shardRate = totalShardRate / searchableReplicationFactors.get(coll).get(sh).doubleValue();
         String elapsedKey = coll + "." + sh;
@@ -486,7 +485,7 @@ public class SearchRateTrigger extends TriggerBase {
     collectionRates.forEach((coll, shardRates) -> {
       double total = shardRates.entrySet().stream()
           .mapToDouble(e -> e.getValue().stream()
-              .mapToDouble(r -> (Double)r.getVariable(AutoScalingParams.RATE)).sum()).sum();
+              .mapToDouble(r -> (Double)r.get(AutoScalingParams.RATE)).sum()).sum();
       if (collections.isEmpty() || collections.contains(coll)) {
         if (total > aboveRate) {
           if (waitForElapsed(coll, now, lastCollectionEvent)) {
@@ -547,13 +546,13 @@ public class SearchRateTrigger extends TriggerBase {
       });
     });
     hotReplicas.forEach(r -> {
-      long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCore());
+      long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
       if (eventTime.get() > time) {
         eventTime.set(time);
       }
     });
     coldReplicas.forEach(r -> {
-      long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCore());
+      long time = lastReplicaEvent.get(r.getCollection() + "." + r.getCoreName());
       if (eventTime.get() > time) {
         eventTime.set(time);
       }
@@ -593,8 +592,8 @@ public class SearchRateTrigger extends TriggerBase {
           .forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
       coldShards.entrySet().forEach(e -> e.getValue()
           .forEach((sh, rate) -> lastShardEvent.put(e.getKey() + "." + sh, now)));
-      hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCore(), now));
-      coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCore(), now));
+      hotReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
+      coldReplicas.forEach(r -> lastReplicaEvent.put(r.getCollection() + "." + r.getCoreName(), now));
     }
   }
 
@@ -604,7 +603,7 @@ public class SearchRateTrigger extends TriggerBase {
                                Map<String, Double> hotNodes,
                                Map<String, Double> hotCollections,
                                Map<String, Map<String, Double>> hotShards,
-                               List<ReplicaInfo> hotReplicas) {
+                               List<Replica> hotReplicas) {
     // calculate the number of replicas to add to each hot shard, based on how much the rate was
     // exceeded - but within limits.
 
@@ -678,7 +677,7 @@ public class SearchRateTrigger extends TriggerBase {
                                 Map<String, Double> coldNodes,
                                 Map<String, Double> coldCollections,
                                 Map<String, Map<String, Double>> coldShards,
-                                List<ReplicaInfo> coldReplicas) {
+                                List<Replica> coldReplicas) {
     // COLD COLLECTIONS
     // Probably can't do anything reasonable about whole cold collections
     // because they may be needed even if not used.
@@ -694,7 +693,7 @@ public class SearchRateTrigger extends TriggerBase {
     // replicas still available (additional non-searchable replicas may exist, too)
     // NOTE: do this before adding ops for DELETENODE because we don't want to attempt
     // deleting replicas that have been already moved elsewhere
-    Map<String, Map<String, List<ReplicaInfo>>> byCollectionByShard = new HashMap<>();
+    Map<String, Map<String, List<Replica>>> byCollectionByShard = new HashMap<>();
     coldReplicas.forEach(ri -> {
       byCollectionByShard.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
           .computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
@@ -702,7 +701,7 @@ public class SearchRateTrigger extends TriggerBase {
     });
     coldShards.forEach((coll, perShard) -> {
       perShard.forEach((shard, rate) -> {
-        List<ReplicaInfo> replicas = byCollectionByShard
+        List<Replica> replicas = byCollectionByShard
             .getOrDefault(coll, Collections.emptyMap())
             .getOrDefault(shard, Collections.emptyList());
         if (replicas.isEmpty()) {
@@ -784,11 +783,11 @@ public class SearchRateTrigger extends TriggerBase {
                            Map<String, Double> hotNodes,
                            Map<String, Double> hotCollections,
                            Map<String, Map<String, Double>> hotShards,
-                           List<ReplicaInfo> hotReplicas,
+                           List<Replica> hotReplicas,
                            Map<String, Double> coldNodes,
                            Map<String, Double> coldCollections,
                            Map<String, Map<String, Double>> coldShards,
-                           List<ReplicaInfo> coldReplicas,
+                           List<Replica> coldReplicas,
                            Set<String> violations) {
       super(TriggerEventType.SEARCHRATE, source, eventTime, null);
       properties.put(TriggerEvent.REQUESTED_OPS, ops);
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
index 25624f4..b8ac313 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimCloudManager.java
@@ -53,7 +53,6 @@ import org.apache.solr.client.solrj.cloud.DistributedQueueFactory;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
@@ -320,7 +319,7 @@ public class SimCloudManager implements SolrCloudManager {
     for (String node : other.getClusterStateProvider().getLiveNodes()) {
       SimClusterStateProvider simClusterStateProvider = cloudManager.getSimClusterStateProvider();
       cloudManager.getSimNodeStateProvider().simSetNodeValues(node, other.getNodeStateProvider().getNodeValues(node, nodeTags));
-      Map<String, Map<String, List<ReplicaInfo>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
+      Map<String, Map<String, List<Replica>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
       simClusterStateProvider.simSetReplicaValues(node, infos, true);
     }
     SimUtils.checkConsistency(cloudManager, config);
@@ -397,7 +396,7 @@ public class SimCloudManager implements SolrCloudManager {
     Map<String, Map<Replica.State, AtomicInteger>> replicaStates = new TreeMap<>();
     int numReplicas = 0;
     for (String node : getLiveNodesSet().get()) {
-      List<ReplicaInfo> replicas = getSimClusterStateProvider().simGetReplicaInfos(node);
+      List<Replica> replicas = getSimClusterStateProvider().simGetReplicaInfos(node);
       numReplicas += replicas.size();
       if (replicas.size() > maxReplicas) {
         maxReplicas = replicas.size();
@@ -405,7 +404,7 @@ public class SimCloudManager implements SolrCloudManager {
       if (minReplicas > replicas.size()) {
         minReplicas = replicas.size();
       }
-      for (ReplicaInfo ri : replicas) {
+      for (Replica ri : replicas) {
         replicaStates.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
             .computeIfAbsent(ri.getState(), s -> new AtomicInteger())
             .incrementAndGet();
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
index 46f1beb..563fd05 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimClusterStateProvider.java
@@ -53,7 +53,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
@@ -139,8 +138,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   private final SimDistribStateManager stateManager;
   private final SimCloudManager cloudManager;
 
-  private final Map<String, List<ReplicaInfo>> nodeReplicaMap = new ConcurrentHashMap<>();
-  private final Map<String, Map<String, List<ReplicaInfo>>> colShardReplicaMap = new ConcurrentHashMap<>();
+  private final Map<String, List<Replica>> nodeReplicaMap = new ConcurrentHashMap<>();
+  private final Map<String, Map<String, List<Replica>>> colShardReplicaMap = new ConcurrentHashMap<>();
   private final Map<String, Object> clusterProperties = new ConcurrentHashMap<>();
   private final Map<String, Map<String, Object>> collProperties = new ConcurrentHashMap<>();
   private final Map<String, Map<String, Map<String, Object>>> sliceProperties = new ConcurrentHashMap<>();
@@ -190,10 +189,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
                 }
                 Map<String, Object> props;
                 synchronized (ri) {
-                  props = new HashMap<>(ri.getVariables());
+                  props = new HashMap<>(ri.getProperties());
                 }
                 props.put(ZkStateReader.NODE_NAME_PROP, n);
-                props.put(ZkStateReader.CORE_NAME_PROP, ri.getCore());
+                props.put(ZkStateReader.CORE_NAME_PROP, ri.getCoreName());
                 props.put(ZkStateReader.REPLICA_TYPE, ri.getType().toString());
                 props.put(ZkStateReader.STATE_PROP, ri.getState().toString());
                 Replica r = new Replica(ri.getName(), props, ri.getCollection(), ri.getShard());
@@ -360,16 +359,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             if (leader != null && r.getName().equals(leader.getName())) {
               props.put("leader", "true");
             }
-            ReplicaInfo ri = new ReplicaInfo(r.getName(), r.getCoreName(), dc.getName(), s.getName(), r.getType(), r.getNodeName(), props);
+            Replica ri = new Replica(r.getName(), r.getNode(), dc.getName(), s.getName(), r.getCoreName(), r.isLeader,
+                r.getState(), r.getType(), props);
             if (leader != null && r.getName().equals(leader.getName())) {
-              ri.getVariables().put("leader", "true");
+              ri.getProperties().put("leader", "true");
             }
-            if (liveNodes.get().contains(r.getNodeName())) {
-              nodeReplicaMap.computeIfAbsent(r.getNodeName(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN).add(ri);
+            if (liveNodes.get().contains(r.getNode())) {
+              nodeReplicaMap.computeIfAbsent(r.getNode(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN).add(ri);
               colShardReplicaMap.computeIfAbsent(ri.getCollection(), name -> new ConcurrentHashMap<>())
                   .computeIfAbsent(ri.getShard(), shard -> new ArrayList<>()).add(ri);
             } else {
-              log.warn("- dropping replica because its node {} is not live: {}", r.getNodeName(), r);
+              log.warn("- dropping replica because its node {} is not live: {}", r.getNode(), r);
             }
           });
         });
@@ -412,13 +412,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     return nodes.get(random.nextInt(nodes.size()));
   }
 
-  private ReplicaInfo getReplicaInfo(Replica r) {
+  private Replica getReplicaInfo(Replica r) {
     @SuppressWarnings({"unchecked"})
-    final List<ReplicaInfo> list = nodeReplicaMap.computeIfAbsent
-      (r.getNodeName(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
+    final List<Replica> list = nodeReplicaMap.computeIfAbsent
+      (r.getNode(), Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
     synchronized (list) {
-      for (ReplicaInfo ri : list) {
-        if (r.getCoreName().equals(ri.getCore()) && r.getName().equals(ri.getName())) {
+      for (Replica ri : list) {
+        if (r.getCoreName().equals(ri.getCoreName()) && r.getName().equals(ri.getName())) {
           return ri;
         }
       }
@@ -529,12 +529,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   // this method needs to be called under a lock
   private void setReplicaStates(String nodeId, Replica.State state, Set<String> changedCollections) {
     @SuppressWarnings({"unchecked"})
-    List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
+    List<Replica> replicas = nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
     synchronized (replicas) {
       replicas.forEach(r -> {
-        r.getVariables().put(ZkStateReader.STATE_PROP, state.toString());
+        r.getProperties().put(ZkStateReader.STATE_PROP, state.toString());
         if (state != Replica.State.ACTIVE) {
-          r.getVariables().remove(ZkStateReader.LEADER_PROP);
+          r.getProperties().remove(ZkStateReader.LEADER_PROP);
         }
         changedCollections.add(r.getCollection());
       });
@@ -628,13 +628,15 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       if (message.getStr(CoreAdminParams.CORE_NODE_NAME) == null) {
         createReplica.coreNodeName = Assign.assignCoreNodeName(stateManager, coll);
       }
-      ReplicaInfo ri = new ReplicaInfo(
+      Replica ri = new Replica(
           createReplica.coreNodeName,
-          createReplica.coreName,
+          createReplica.node,
           createReplica.collectionName,
           createReplica.sliceName,
+          createReplica.coreName,
+          false,
+          Replica.State.DOWN,
           createReplica.replicaType,
-          createReplica.node,
           message.getProperties()
       );
       simAddReplica(ri.getNode(), ri, true);
@@ -653,17 +655,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
    * @param runLeaderElection if true then run a leader election after adding the replica.
    */
   @SuppressWarnings({"unchecked"})
-  public void simAddReplica(String nodeId, ReplicaInfo replicaInfo, boolean runLeaderElection) throws Exception {
+  public void simAddReplica(String nodeId, Replica replicaInfo, boolean runLeaderElection) throws Exception {
     ensureNotClosed();
     lock.lockInterruptibly();
     try {
 
       // make sure SolrCore name is unique across cluster and coreNodeName within collection
-      for (Map.Entry<String, List<ReplicaInfo>> e : nodeReplicaMap.entrySet()) {
-        final List<ReplicaInfo> replicas = e.getValue();
+      for (Map.Entry<String, List<Replica>> e : nodeReplicaMap.entrySet()) {
+        final List<Replica> replicas = e.getValue();
         synchronized (replicas) {
-          for (ReplicaInfo ri : replicas) {
-            if (ri.getCore().equals(replicaInfo.getCore())) {
+          for (Replica ri : replicas) {
+            if (ri.getCoreName().equals(replicaInfo.getCoreName())) {
               throw new Exception("Duplicate SolrCore name for existing=" + ri + " on node " + e.getKey() + " and new=" + replicaInfo);
             }
             if (ri.getName().equals(replicaInfo.getName()) && ri.getCollection().equals(replicaInfo.getCollection())) {
@@ -676,11 +678,11 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         throw new Exception("Target node " + nodeId + " is not live: " + liveNodes);
       }
       // verify info
-      if (replicaInfo.getCore() == null) {
+      if (replicaInfo.getCoreName() == null) {
         throw new Exception("Missing core: " + replicaInfo);
       }
       // XXX replica info is not supposed to have this as a variable
-      replicaInfo.getVariables().remove(ZkStateReader.SHARD_ID_PROP);
+      replicaInfo.getProperties().remove(ZkStateReader.SHARD_ID_PROP);
       if (replicaInfo.getName() == null) {
         throw new Exception("Missing name: " + replicaInfo);
       }
@@ -694,11 +696,11 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       opDelay(replicaInfo.getCollection(), CollectionParams.CollectionAction.ADDREPLICA.name());
 
       // mark replica as active
-      replicaInfo.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
+      replicaInfo.getProperties().put(ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
       // add a property expected in Policy calculations, if missing
-      if (replicaInfo.getVariable(Type.CORE_IDX.metricsAttribute) == null) {
-        replicaInfo.getVariables().put(Type.CORE_IDX.metricsAttribute, new AtomicLong(SimCloudManager.DEFAULT_IDX_SIZE_BYTES));
-        replicaInfo.getVariables().put(Variable.coreidxsize,
+      if (replicaInfo.get(Type.CORE_IDX.metricsAttribute) == null) {
+        replicaInfo.getProperties().put(Type.CORE_IDX.metricsAttribute, new AtomicLong(SimCloudManager.DEFAULT_IDX_SIZE_BYTES));
+        replicaInfo.getProperties().put(Variable.coreidxsize,
             new AtomicDouble((Double)Type.CORE_IDX.convertVal(SimCloudManager.DEFAULT_IDX_SIZE_BYTES)));
       }
       nodeReplicaMap.computeIfAbsent(nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN).add(replicaInfo);
@@ -719,13 +721,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         throw new Exception("Missing '" + ImplicitSnitch.DISK + "' in node metrics for node " + nodeId);
         //disk = SimCloudManager.DEFAULT_FREE_DISK;
       }
-      long replicaSize = ((Number)replicaInfo.getVariable(Type.CORE_IDX.metricsAttribute)).longValue();
+      long replicaSize = ((Number)replicaInfo.get(Type.CORE_IDX.metricsAttribute)).longValue();
       Number replicaSizeGB = (Number)Type.CORE_IDX.convertVal(replicaSize);
       cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk.doubleValue() - replicaSizeGB.doubleValue());
       // fake metrics
       String registry = SolrMetricManager.getRegistryName(SolrInfoBean.Group.core, replicaInfo.getCollection(),
           replicaInfo.getShard(),
-          Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCore()));
+          Utils.parseMetricsReplicaName(replicaInfo.getCollection(), replicaInfo.getCoreName()));
       cloudManager.getMetricManager().registry(registry).counter("UPDATE./update.requests");
       cloudManager.getMetricManager().registry(registry).counter("QUERY./select.requests");
       cloudManager.getMetricManager().registerGauge(null, registry,
@@ -752,12 +754,12 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     lock.lockInterruptibly();
     try {
       @SuppressWarnings({"unchecked"})
-      final List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent
+      final List<Replica> replicas = nodeReplicaMap.computeIfAbsent
         (nodeId, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
       synchronized (replicas) {
         for (int i = 0; i < replicas.size(); i++) {
           if (collection.equals(replicas.get(i).getCollection()) && coreNodeName.equals(replicas.get(i).getName())) {
-            ReplicaInfo ri = replicas.remove(i);
+            Replica ri = replicas.remove(i);
             colShardReplicaMap.computeIfAbsent(ri.getCollection(), c -> new ConcurrentHashMap<>())
               .computeIfAbsent(ri.getShard(), s -> new ArrayList<>())
               .remove(ri);
@@ -776,10 +778,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
               if (disk == null || disk.doubleValue() == 0.0) {
                 throw new Exception("Unexpected value of 'freedisk' (" + disk + ") on node: " + nodeId);
               }
-              if (ri.getVariable(Type.CORE_IDX.metricsAttribute) == null) {
+              if (ri.get(Type.CORE_IDX.metricsAttribute) == null) {
                 throw new RuntimeException("Missing replica size: " + ri);
               }
-              long replicaSize = ((Number)ri.getVariable(Type.CORE_IDX.metricsAttribute)).longValue();
+              long replicaSize = ((Number)ri.get(Type.CORE_IDX.metricsAttribute)).longValue();
               Number replicaSizeGB = (Number)Type.CORE_IDX.convertVal(replicaSize);
               cloudManager.getSimNodeStateProvider().simSetNodeValue(nodeId, ImplicitSnitch.DISK, disk.doubleValue() + replicaSizeGB.doubleValue());
             }
@@ -890,7 +892,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       }
       
       final Replica leader = s.getLeader();
-      if (null != leader && liveNodes.contains(leader.getNodeName())) {
+      if (null != leader && liveNodes.contains(leader.getNode())) {
         log.trace("-- already has livenode leader, skipping leader election {} / {}",
                   collection, slice);
         return;
@@ -907,17 +909,17 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       ActionThrottle lt = getThrottle(collection, s.getName());
       synchronized (lt) {
         // collect all active and live
-        List<ReplicaInfo> active = new ArrayList<>();
+        List<Replica> active = new ArrayList<>();
         AtomicBoolean alreadyHasLeader = new AtomicBoolean(false);
         s.getReplicas().forEach(r -> {
             // find our ReplicaInfo for this replica
-            ReplicaInfo ri = getReplicaInfo(r);
+            Replica ri = getReplicaInfo(r);
             if (ri == null) {
               throw new IllegalStateException("-- could not find ReplicaInfo for replica " + r);
             }
             synchronized (ri) {
               if (r.isActive(liveNodes.get())) {
-                if (ri.getVariables().get(ZkStateReader.LEADER_PROP) != null) {
+                if (ri.getProperties().get(ZkStateReader.LEADER_PROP) != null) {
                   if (log.isTraceEnabled()) {
                     log.trace("-- found existing leader {} / {}: {}, {}", collection, s.getName(), ri, r);
                   }
@@ -930,9 +932,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
                 if (log.isTraceEnabled()) {
                   log.trace("-- replica not active on live nodes: {}, {}", liveNodes.get(), r);
                 }
-                if (!liveNodes.contains(r.getNodeName())) {
-                  ri.getVariables().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
-                  ri.getVariables().remove(ZkStateReader.LEADER_PROP);
+                if (!liveNodes.contains(r.getNode())) {
+                  ri.getProperties().put(ZkStateReader.STATE_PROP, Replica.State.DOWN.toString());
+                  ri.getProperties().remove(ZkStateReader.LEADER_PROP);
                   stateChanged.set(true);
                 }
               }
@@ -954,8 +956,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           return;
         }
         // pick first active one
-        ReplicaInfo ri = null;
-        for (ReplicaInfo a : active) {
+        Replica ri = null;
+        for (Replica a : active) {
           if (!a.getType().equals(Replica.Type.PULL)) {
             ri = a;
             break;
@@ -969,7 +971,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         lt.minimumWaitBetweenActions();
         lt.markAttemptingAction();
         synchronized (ri) {
-          ri.getVariables().put(ZkStateReader.LEADER_PROP, "true");
+          ri.getProperties().put(ZkStateReader.LEADER_PROP, "true");
         }
         if (log.isDebugEnabled()) {
           log.debug("-- elected new leader for {} / {} (currentVersion={}): {}", collection,
@@ -1106,8 +1108,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             replicaProps.put("SEARCHER.searcher.deletedDocs", new AtomicLong(0));
             replicaProps.put("SEARCHER.searcher.numDocs", new AtomicLong(0));
             replicaProps.put("SEARCHER.searcher.maxDoc", new AtomicLong(0));
-            ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, withCollection, 0),
-                coreName, withCollection, withCollectionShard, pos.type, pos.node, replicaProps);
+            Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, withCollection, 0),
+                pos.node, withCollection, withCollectionShard, coreName, false, Replica.State.DOWN,
+                pos.type, replicaProps);
             cloudManager.submit(() -> {
               simAddReplica(pos.node, ri, false);
               // do not count down the latch here
@@ -1129,8 +1132,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         replicaProps.put("SEARCHER.searcher.deletedDocs", new AtomicLong(0));
         replicaProps.put("SEARCHER.searcher.numDocs", new AtomicLong(0));
         replicaProps.put("SEARCHER.searcher.maxDoc", new AtomicLong(0));
-        ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
-            coreName, collectionName, pos.shard, pos.type, pos.node, replicaProps);
+        Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+            pos.node, collectionName, pos.shard, coreName, false, Replica.State.DOWN,
+            pos.type, replicaProps);
         cloudManager.submit(() -> {
           simAddReplica(pos.node, ri, true);
           finalStateLatch.countDown();
@@ -1187,8 +1191,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       opDelays.remove(collection);
       nodeReplicaMap.forEach((n, replicas) -> {
           synchronized (replicas) {  
-            for (Iterator<ReplicaInfo> it = replicas.iterator(); it.hasNext(); ) {
-              ReplicaInfo ri = it.next();
+            for (Iterator<Replica> it = replicas.iterator(); it.hasNext(); ) {
+              Replica ri = it.next();
               if (ri.getCollection().equals(collection)) {
                 it.remove();
                 // update the number of cores in node values
@@ -1292,11 +1296,11 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     }
 
     opDelay(collection, CollectionParams.CollectionAction.MOVEREPLICA.name());
-    ReplicaInfo ri = getReplicaInfo(replica);
+    Replica ri = getReplicaInfo(replica);
     if (ri != null) {
-      if (ri.getVariable(Type.CORE_IDX.tagName) != null) {
+      if (ri.get(Type.CORE_IDX.tagName) != null) {
         // simulate very large replicas - add additional delay of 5s / GB
-        long sizeInGB = ((Number)ri.getVariable(Type.CORE_IDX.tagName)).longValue();
+        long sizeInGB = ((Number)ri.get(Type.CORE_IDX.tagName)).longValue();
         long opDelay = opDelays.getOrDefault(ri.getCollection(), Collections.emptyMap())
             .getOrDefault(CollectionParams.CollectionAction.MOVEREPLICA.name(), defaultOpDelays.get(CollectionParams.CollectionAction.MOVEREPLICA.name()));
         opDelay = TimeUnit.MILLISECONDS.toSeconds(opDelay);
@@ -1315,12 +1319,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     Map<String, Object> props = replica.getProperties().entrySet().stream()
         .filter(e -> !NO_COPY_PROPS.contains(e.getKey()))
         .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
-    ReplicaInfo newReplica = new ReplicaInfo(coreNodeName, newSolrCoreName, collection, slice.getName(), replica.getType(), targetNode, props);
+    Replica newReplica = new Replica(coreNodeName, targetNode, collection, slice.getName(), newSolrCoreName,
+        false, Replica.State.DOWN, replica.getType(), props);
     log.debug("-- new replica: {}", newReplica);
     // xxx should run leader election here already?
     simAddReplica(targetNode, newReplica, false);
     // this will trigger leader election
-    simRemoveReplica(replica.getNodeName(), collection, replica.getName());
+    simRemoveReplica(replica.getNode(), collection, replica.getName());
     results.add("success", "");
   }
 
@@ -1458,7 +1463,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
     if (sessionWrapper != null) sessionWrapper.release();
 
     // adjust numDocs / deletedDocs / maxDoc
-    String numDocsStr = String.valueOf(getReplicaInfo(leader).getVariable("SEARCHER.searcher.numDocs", "0"));
+    String numDocsStr = String.valueOf(getReplicaInfo(leader).get("SEARCHER.searcher.numDocs", "0"));
     long numDocs = Long.parseLong(numDocsStr);
     long newNumDocs = numDocs / subSlices.size();
     long remainderDocs = numDocs % subSlices.size();
@@ -1504,8 +1509,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       replicaProps.put(Type.CORE_IDX.metricsAttribute, new AtomicLong(replicasIndexSize));
       replicaProps.put(Variable.coreidxsize, new AtomicDouble((Double)Type.CORE_IDX.convertVal(replicasIndexSize)));
 
-      ReplicaInfo ri = new ReplicaInfo("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
-          solrCoreName, collectionName, replicaPosition.shard, replicaPosition.type, subShardNodeName, replicaProps);
+      Replica ri = new Replica("core_node" + Assign.incAndGetId(stateManager, collectionName, 0),
+          subShardNodeName, collectionName, replicaPosition.shard, solrCoreName, false,
+          Replica.State.DOWN, replicaPosition.type, replicaProps);
       simAddReplica(replicaPosition.node, ri, false);
     }
     simRunLeaderElection(Collections.singleton(collectionName), true);
@@ -1620,9 +1626,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       colShardReplicaMap.computeIfAbsent(collectionName, c -> new ConcurrentHashMap<>()).remove(sliceName);
       nodeReplicaMap.forEach((n, replicas) -> {
           synchronized (replicas) {
-            Iterator<ReplicaInfo> it = replicas.iterator();
+            Iterator<Replica> it = replicas.iterator();
             while (it.hasNext()) {
-              ReplicaInfo ri = it.next();
+              Replica ri = it.next();
               if (ri.getCollection().equals(collectionName) && ri.getShard().equals(sliceName)) {
                 it.remove();
               }
@@ -1700,8 +1706,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           throw new IOException("-- no leader in " + s);
         }
         cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
-        ReplicaInfo ri = getReplicaInfo(leader);
-        Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
+        Replica ri = getReplicaInfo(leader);
+        Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
         if (numDocs == null || numDocs.intValue() <= 0) {
           if (log.isDebugEnabled()) {
             log.debug("-- attempting to delete nonexistent doc {} from {}", id, s.getLeader());
@@ -1712,7 +1718,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         // this is somewhat wrong - we should wait until buffered updates are applied
         // but this way the freedisk changes are much easier to track
         s.getReplicas().forEach(r ->
-            freediskDeltaPerNode.computeIfAbsent(r.getNodeName(), node -> new AtomicLong(0))
+            freediskDeltaPerNode.computeIfAbsent(r.getNode(), node -> new AtomicLong(0))
                 .addAndGet(DEFAULT_DOC_SIZE_BYTES));
 
         AtomicLong bufferedUpdates = (AtomicLong)sliceProperties.get(collection).get(s.getName()).get(BUFFERED_UPDATES);
@@ -1727,7 +1733,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           continue;
         }
         deletesPerShard.computeIfAbsent(s.getName(), slice -> new AtomicLong(0)).incrementAndGet();
-        Number indexSize = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
+        Number indexSize = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
         if (indexSize != null) {
           indexSizePerShard.put(s.getName(), indexSize);
         }
@@ -1775,19 +1781,19 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           }
 
           cloudManager.getMetricManager().registry(createRegistryName(collection, s.getName(), leader)).counter("UPDATE./update.requests").inc();
-          ReplicaInfo ri = getReplicaInfo(leader);
-          Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
+          Replica ri = getReplicaInfo(leader);
+          Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
           if (numDocs == null || numDocs.intValue() == 0) {
             continue;
           }
           lock.lockInterruptibly();
           try {
-            Number indexSize = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
+            Number indexSize = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
             if (indexSize != null) {
               long delta = indexSize.longValue() < SimCloudManager.DEFAULT_IDX_SIZE_BYTES ? 0 :
                   indexSize.longValue() - SimCloudManager.DEFAULT_IDX_SIZE_BYTES;
               s.getReplicas().forEach(r ->
-                  freediskDeltaPerNode.computeIfAbsent(r.getNodeName(), node -> new AtomicLong(0))
+                  freediskDeltaPerNode.computeIfAbsent(r.getNode(), node -> new AtomicLong(0))
                   .addAndGet(delta));
             } else {
               throw new RuntimeException("Missing index size in " + ri);
@@ -1876,7 +1882,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             modified = true;
             long perSliceCount = perSlice[i];
             s.getReplicas().forEach(r ->
-                freediskDeltaPerNode.computeIfAbsent(r.getNodeName(), node -> new AtomicLong(0))
+                freediskDeltaPerNode.computeIfAbsent(r.getNode(), node -> new AtomicLong(0))
                     .addAndGet(-perSliceCount * DEFAULT_DOC_SIZE_BYTES));
             AtomicLong bufferedUpdates = (AtomicLong)sliceProperties.get(collection).get(s.getName()).get(BUFFERED_UPDATES);
             if (bufferedUpdates != null) {
@@ -1903,7 +1909,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
                 .incrementAndGet();
             modified = true;
             s.getReplicas().forEach(r ->
-                freediskDeltaPerNode.computeIfAbsent(r.getNodeName(), node -> new AtomicLong())
+                freediskDeltaPerNode.computeIfAbsent(r.getNode(), node -> new AtomicLong())
                     .addAndGet(-DEFAULT_DOC_SIZE_BYTES));
             AtomicLong bufferedUpdates = (AtomicLong)sliceProperties.get(collection).get(s.getName()).get(BUFFERED_UPDATES);
             if (bufferedUpdates != null) {
@@ -1974,8 +1980,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       try {
         coll.getSlices().forEach(s -> {
           Replica leader = s.getLeader();
-          ReplicaInfo ri = getReplicaInfo(leader);
-          Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
+          Replica ri = getReplicaInfo(leader);
+          Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
           if (numDocs == null || numDocs.intValue() == 0) {
             numDocs = 0;
           }
@@ -2015,8 +2021,8 @@ public class SimClusterStateProvider implements ClusterStateProvider {
       if (r == null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, collection + "/" + s.getName() + " has no leader");
       }
-      ReplicaInfo ri = getReplicaInfo(r);
-      Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs", 0L);
+      Replica ri = getReplicaInfo(r);
+      Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs", 0L);
       count.addAndGet(numDocs.longValue());
       AtomicLong bufferedUpdates = (AtomicLong)sliceProperties.get(collection).get(s.getName()).get(BUFFERED_UPDATES);
       if (bufferedUpdates != null) {
@@ -2212,7 +2218,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
    *               divided by the number of replicas.
    */
   public void simSetShardValue(String collection, String shard, String key, Object value, boolean delta, boolean divide) throws Exception {
-    final List<ReplicaInfo> infos;
+    final List<Replica> infos;
     if (shard == null) {
       infos = new ArrayList<>();
       colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
@@ -2231,13 +2237,13 @@ public class SimClusterStateProvider implements ClusterStateProvider {
         value = ((Number) value).doubleValue() / infos.size();
       }
     }
-    for (ReplicaInfo r : infos) {
+    for (Replica r : infos) {
       synchronized (r) {
         if (value == null) {
-          r.getVariables().remove(key);
+          r.getProperties().remove(key);
         } else {
           if (delta) {
-            Object prevValue = r.getVariables().get(key);
+            Object prevValue = r.getProperties().get(key);
             if (prevValue != null) {
               if ((prevValue instanceof Number) && (value instanceof Number)) {
                 if (((prevValue instanceof Long) || (prevValue instanceof Integer) ||
@@ -2250,14 +2256,14 @@ public class SimClusterStateProvider implements ClusterStateProvider {
                   } else if (prevValue instanceof AtomicInteger) {
                     ((AtomicInteger)prevValue).set(((Number)prevValue).intValue() + ((Number)value).intValue());
                   } else {
-                    r.getVariables().put(key, newValue);
+                    r.getProperties().put(key, newValue);
                   }
                 } else {
                   double newValue = ((Number)prevValue).doubleValue() + ((Number)value).doubleValue();
                   if (prevValue instanceof AtomicDouble) {
                     ((AtomicDouble)prevValue).set(newValue);
                   } else {
-                    r.getVariables().put(key, newValue);
+                    r.getProperties().put(key, newValue);
                   }
                 }
               } else {
@@ -2265,24 +2271,24 @@ public class SimClusterStateProvider implements ClusterStateProvider {
               }
             } else {
               if (value instanceof Integer) {
-                r.getVariables().put(key, new AtomicInteger((Integer)value));
+                r.getProperties().put(key, new AtomicInteger((Integer)value));
               } else if (value instanceof Long) {
-                r.getVariables().put(key, new AtomicLong((Long)value));
+                r.getProperties().put(key, new AtomicLong((Long)value));
               } else if (value instanceof Double) {
-                r.getVariables().put(key, new AtomicDouble((Double)value));
+                r.getProperties().put(key, new AtomicDouble((Double)value));
               } else {
-                r.getVariables().put(key, value);
+                r.getProperties().put(key, value);
               }
             }
           } else {
             if (value instanceof Integer) {
-              r.getVariables().put(key, new AtomicInteger((Integer)value));
+              r.getProperties().put(key, new AtomicInteger((Integer)value));
             } else if (value instanceof Long) {
-              r.getVariables().put(key, new AtomicLong((Long)value));
+              r.getProperties().put(key, new AtomicLong((Long)value));
             } else if (value instanceof Double) {
-              r.getVariables().put(key, new AtomicDouble((Double)value));
+              r.getProperties().put(key, new AtomicDouble((Double)value));
             } else {
-              r.getVariables().put(key, value);
+              r.getProperties().put(key, value);
             }
           }
         }
@@ -2291,26 +2297,26 @@ public class SimClusterStateProvider implements ClusterStateProvider {
   }
 
   @SuppressWarnings({"unchecked"})
-  public void simSetReplicaValues(String node, Map<String, Map<String, List<ReplicaInfo>>> source, boolean overwrite) {
-    List<ReplicaInfo> infos = nodeReplicaMap.get(node);
+  public void simSetReplicaValues(String node, Map<String, Map<String, List<Replica>>> source, boolean overwrite) {
+    List<Replica> infos = nodeReplicaMap.get(node);
     if (infos == null) {
       throw new RuntimeException("Node not present: " + node);
     }
     // core_node_name is not unique across collections
-    Map<String, Map<String, ReplicaInfo>> infoMap = new HashMap<>();
+    Map<String, Map<String, Replica>> infoMap = new HashMap<>();
     infos.forEach(ri -> infoMap.computeIfAbsent(ri.getCollection(), Utils.NEW_HASHMAP_FUN).put(ri.getName(), ri));
     source.forEach((coll, shards) -> shards.forEach((shard, replicas) -> replicas.forEach(r -> {
-      ReplicaInfo target = infoMap.getOrDefault(coll, Collections.emptyMap()).get(r.getName());
+      Replica target = infoMap.getOrDefault(coll, Collections.emptyMap()).get(r.getName());
       if (target == null) {
         throw new RuntimeException("Unable to find simulated replica of " + r);
       }
-      r.getVariables().forEach((k, v) -> {
-        if (target.getVariables().containsKey(k)) {
+      r.getProperties().forEach((k, v) -> {
+        if (target.getProperties().containsKey(k)) {
           if (overwrite) {
-            target.getVariables().put(k, v);
+            target.getProperties().put(k, v);
           }
         } else {
-          target.getVariables().put(k, v);
+          target.getProperties().put(k, v);
         }
       });
     })));
@@ -2321,29 +2327,29 @@ public class SimClusterStateProvider implements ClusterStateProvider {
    * @param node node id
    * @return copy of the list of replicas on that node, or empty list if none
    */
-  public List<ReplicaInfo> simGetReplicaInfos(String node) {
+  public List<Replica> simGetReplicaInfos(String node) {
     @SuppressWarnings({"unchecked"})
-    final List<ReplicaInfo> replicas = nodeReplicaMap.computeIfAbsent
+    final List<Replica> replicas = nodeReplicaMap.computeIfAbsent
       (node, Utils.NEW_SYNCHRONIZED_ARRAYLIST_FUN);
     // make a defensive copy to avoid ConcurrentModificationException
-    return Arrays.asList(replicas.toArray(new ReplicaInfo[replicas.size()]));
+    return Arrays.asList(replicas.toArray(new Replica[replicas.size()]));
   }
 
-  public List<ReplicaInfo> simGetReplicaInfos(String collection, String shard) {
-    List<ReplicaInfo> replicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
+  public List<Replica> simGetReplicaInfos(String collection, String shard) {
+    List<Replica> replicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>())
         .computeIfAbsent(shard, s -> new ArrayList<>());
     if (replicas == null) {
       return Collections.emptyList();
     } else {
       // make a defensive copy to avoid ConcurrentModificationException
-      return Arrays.asList(replicas.toArray(new ReplicaInfo[replicas.size()]));
+      return Arrays.asList(replicas.toArray(new Replica[replicas.size()]));
     }
   }
 
-  public ReplicaInfo simGetReplicaInfo(String collection, String coreNode) {
-    Map<String, List<ReplicaInfo>> shardsReplicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>());
-    for (List<ReplicaInfo> replicas : shardsReplicas.values()) {
-      for (ReplicaInfo ri : replicas) {
+  public Replica simGetReplicaInfo(String collection, String coreNode) {
+    Map<String, List<Replica>> shardsReplicas = colShardReplicaMap.computeIfAbsent(collection, c -> new ConcurrentHashMap<>());
+    for (List<Replica> replicas : shardsReplicas.values()) {
+      for (Replica ri : replicas) {
         if (ri.getName().equals(coreNode)) {
           return ri;
         }
@@ -2388,10 +2394,10 @@ public class SimClusterStateProvider implements ClusterStateProvider {
           totalReplicas += s.getReplicas().size();
           if (s.getState() != Slice.State.ACTIVE) {
             if (!s.getReplicas().isEmpty()) {
-              ReplicaInfo ri = getReplicaInfo(s.getReplicas().iterator().next());
+              Replica ri = getReplicaInfo(s.getReplicas().iterator().next());
               if (ri != null) {
-                Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
-                Number numBytes = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
+                Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
+                Number numBytes = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
                 if (numDocs != null) {
                   inactiveDocs.addValue(numDocs.doubleValue());
                 }
@@ -2421,7 +2427,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
               leader = s.getReplicas().iterator().next();
             }
           }
-          ReplicaInfo ri = null;
+          Replica ri = null;
           if (leader != null) {
             ri = getReplicaInfo(leader);
             if (ri == null) {
@@ -2429,9 +2435,9 @@ public class SimClusterStateProvider implements ClusterStateProvider {
             }
           }
           if (ri != null) {
-            Number numDocs = (Number)ri.getVariable("SEARCHER.searcher.numDocs");
-            Number delDocs = (Number)ri.getVariable("SEARCHER.searcher.deleteDocs");
-            Number numBytes = (Number)ri.getVariable(Type.CORE_IDX.metricsAttribute);
+            Number numDocs = (Number)ri.get("SEARCHER.searcher.numDocs");
+            Number delDocs = (Number)ri.get("SEARCHER.searcher.deleteDocs");
+            Number numBytes = (Number)ri.get(Type.CORE_IDX.metricsAttribute);
             if (numDocs != null) {
               docs.addValue(numDocs.doubleValue());
             }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
index 17b6d28..d0b7b89 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimNodeStateProvider.java
@@ -35,7 +35,7 @@ import java.util.regex.Pattern;
 import java.util.stream.Collectors;
 
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.util.Utils;
 import org.slf4j.Logger;
@@ -291,14 +291,14 @@ public class SimNodeStateProvider implements NodeStateProvider {
       String collection = m.group(1);
       String shard = m.group(2);
       String replica = m.group(3);
-      List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(collection, shard);
+      List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(collection, shard);
       replicas.forEach(r -> {
-        if (r.getNode().equals(node) && r.getCore().endsWith(replica)) {
-          Object value = r.getVariables().get(key);
+        if (r.getNode().equals(node) && r.getCoreName().endsWith(replica)) {
+          Object value = r.getProperties().get(key);
           if (value != null) {
             values.put(tag, value);
           } else {
-            value = r.getVariables().get(tag);
+            value = r.getProperties().get(tag);
             if (value != null) {
               values.put(tag, value);
             }
@@ -334,18 +334,18 @@ public class SimNodeStateProvider implements NodeStateProvider {
   }
 
   @Override
-  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
-    List<ReplicaInfo> replicas = clusterStateProvider.simGetReplicaInfos(node);
+  public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
+    List<Replica> replicas = clusterStateProvider.simGetReplicaInfos(node);
     if (replicas == null || replicas.isEmpty()) {
       return new HashMap<>();
     }
-    Map<String, Map<String, List<ReplicaInfo>>> res = new HashMap<>();
+    Map<String, Map<String, List<Replica>>> res = new HashMap<>();
     // TODO: probably needs special treatment for "metrics:solr.core..." tags
-    for (ReplicaInfo r : replicas) {
+    for (Replica r : replicas) {
       @SuppressWarnings({"unchecked"})
-      Map<String, List<ReplicaInfo>> perCollection = res.computeIfAbsent(r.getCollection(), Utils.NEW_HASHMAP_FUN);
+      Map<String, List<Replica>> perCollection = res.computeIfAbsent(r.getCollection(), Utils.NEW_HASHMAP_FUN);
       @SuppressWarnings({"unchecked"})
-      List<ReplicaInfo> perShard = perCollection.computeIfAbsent(r.getShard(), Utils.NEW_ARRAYLIST_FUN);
+      List<Replica> perShard = perCollection.computeIfAbsent(r.getShard(), Utils.NEW_ARRAYLIST_FUN);
       // XXX filter out some properties?
       perShard.add(r);
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
index 6adb812..5cba5f5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimScenario.java
@@ -49,7 +49,6 @@ import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Clause;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventProcessorStage;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
@@ -66,6 +65,7 @@ import org.apache.solr.cloud.autoscaling.AutoScalingHandler;
 import org.apache.solr.cloud.autoscaling.TriggerEvent;
 import org.apache.solr.cloud.autoscaling.TriggerListener;
 import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.AutoScalingParams;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CollectionParams;
@@ -561,14 +561,14 @@ public class SimScenario implements AutoCloseable {
         }
         Map<String, Object> paramsMap = new LinkedHashMap<>();
         params.toMap(paramsMap);
-        ReplicaInfo info = scenario.cluster.getSimClusterStateProvider().simGetReplicaInfo(
+        Replica info = scenario.cluster.getSimClusterStateProvider().simGetReplicaInfo(
             params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
         if (info == null) {
           log.error("Could not find ReplicaInfo for params: {}", params);
         } else if (scenario.verbose) {
           paramsMap.put("replicaInfo", info);
-        } else if (info.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
-          paramsMap.put(Variable.Type.CORE_IDX.tagName, info.getVariable(Variable.Type.CORE_IDX.tagName));
+        } else if (info.get(Variable.Type.CORE_IDX.tagName) != null) {
+          paramsMap.put(Variable.Type.CORE_IDX.tagName, info.get(Variable.Type.CORE_IDX.tagName));
         }
         try {
           scenario.cluster.request(operation);
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
index 5141317..a94aeac 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SimUtils.java
@@ -36,7 +36,6 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Cell;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Row;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
 import org.apache.solr.client.solrj.request.CollectionApiMapping;
@@ -110,9 +109,9 @@ public class SimUtils {
         }
       });
     });
-    Map<String, Map<String, ReplicaInfo>> allReplicaInfos = new HashMap<>();
+    Map<String, Map<String, Replica>> allReplicaInfos = new HashMap<>();
     solrCloudManager.getClusterStateProvider().getLiveNodes().forEach(n -> {
-      Map<String, Map<String, List<ReplicaInfo>>> infos = solrCloudManager.getNodeStateProvider().getReplicaInfo(n, replicaTags);
+      Map<String, Map<String, List<Replica>>> infos = solrCloudManager.getNodeStateProvider().getReplicaInfo(n, replicaTags);
       infos.forEach((coll, shards) -> shards.forEach((shard, replicas) -> replicas.forEach(r -> {
         if (allReplicaInfos.containsKey(r.getName())) {
           throw new RuntimeException("duplicate core_node name in NodeStateProvider: " + allReplicaInfos.get(r.getName()) + " versus " + r);
@@ -152,9 +151,9 @@ public class SimUtils {
     });
     // verify all replicas have size info
     allReplicaInfos.forEach((coll, replicas) -> replicas.forEach((core, ri) -> {
-          Number size = (Number) ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute);
+          Number size = (Number) ri.get(Variable.Type.CORE_IDX.metricsAttribute);
           if (size == null) {
-            size = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
+            size = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
             if (size == null) {
 //              for (String node : solrCloudManager.getClusterStateProvider().getLiveNodes()) {
 //                log.error("Check for missing values: {}: {}", node, solrCloudManager.getNodeStateProvider().getReplicaInfo(node, SnapshotNodeStateProvider.REPLICA_TAGS));
@@ -184,7 +183,7 @@ public class SimUtils {
       coll.getSlices().forEach(s -> {
         numCores.addAndGet(s.getReplicas().size());
         s.getReplicas().forEach(r -> {
-          nodes.computeIfAbsent(r.getNodeName(), n -> new HashMap<>())
+          nodes.computeIfAbsent(r.getNode(), n -> new HashMap<>())
               .computeIfAbsent(s.getName(), slice -> new AtomicInteger()).incrementAndGet();
         });
       });
@@ -261,17 +260,17 @@ public class SimUtils {
       }
       row.forEachReplica(ri -> {
         Map<String, Object> perReplica = collReplicas.computeIfAbsent(ri.getCollection(), c -> new TreeMap<>())
-            .computeIfAbsent(ri.getCore().substring(ri.getCollection().length() + 1), core -> new LinkedHashMap<>());
+            .computeIfAbsent(ri.getCoreName().substring(ri.getCollection().length() + 1), core -> new LinkedHashMap<>());
 //            if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
 //              perReplica.put(Variable.Type.CORE_IDX.tagName, ri.getVariable(Variable.Type.CORE_IDX.tagName));
 //            }
-        if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) != null) {
-          perReplica.put(Variable.Type.CORE_IDX.metricsAttribute, ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute));
-          if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
-            perReplica.put(Variable.Type.CORE_IDX.tagName, ri.getVariable(Variable.Type.CORE_IDX.tagName));
+        if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) != null) {
+          perReplica.put(Variable.Type.CORE_IDX.metricsAttribute, ri.get(Variable.Type.CORE_IDX.metricsAttribute));
+          if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
+            perReplica.put(Variable.Type.CORE_IDX.tagName, ri.get(Variable.Type.CORE_IDX.tagName));
           } else {
             perReplica.put(Variable.Type.CORE_IDX.tagName,
-                Variable.Type.CORE_IDX.convertVal(ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute)));
+                Variable.Type.CORE_IDX.convertVal(ri.get(Variable.Type.CORE_IDX.metricsAttribute)));
           }
         }
         perReplica.put("coreNode", ri.getName());
@@ -279,7 +278,7 @@ public class SimUtils {
           perReplica.put("leader", true);
           Double totalSize = (Double)collStats.computeIfAbsent(ri.getCollection(), c -> new HashMap<>())
               .computeIfAbsent("avgShardSize", size -> 0.0);
-          Number riSize = (Number)ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute);
+          Number riSize = (Number)ri.get(Variable.Type.CORE_IDX.metricsAttribute);
           if (riSize != null) {
             totalSize += riSize.doubleValue();
             collStats.get(ri.getCollection()).put("avgShardSize", totalSize);
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotCloudManager.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotCloudManager.java
index 9dc0b4a..2a0a25d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotCloudManager.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotCloudManager.java
@@ -42,11 +42,11 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ObjectCache;
@@ -193,7 +193,7 @@ public class SnapshotCloudManager implements SolrCloudManager {
           if (s.getOperation() instanceof V2Request) {
             params = SimUtils.v2AdminRequestToV1Params((V2Request)s.getOperation());
           }
-          ReplicaInfo info = nodeStateProvider.getReplicaInfo(
+          Replica info = nodeStateProvider.getReplicaInfo(
               params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
           if (info == null) {
             log.warn("Can't find ReplicaInfo for suggested operation: {}", s);
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotNodeStateProvider.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotNodeStateProvider.java
index e8b7828..53ff928 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotNodeStateProvider.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/sim/SnapshotNodeStateProvider.java
@@ -30,15 +30,15 @@ import java.util.Set;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
+import org.apache.solr.common.cloud.Replica;
 
 /**
  * Read-only snapshot of another {@link NodeStateProvider}.
  */
 public class SnapshotNodeStateProvider implements NodeStateProvider {
   private Map<String, Map<String, Object>> nodeValues = new LinkedHashMap<>();
-  private Map<String, Map<String, Map<String, List<ReplicaInfo>>>> replicaInfos = new LinkedHashMap<>();
+  private Map<String, Map<String, Map<String, List<Replica>>>> replicaInfos = new LinkedHashMap<>();
 
   private static double GB = 1024.0d * 1024.0d * 1024.0d;
 
@@ -59,11 +59,11 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
     replicaTags.addAll(config.getPolicy().getPerReplicaAttributes());
     for (String node : other.getClusterStateProvider().getLiveNodes()) {
       nodeValues.put(node, new LinkedHashMap<>(other.getNodeStateProvider().getNodeValues(node, nodeTags)));
-      Map<String, Map<String, List<ReplicaInfo>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
+      Map<String, Map<String, List<Replica>>> infos = other.getNodeStateProvider().getReplicaInfo(node, replicaTags);
       infos.forEach((collection, shards) -> {
         shards.forEach((shard, replicas) -> {
           replicas.forEach(r -> {
-            List<ReplicaInfo> myReplicas = replicaInfos
+            List<Replica> myReplicas = replicaInfos
                 .computeIfAbsent(node, n -> new LinkedHashMap<>())
                 .computeIfAbsent(collection, c -> new LinkedHashMap<>())
                 .computeIfAbsent(shard, s -> new ArrayList<>());
@@ -72,16 +72,16 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
             if (r.isLeader) { // ReplicaInfo.toMap doesn't write this!!!
               ((Map<String, Object>)rMap.values().iterator().next()).put("leader", "true");
             }
-            ReplicaInfo ri = new ReplicaInfo(rMap);
+            Replica ri = new Replica(rMap);
             // put in "leader" again if present
             if (r.isLeader) {
-              ri.getVariables().put("leader", "true");
+              ri.getProperties().put("leader", "true");
             }
             // externally produced snapshots may not include the right units
-            if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) == null) {
-              if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
-                Number indexSizeGB = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
-                ri.getVariables().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
+            if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
+              if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
+                Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
+                ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
               } else {
                 throw new RuntimeException("Missing size information for replica: " + ri);
               }
@@ -102,21 +102,21 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
     Objects.requireNonNull(snapshot);
     nodeValues = (Map<String, Map<String, Object>>)snapshot.getOrDefault("nodeValues", Collections.emptyMap());
     ((Map<String, Object>)snapshot.getOrDefault("replicaInfos", Collections.emptyMap())).forEach((node, v) -> {
-      Map<String, Map<String, List<ReplicaInfo>>> perNode = replicaInfos.computeIfAbsent(node, n -> new LinkedHashMap<>());
+      Map<String, Map<String, List<Replica>>> perNode = replicaInfos.computeIfAbsent(node, n -> new LinkedHashMap<>());
       ((Map<String, Object>)v).forEach((collection, shards) -> {
-        Map<String, List<ReplicaInfo>> perColl = perNode.computeIfAbsent(collection, c -> new LinkedHashMap<>());
+        Map<String, List<Replica>> perColl = perNode.computeIfAbsent(collection, c -> new LinkedHashMap<>());
         ((Map<String, Object>)shards).forEach((shard, replicas) -> {
-          List<ReplicaInfo> infos = perColl.computeIfAbsent(shard, s -> new ArrayList<>());
+          List<Replica> infos = perColl.computeIfAbsent(shard, s -> new ArrayList<>());
           ((List<Map<String, Object>>)replicas).forEach(replicaMap -> {
-            ReplicaInfo ri = new ReplicaInfo(new LinkedHashMap<>(replicaMap)); // constructor modifies this map
+            Replica ri = new Replica(new LinkedHashMap<>(replicaMap)); // constructor modifies this map
             if (ri.isLeader) {
-              ri.getVariables().put("leader", "true");
+              ri.getProperties().put("leader", "true");
             }
             // externally produced snapshots may not include the right units
-            if (ri.getVariable(Variable.Type.CORE_IDX.metricsAttribute) == null) {
-                if (ri.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
-                  Number indexSizeGB = (Number) ri.getVariable(Variable.Type.CORE_IDX.tagName);
-                  ri.getVariables().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
+            if (ri.get(Variable.Type.CORE_IDX.metricsAttribute) == null) {
+                if (ri.get(Variable.Type.CORE_IDX.tagName) != null) {
+                  Number indexSizeGB = (Number) ri.get(Variable.Type.CORE_IDX.tagName);
+                  ri.getProperties().put(Variable.Type.CORE_IDX.metricsAttribute, indexSizeGB.doubleValue() * GB);
                 } else {
                   throw new RuntimeException("Missing size information for replica: " + ri);
               }
@@ -165,17 +165,17 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
   }
 
   @Override
-  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
-    Map<String, Map<String, List<ReplicaInfo>>> result = new LinkedHashMap<>();
-    Map<String, Map<String, List<ReplicaInfo>>> infos = replicaInfos.getOrDefault(node, Collections.emptyMap());
+  public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
+    Map<String, Map<String, List<Replica>>> result = new LinkedHashMap<>();
+    Map<String, Map<String, List<Replica>>> infos = replicaInfos.getOrDefault(node, Collections.emptyMap());
     // deep copy
     infos.forEach((coll, shards) -> {
       shards.forEach((shard, replicas) -> {
         replicas.forEach(ri -> {
-          List<ReplicaInfo> myReplicas = result
+          List<Replica> myReplicas = result
               .computeIfAbsent(coll, c -> new LinkedHashMap<>())
               .computeIfAbsent(shard, s -> new ArrayList<>());
-          ReplicaInfo myReplica = (ReplicaInfo)ri.clone();
+          Replica myReplica = (Replica)ri.clone();
           myReplicas.add(myReplica);
         });
       });
@@ -183,12 +183,12 @@ public class SnapshotNodeStateProvider implements NodeStateProvider {
     return result;
   }
 
-  public ReplicaInfo getReplicaInfo(String collection, String coreNode) {
-    for (Map<String, Map<String, List<ReplicaInfo>>> perNode : replicaInfos.values()) {
-      for (List<ReplicaInfo> perShard : perNode.getOrDefault(collection, Collections.emptyMap()).values()) {
-        for (ReplicaInfo ri : perShard) {
+  public Replica getReplicaInfo(String collection, String coreNode) {
+    for (Map<String, Map<String, List<Replica>>> perNode : replicaInfos.values()) {
+      for (List<Replica> perShard : perNode.getOrDefault(collection, Collections.emptyMap()).values()) {
+        for (Replica ri : perShard) {
           if (ri.getName().equals(coreNode)) {
-            return (ReplicaInfo)ri.clone();
+            return (Replica)ri.clone();
           }
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
index 56bcfd5..bcb7f68 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/NodeMutator.java
@@ -57,7 +57,7 @@ public class NodeMutator {
 
         Collection<Replica> replicas = slice.getReplicas();
         for (Replica replica : replicas) {
-          String rNodeName = replica.getNodeName();
+          String rNodeName = replica.getNode();
           if (rNodeName == null) {
             throw new RuntimeException("Replica without node name! " + replica);
           }
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
index 769be53..3654ea6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ReplicaMutator.java
@@ -72,7 +72,7 @@ public class ReplicaMutator {
 
     Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
     replicaProps.put(key, value);
-    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
+    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getShard());
   }
 
   protected Replica unsetProperty(Replica replica, String key) {
@@ -81,7 +81,7 @@ public class ReplicaMutator {
     if (!replica.containsKey(key)) return replica;
     Map<String, Object> replicaProps = new LinkedHashMap<>(replica.getProperties());
     replicaProps.remove(key);
-    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getSlice());
+    return new Replica(replica.getName(), replicaProps, replica.getCollection(), replica.getShard());
   }
 
   protected Replica setLeader(Replica replica) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
index 6f68d1c..66273f8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
+++ b/solr/core/src/java/org/apache/solr/cloud/rule/ReplicaAssigner.java
@@ -90,8 +90,8 @@ public class ReplicaAssigner {
         DocCollection coll = entry.getValue();
         for (Slice slice : coll.getSlices()) {
           for (Replica replica : slice.getReplicas()) {
-            AtomicInteger count = nodeVsCores.get(replica.getNodeName());
-            if (count == null) nodeVsCores.put(replica.getNodeName(), count = new AtomicInteger());
+            AtomicInteger count = nodeVsCores.get(replica.getNode());
+            if (count == null) nodeVsCores.put(replica.getNode(), count = new AtomicInteger());
             count.incrementAndGet();
           }
         }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreSorter.java b/solr/core/src/java/org/apache/solr/core/CoreSorter.java
index 06e75f2..7b26c3d 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreSorter.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreSorter.java
@@ -102,11 +102,11 @@ public final class CoreSorter implements Comparator<CoreDescriptor> {
       if (shardsVsReplicaCounts.containsKey(sliceName)) continue;
       CountsForEachShard c = new CountsForEachShard(0, 0, 0);
       for (Replica replica : getReplicas(state, coll, cloudDescriptor.getShardId())) {
-        if (replica.getNodeName().equals(myNodeName)) {
+        if (replica.getNode().equals(myNodeName)) {
           c.myReplicas++;
         } else {
           Set<String> liveNodes = state.getLiveNodes();
-          if (liveNodes.contains(replica.getNodeName())) {
+          if (liveNodes.contains(replica.getNode())) {
             c.totalReplicasInLiveNodes++;
           } else {
             c.totalReplicasInDownNodes++;
diff --git a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
index 46c009c..1e13e7f 100644
--- a/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
+++ b/solr/core/src/java/org/apache/solr/handler/IndexFetcher.java
@@ -398,7 +398,7 @@ public class IndexFetcher {
           }
           return IndexFetchResult.LEADER_IS_NOT_ACTIVE;
         }
-        if (!solrCore.getCoreContainer().getZkController().getClusterState().liveNodesContain(replica.getNodeName())) {
+        if (!solrCore.getCoreContainer().getZkController().getClusterState().liveNodesContain(replica.getNode())) {
           if (log.isInfoEnabled()) {
             log.info("Replica {} is leader but it's not hosted on a live node, skipping replication", replica.getName());
           }
diff --git a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
index 9ff0e66..3be40b6 100644
--- a/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
@@ -871,7 +871,7 @@ public class SolrConfigHandler extends RequestHandlerBase implements SolrCoreAwa
         if (replicasMap != null) {
           for (Map.Entry<String, Replica> entry : replicasMap.entrySet()) {
             Replica replica = entry.getValue();
-            if (replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+            if (replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())) {
               activeReplicaCoreUrls.add(replica.getCoreUrl());
             }
           }
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 9b1cf78..7f9a0af 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -1336,7 +1336,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
 
       final Set<String> liveNodes = clusterState.getLiveNodes();
       List<Replica> liveReplicas = slice.getReplicas().stream()
-          .filter(rep -> liveNodes.contains(rep.getNodeName())).collect(Collectors.toList());
+          .filter(rep -> liveNodes.contains(rep.getNode())).collect(Collectors.toList());
       boolean shouldIncreaseReplicaTerms = liveReplicas.stream()
           .noneMatch(rep -> zkShardTerms.registered(rep.getName()) && zkShardTerms.canBecomeLeader(rep.getName()));
       // we won't increase replica's terms if exist a live replica with term equals to leader
@@ -1424,7 +1424,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
                 log.debug("Checking replica status, collection={} replica={} state={}", collectionName,
                     replica.getCoreUrl(), state);
               }
-              if (!n.contains(replica.getNodeName())
+              if (!n.contains(replica.getNode())
                   || !state.equals(Replica.State.ACTIVE.toString())) {
                 replicaNotAliveCnt++;
                 return false;
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
index 5c475a1..607fb8b 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/MetricsHistoryHandler.java
@@ -60,7 +60,6 @@ import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
 import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
@@ -488,7 +487,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
         return;
       }
       // add core-level stats
-      Map<String, Map<String, List<ReplicaInfo>>> infos = nodeStateProvider.getReplicaInfo(node, collTags);
+      Map<String, Map<String, List<Replica>>> infos = nodeStateProvider.getReplicaInfo(node, collTags);
       infos.forEach((coll, shards) -> {
         shards.forEach((sh, replicas) -> {
           String registry = SolrMetricManager.getRegistryName(Group.collection, coll);
@@ -497,7 +496,7 @@ public class MetricsHistoryHandler extends RequestHandlerBase implements Permiss
               .computeIfAbsent(registry, r -> new HashMap<>());
           replicas.forEach(ri -> {
             collTags.forEach(tag -> {
-              double value = ((Number)ri.getVariable(tag, 0.0)).doubleValue();
+              double value = ((Number)ri.get(tag, 0.0)).doubleValue();
               DoubleAdder adder = (DoubleAdder)perReg.computeIfAbsent(tag, t -> new DoubleAdder());
               adder.add(value);
             });
diff --git a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
index 7f114d9..5b7eebc 100644
--- a/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
+++ b/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
@@ -303,7 +303,7 @@ public final class ManagedIndexSchema extends IndexSchema {
             Replica replica = entry.getValue();
             if (!localCoreNodeName.equals(replica.getName()) &&
                 replica.getState() == Replica.State.ACTIVE &&
-                liveNodes.contains(replica.getNodeName())) {
+                liveNodes.contains(replica.getNode())) {
               ZkCoreNodeProps replicaCoreProps = new ZkCoreNodeProps(replica);
               activeReplicaCoreUrls.add(replicaCoreProps.getCoreUrl());
             }
diff --git a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
index 423fd25..fac23ba 100644
--- a/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
+++ b/solr/core/src/java/org/apache/solr/search/join/ScoreJoinQParserPlugin.java
@@ -312,7 +312,7 @@ public class ScoreJoinQParserPlugin extends QParserPlugin {
             "SolrCloud join: To join with a sharded collection, use method=crossCollection.");
 
       for (Replica replica : slice.getReplicas()) {
-        if (replica.getNodeName().equals(nodeName)) {
+        if (replica.getNode().equals(nodeName)) {
           fromReplica = replica.getStr(ZkStateReader.CORE_NAME_PROP);
           // found local replica, but is it Active?
           if (replica.getState() != Replica.State.ACTIVE)
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 2b64cc0..29d3eaf 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -960,7 +960,7 @@ public class HttpSolrCall {
       RandomIterator<Replica> it = new RandomIterator<>(random, replicas);
       while (it.hasNext()) {
         Replica replica = it.next();
-        if (liveNodes.contains(replica.getNodeName()) && replica.getState() == Replica.State.ACTIVE) {
+        if (liveNodes.contains(replica.getNode()) && replica.getState() == Replica.State.ACTIVE) {
           SolrCore core = checkProps(replica);
           if (core != null) return core;
         }
@@ -1062,7 +1062,7 @@ public class HttpSolrCall {
       Collections.shuffle(randomizedReplicas, random);
 
       for (Replica replica : randomizedReplicas) {
-        if (!activeReplicas || (liveNodes.contains(replica.getNodeName())
+        if (!activeReplicas || (liveNodes.contains(replica.getNode())
             && replica.getState() == Replica.State.ACTIVE)) {
 
           if (byCoreName && !origCorename.equals(replica.getStr(CORE_NAME_PROP))) {
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index 8da2df7..71cd9cb 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -698,7 +698,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
               log.debug("skip url:{} cause its term is less than leader", replica.getCoreUrl());
             }
             skippedCoreNodeNames.add(replica.getName());
-          } else if (!clusterState.getLiveNodes().contains(replica.getNodeName()) || replica.getState() == Replica.State.DOWN) {
+          } else if (!clusterState.getLiveNodes().contains(replica.getNode()) || replica.getState() == Replica.State.DOWN) {
             skippedCoreNodeNames.add(replica.getName());
           } else {
             nodes.add(new SolrCmdDistributor.StdNode(new ZkCoreNodeProps(replica), collection, shardId, maxRetriesToFollowers));
@@ -854,7 +854,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
           log.debug("skip url:{} cause its term is less than leader", replica.getCoreUrl());
         }
         skippedCoreNodeNames.add(replica.getName());
-      } else if (!clusterState.getLiveNodes().contains(replica.getNodeName())
+      } else if (!clusterState.getLiveNodes().contains(replica.getNode())
           || replica.getState() == Replica.State.DOWN) {
         skippedCoreNodeNames.add(replica.getName());
       } else {
@@ -879,7 +879,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
                 || coll.getRouter().isTargetSlice(docId, doc, req.getParams(), aslice.getName(), coll))) {
           Replica sliceLeader = aslice.getLeader();
           // slice leader can be null because node/shard is created zk before leader election
-          if (sliceLeader != null && clusterState.liveNodesContain(sliceLeader.getNodeName()))  {
+          if (sliceLeader != null && clusterState.liveNodesContain(sliceLeader.getNode()))  {
             if (nodes == null) nodes = new ArrayList<>();
             ZkCoreNodeProps nodeProps = new ZkCoreNodeProps(sliceLeader);
             nodes.add(new SolrCmdDistributor.StdNode(nodeProps, coll.getName(), aslice.getName()));
diff --git a/solr/core/src/java/org/apache/solr/util/SolrCLI.java b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
index aa68726..922fe63 100755
--- a/solr/core/src/java/org/apache/solr/util/SolrCLI.java
+++ b/solr/core/src/java/org/apache/solr/util/SolrCLI.java
@@ -100,7 +100,6 @@ import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.AutoScalingConfig;
 import org.apache.solr.client.solrj.cloud.autoscaling.Policy;
 import org.apache.solr.client.solrj.cloud.autoscaling.PolicyHelper;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
@@ -1171,14 +1170,14 @@ public class SolrCLI implements CLIO {
           }
           Map<String, Object> paramsMap = new LinkedHashMap<>();
           params.toMap(paramsMap);
-          ReplicaInfo info = simCloudManager.getSimClusterStateProvider().simGetReplicaInfo(
+          Replica info = simCloudManager.getSimClusterStateProvider().simGetReplicaInfo(
               params.get(CollectionAdminParams.COLLECTION), params.get("replica"));
           if (info == null) {
             CLIO.err("Could not find ReplicaInfo for params: " + params);
           } else if (verbose) {
             paramsMap.put("replicaInfo", info);
-          } else if (info.getVariable(Variable.Type.CORE_IDX.tagName) != null) {
-            paramsMap.put(Variable.Type.CORE_IDX.tagName, info.getVariable(Variable.Type.CORE_IDX.tagName));
+          } else if (info.get(Variable.Type.CORE_IDX.tagName) != null) {
+            paramsMap.put(Variable.Type.CORE_IDX.tagName, info.get(Variable.Type.CORE_IDX.tagName));
           }
           if (withSuggestions) {
             perStepOps.add(paramsMap);
diff --git a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
index 5be91da..ff94ea1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
@@ -227,7 +227,7 @@ public class ChaosMonkeyShardSplitTest extends ShardSplitTest {
       Collection<Replica> replicas = slice.getReplicas();
       boolean allActive = true;
       for (Replica replica : replicas) {
-        if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
+        if (!clusterState.liveNodesContain(replica.getNode()) || replica.getState() != Replica.State.ACTIVE) {
           allActive = false;
           break;
         }
diff --git a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtilTest.java b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtilTest.java
index 89e9007..f6f4c07 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtilTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ClusterStateMockUtilTest.java
@@ -47,7 +47,7 @@ public class ClusterStateMockUtilTest extends SolrTestCaseJ4 {
       assertEquals(1, slice1.getReplicas().size());
       Replica replica1 = slice1.getReplica("replica1");
       assertNotNull(replica1);
-      assertEquals("baseUrl1_", replica1.getNodeName());
+      assertEquals("baseUrl1_", replica1.getNode());
       assertEquals("slice1_replica1", replica1.getCoreName());
       assertEquals("http://baseUrl1", replica1.getBaseUrl());
       assertEquals("http://baseUrl1/slice1_replica1/", replica1.getCoreUrl());
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index 35daa44..002df40 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -522,7 +522,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     Replica newReplica = grabNewReplica(response, getCollectionState(collectionName));
     assertEquals(0, response.getStatus());
     assertTrue(response.isSuccess());
-    assertTrue(newReplica.getNodeName().equals(node));
+    assertTrue(newReplica.getNode().equals(node));
 
     // Test DELETEREPLICA
     response = CollectionAdminRequest.deleteReplica(collectionName, "shard1", newReplica.getName())
@@ -657,7 +657,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     // test for replicas not active - SOLR-13882
     DocCollection coll = cluster.getSolrClient().getClusterStateProvider().getClusterState().getCollection(collectionName);
     Replica firstReplica = coll.getSlice("shard1").getReplicas().iterator().next();
-    String firstNode = firstReplica.getNodeName();
+    String firstNode = firstReplica.getNode();
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       if (jetty.getNodeName().equals(firstNode)) {
         cluster.stopJettySolrRunner(jetty);
diff --git a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
index cd6edd9..919f96f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -153,7 +153,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     assertEquals(1, coll.getNumPullReplicas().intValue()); // per-shard
     assertTrue("nodeSet didn't work?",
         coll.getSlices().stream().flatMap(s -> s.getReplicas().stream())
-            .map(Replica::getNodeName).allMatch(createNode::equals));
+            .map(Replica::getNode).allMatch(createNode::equals));
 
     // Test Alias metadata:
     Aliases aliases = cluster.getSolrClient().getZkStateReader().getAliases();
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
index 84a5ece..0252a10 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteNodeTest.java
@@ -84,7 +84,7 @@ public class DeleteNodeTest extends SolrCloudTestCase {
         boolean hasOtherNonPullReplicas = false;
         for (Replica r: slice.getReplicas()) {
           if (!r.getName().equals(replica.getName()) &&
-              !r.getNodeName().equals(node2bdecommissioned) &&
+              !r.getNode().equals(node2bdecommissioned) &&
               r.getType() != Replica.Type.PULL) {
             hasOtherNonPullReplicas = true;
             break;
diff --git a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
index 8340458..c9d732c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DeleteReplicaTest.java
@@ -239,7 +239,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
     ZkNodeProps m = new ZkNodeProps(
         Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
         ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
-        ZkStateReader.NODE_NAME_PROP, replica.getNodeName(),
+        ZkStateReader.NODE_NAME_PROP, replica.getNode(),
         ZkStateReader.COLLECTION_PROP, collectionName,
         ZkStateReader.CORE_NODE_NAME_PROP, replica.getName(),
         ZkStateReader.BASE_URL_PROP, replica.getBaseUrl());
@@ -305,7 +305,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
           ZkNodeProps m = new ZkNodeProps(
               Overseer.QUEUE_OPERATION, OverseerAction.DELETECORE.toLower(),
               ZkStateReader.CORE_NAME_PROP, replica1.getCoreName(),
-              ZkStateReader.NODE_NAME_PROP, replica1.getNodeName(),
+              ZkStateReader.NODE_NAME_PROP, replica1.getNode(),
               ZkStateReader.COLLECTION_PROP, collectionName,
               ZkStateReader.CORE_NODE_NAME_PROP, replica1.getName(),
               ZkStateReader.BASE_URL_PROP, replica1.getBaseUrl());
@@ -390,7 +390,7 @@ public class DeleteReplicaTest extends SolrCloudTestCase {
   private JettySolrRunner getJettyForReplica(Replica replica) {
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       String nodeName = jetty.getNodeName();
-      if (nodeName != null && nodeName.equals(replica.getNodeName())) return jetty;
+      if (nodeName != null && nodeName.equals(replica.getNode())) return jetty;
     }
     throw new IllegalArgumentException("Can not find jetty for replica "+ replica);
   }
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
index d9a1a09..027af11 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribDocExpirationUpdateProcessorTest.java
@@ -232,7 +232,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
       boolean firstReplica = true;
       for (Replica replica : shard) {
         coresCompared++;
-        assertEquals(shard.getName(), replica.getSlice()); // sanity check
+        assertEquals(shard.getName(), replica.getShard()); // sanity check
         final String core = replica.getCoreName();
         final ReplicaData initData = initReplicaData.get(core);
         final ReplicaData finalData = finalReplicaData.get(core);
@@ -301,7 +301,7 @@ public class DistribDocExpirationUpdateProcessorTest extends SolrCloudTestCase {
                                   "rows", "0",
                                   "_trace", "counting_docs"))).process(client).getResults().getNumFound();
 
-        final ReplicaData data = new ReplicaData(replica.getSlice(),coreName,(Long)version,numDocs);
+        final ReplicaData data = new ReplicaData(replica.getShard(),coreName,(Long)version,numDocs);
         log.info("{}", data);
         results.put(coreName, data);
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/DistribJoinFromCollectionTest.java b/solr/core/src/test/org/apache/solr/cloud/DistribJoinFromCollectionTest.java
index 805e013..a023962 100644
--- a/solr/core/src/test/org/apache/solr/cloud/DistribJoinFromCollectionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/DistribJoinFromCollectionTest.java
@@ -92,7 +92,7 @@ public class DistribJoinFromCollectionTest extends SolrCloudTestCase{
     ClusterState cs = zkStateReader.getClusterState();
     for (Slice slice : cs.getCollection(toColl).getActiveSlices())
       for (Replica replica : slice.getReplicas())
-        nodeSet.add(replica.getNodeName());
+        nodeSet.add(replica.getNode());
     assertTrue(nodeSet.size() > 0);
 
     // deploy the "from" collection to all nodes where the "to" collection exists
diff --git a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
index 68b2b43..05299a0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/ForceLeaderTest.java
@@ -106,7 +106,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
 
       int numReplicasOnLiveNodes = 0;
       for (Replica rep : clusterState.getCollection(testCollectionName).getSlice(SHARD1).getReplicas()) {
-        if (clusterState.getLiveNodes().contains(rep.getNodeName())) {
+        if (clusterState.getLiveNodes().contains(rep.getNode())) {
           numReplicasOnLiveNodes++;
         }
       }
@@ -202,7 +202,7 @@ public class ForceLeaderTest extends HttpPartitionTest {
 
     // Kill the leader
     if (log.isInfoEnabled()) {
-      log.info("Killing leader for shard1 of {} on node {}", collectionName, leader.getNodeName());
+      log.info("Killing leader for shard1 of {} on node {}", collectionName, leader.getNode());
     }
     leaderJetty.stop();
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
index 6ca5533..3be488d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/HttpPartitionTest.java
@@ -399,7 +399,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
 
     Replica leader =
         cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
-    String leaderNode = leader.getNodeName();
+    String leaderNode = leader.getNode();
     assertNotNull("Could not find leader for shard1 of "+
         testCollectionName+"; clusterState: "+printClusterStateInfo(testCollectionName), leader);
     JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
@@ -582,7 +582,7 @@ public class HttpPartitionTest extends AbstractFullDistribZkTestBase {
   }
 
   protected int getReplicaPort(Replica replica) {
-    String replicaNode = replica.getNodeName();    
+    String replicaNode = replica.getNode();
     String tmp = replicaNode.substring(replicaNode.indexOf(':')+1);
     if (tmp.indexOf('_') != -1)
       tmp = tmp.substring(0,tmp.indexOf('_'));
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
index fc340e0..accfdf7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailoverAfterPartitionTest.java
@@ -106,7 +106,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
         
     Replica leader = 
         cloudClient.getZkStateReader().getLeaderRetry(testCollectionName, "shard1");
-    String leaderNode = leader.getNodeName();
+    String leaderNode = leader.getNode();
     assertNotNull("Could not find leader for shard1 of "+
       testCollectionName+"; clusterState: "+printClusterStateInfo(testCollectionName), leader);
     JettySolrRunner leaderJetty = getJettyOnPort(getReplicaPort(leader));
@@ -130,7 +130,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
   
     Thread.sleep(sleepMsBeforeHealPartition);
     
-    String shouldNotBeNewLeaderNode = notLeaders.get(0).getNodeName();
+    String shouldNotBeNewLeaderNode = notLeaders.get(0).getNode();
 
     //chaosMonkey.expireSession(leaderJetty);
     // kill the leader
@@ -156,7 +156,7 @@ public class LeaderFailoverAfterPartitionTest extends HttpPartitionTest {
     assertTrue("Expected node "+shouldNotBeNewLeaderNode+
         " to NOT be the new leader b/c it was out-of-sync with the old leader! ClusterState: "+
         printClusterStateInfo(testCollectionName),
-        !shouldNotBeNewLeaderNode.equals(newLeader.getNodeName()));
+        !shouldNotBeNewLeaderNode.equals(newLeader.getNode()));
     
     proxy0.reopen();
     
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
index 8cb40dd..16f392a 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderFailureAfterFreshStartTest.java
@@ -221,7 +221,7 @@ public class LeaderFailureAfterFreshStartTest extends AbstractFullDistribZkTestB
           .collect(Collectors.toList());
 
       for (Replica replica : replicasToCheck) {
-        if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
+        if (!clusterState.liveNodesContain(replica.getNode()) || replica.getState() != Replica.State.ACTIVE) {
           allActive = false;
           break;
         }
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
index 9c5ff4d..87fff33 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
@@ -96,8 +96,8 @@ public class LeaderTragicEventTest extends SolrCloudTestCase {
       });
       ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), collection, 120000);
       Slice shard = getCollectionState(collection).getSlice("shard1");
-      assertNotSame(shard.getLeader().getNodeName(), oldLeader.getNodeName());
-      assertEquals(getNonLeader(shard).getNodeName(), oldLeader.getNodeName());
+      assertNotSame(shard.getLeader().getNode(), oldLeader.getNode());
+      assertEquals(getNonLeader(shard).getNode(), oldLeader.getNode());
 
       for (String id : addedIds) {
         assertNotNull(cluster.getSolrClient().getById(collection,id));
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
index fb0cbbd..77893fe 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
@@ -144,7 +144,7 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
       if (newLeader == null) {
         return false;
       }
-      return newLeader.getNodeName().equals(cluster.getJettySolrRunner(1).getNodeName());
+      return newLeader.getNode().equals(cluster.getJettySolrRunner(1).getNodeName());
     });
 
     try (ZkShardTerms zkShardTerms = new ZkShardTerms(collectionName, "shard1", cluster.getZkClient())) {
@@ -183,7 +183,7 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
     
     waitForState("Timeout waiting for 1x2 collection", collectionName, clusterShape(1, 2));
     Replica replica1 = getCollectionState(collectionName).getSlice("shard1")
-        .getReplicas(replica -> replica.getNodeName().equals(cluster.getJettySolrRunner(1).getNodeName())).get(0);
+        .getReplicas(replica -> replica.getNode().equals(cluster.getJettySolrRunner(1).getNodeName())).get(0);
 
     CollectionAdminRequest.addReplicaToShard(collectionName, "shard1")
         .setNode(cluster.getJettySolrRunner(2).getNodeName())
@@ -193,7 +193,7 @@ public class LeaderVoteWaitTimeoutTest extends SolrCloudTestCase {
     
     waitForState("Timeout waiting for 1x3 collection", collectionName, clusterShape(1, 3));
     Replica replica2 = getCollectionState(collectionName).getSlice("shard1")
-        .getReplicas(replica -> replica.getNodeName().equals(cluster.getJettySolrRunner(2).getNodeName())).get(0);
+        .getReplicas(replica -> replica.getNode().equals(cluster.getJettySolrRunner(2).getNodeName())).get(0);
 
     cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "1"));
     cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2"));
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSFailoverTest.java
index f66029b..534bb33 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaHDFSFailoverTest.java
@@ -102,13 +102,13 @@ public class MoveReplicaHDFSFailoverTest extends SolrCloudTestCase {
     docCollection = zkStateReader.getClusterState().getCollection(coll);
     assertEquals(1, docCollection.getSlice("shard1").getReplicas().size());
     Replica newReplica = docCollection.getReplicas().iterator().next();
-    assertEquals(newReplica.getNodeName(), cluster.getJettySolrRunner(1).getNodeName());
+    assertEquals(newReplica.getNode(), cluster.getJettySolrRunner(1).getNodeName());
     assertTrue(newReplica.getStr("ulogDir"), newReplica.getStr("ulogDir").equals(ulogDir) || newReplica.getStr("ulogDir").equals(ulogDir+'/'));
     assertTrue(newReplica.getStr("dataDir"),newReplica.getStr("dataDir").equals(dataDir) || newReplica.getStr("dataDir").equals(dataDir+'/'));
 
     assertEquals(replica.getName(), newReplica.getName());
     assertEquals(replica.getCoreName(), newReplica.getCoreName());
-    assertFalse(replica.getNodeName().equals(newReplica.getNodeName()));
+    assertFalse(replica.getNode().equals(newReplica.getNode()));
     final int numDocs = 100;
     addDocs(coll, numDocs);  // indexed but not committed
 
@@ -124,7 +124,7 @@ public class MoveReplicaHDFSFailoverTest extends SolrCloudTestCase {
     docCollection = zkStateReader.getClusterState().getCollection(coll);
     assertEquals(1, docCollection.getReplicas().size());
     newReplica = docCollection.getReplicas().iterator().next();
-    assertEquals(newReplica.getNodeName(), cluster.getJettySolrRunner(0).getNodeName());
+    assertEquals(newReplica.getNode(), cluster.getJettySolrRunner(0).getNodeName());
     assertTrue(newReplica.getStr("ulogDir"), newReplica.getStr("ulogDir").equals(ulogDir) || newReplica.getStr("ulogDir").equals(ulogDir+'/'));
     assertTrue(newReplica.getStr("dataDir"),newReplica.getStr("dataDir").equals(dataDir) || newReplica.getStr("dataDir").equals(dataDir+'/'));
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
index f4191d8..812b8e0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
@@ -116,7 +116,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
     Collections.shuffle(l, random());
     String targetNode = null;
     for (String node : liveNodes) {
-      if (!replica.getNodeName().equals(node)) {
+      if (!replica.getNode().equals(node)) {
         targetNode = node;
         break;
       }
@@ -129,7 +129,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
       }
     }
 
-    int sourceNumCores = getNumOfCores(cloudClient, replica.getNodeName(), coll, replica.getType().name());
+    int sourceNumCores = getNumOfCores(cloudClient, replica.getNode(), coll, replica.getType().name());
     int targetNumCores = getNumOfCores(cloudClient, targetNode, coll, replica.getType().name());
 
     CollectionAdminRequest.MoveReplica moveReplica = createMoveReplicaRequest(coll, replica, targetNode);
@@ -149,7 +149,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
       Thread.sleep(500);
     }
     assertTrue(success);
-    assertEquals("should be one less core on the source node!", sourceNumCores - 1, getNumOfCores(cloudClient, replica.getNodeName(), coll, replica.getType().name()));
+    assertEquals("should be one less core on the source node!", sourceNumCores - 1, getNumOfCores(cloudClient, replica.getNode(), coll, replica.getType().name()));
     assertEquals("should be one more core on target node!", targetNumCores + 1, getNumOfCores(cloudClient, targetNode, coll, replica.getType().name()));
     // wait for recovery
     boolean recovered = false;
@@ -161,7 +161,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
       boolean hasLeaders = true;
       if (replicas != null && !replicas.isEmpty()) {
         for (Replica r : replicas) {
-          if (!r.getNodeName().equals(targetNode)) {
+          if (!r.getNode().equals(targetNode)) {
             continue;
           }
           if (!r.isActive(Collections.singleton(targetNode))) {
@@ -194,7 +194,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
     moveReplica = createMoveReplicaRequest(coll, replica, targetNode, shardId);
     moveReplica.setInPlaceMove(inPlaceMove);
     moveReplica.process(cloudClient);
-    checkNumOfCores(cloudClient, replica.getNodeName(), coll, sourceNumCores);
+    checkNumOfCores(cloudClient, replica.getNode(), coll, sourceNumCores);
     // wait for recovery
     recovered = false;
     for (int i = 0; i < 300; i++) {
@@ -205,10 +205,10 @@ public class MoveReplicaTest extends SolrCloudTestCase {
       boolean hasLeaders = true;
       if (replicas != null && !replicas.isEmpty()) {
         for (Replica r : replicas) {
-          if (!r.getNodeName().equals(replica.getNodeName())) {
+          if (!r.getNode().equals(replica.getNode())) {
             continue;
           }
-          if (!r.isActive(Collections.singleton(replica.getNodeName()))) {
+          if (!r.isActive(Collections.singleton(replica.getNode()))) {
             log.info("Not active yet: {}", r);
             allActive = false;
           }
@@ -262,14 +262,14 @@ public class MoveReplicaTest extends SolrCloudTestCase {
     int count = 10;
     do {
       replica = getRandomReplica(coll, cloudClient);
-    } while (!replica.getNodeName().equals(overseerLeader) && count-- > 0);
+    } while (!replica.getNode().equals(overseerLeader) && count-- > 0);
     assertNotNull("could not find non-overseer replica???", replica);
     Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
     ArrayList<String> l = new ArrayList<>(liveNodes);
     Collections.shuffle(l, random());
     String targetNode = null;
     for (String node : liveNodes) {
-      if (!replica.getNodeName().equals(node) && !overseerLeader.equals(node)) {
+      if (!replica.getNode().equals(node) && !overseerLeader.equals(node)) {
         targetNode = node;
         break;
       }
@@ -309,7 +309,7 @@ public class MoveReplicaTest extends SolrCloudTestCase {
   }
 
   private CollectionAdminRequest.MoveReplica createMoveReplicaRequest(String coll, Replica replica, String targetNode, String shardId) {
-    return new CollectionAdminRequest.MoveReplica(coll, shardId, targetNode, replica.getNodeName());
+    return new CollectionAdminRequest.MoveReplica(coll, shardId, targetNode, replica.getNode());
   }
 
   private CollectionAdminRequest.MoveReplica createMoveReplicaRequest(String coll, Replica replica, String targetNode) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
index c20e3eb..6c7ebccd 100644
--- a/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/PeerSyncReplicationTest.java
@@ -354,7 +354,7 @@ public class PeerSyncReplicationTest extends AbstractFullDistribZkTestBase {
               .collect(Collectors.toList());
 
       for (Replica replica : replicasToCheck) {
-        if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
+        if (!clusterState.liveNodesContain(replica.getNode()) || replica.getState() != Replica.State.ACTIVE) {
           allActive = false;
           break;
         }
diff --git a/solr/core/src/test/org/apache/solr/cloud/RoutingToNodesWithPropertiesTest.java b/solr/core/src/test/org/apache/solr/cloud/RoutingToNodesWithPropertiesTest.java
index 4f9cbfc..2624da7 100644
--- a/solr/core/src/test/org/apache/solr/cloud/RoutingToNodesWithPropertiesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/RoutingToNodesWithPropertiesTest.java
@@ -96,9 +96,9 @@ public class RoutingToNodesWithPropertiesTest extends SolrCloudTestCase {
       int numReplicaInZone1 = 0;
       int numReplicaInZone2 = 0;
       for (Replica replica : slice.getReplicas()) {
-        if (zone1Nodes.contains(replica.getNodeName()))
+        if (zone1Nodes.contains(replica.getNode()))
           numReplicaInZone1++;
-        if (zone2Nodes.contains(replica.getNodeName()))
+        if (zone2Nodes.contains(replica.getNode()))
           numReplicaInZone2++;
       }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
index 3fe4d7d..1037d3d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
@@ -361,9 +361,9 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
             if (slice.getState() == Slice.State.ACTIVE) {
               Collection<Replica> replicas = slice.getReplicas();
               for (Replica replica : replicas) {
-                if (nodeNames.contains(replica.getNodeName())) {
+                if (nodeNames.contains(replica.getNode())) {
                   boolean live = clusterState.liveNodesContain(replica
-                      .getNodeName());
+                      .getNode());
                   if (live) {
                     success = false;
                   }
@@ -396,7 +396,7 @@ public class SharedFSAutoReplicaFailoverTest extends AbstractFullDistribZkTestBa
         for (Slice slice : slices) {
           int count = 0;
           for (Replica replica : slice.getReplicas()) {
-            if (replica.getState() == Replica.State.ACTIVE && clusterState.liveNodesContain(replica.getNodeName())) {
+            if (replica.getState() == Replica.State.ACTIVE && clusterState.liveNodesContain(replica.getNode())) {
               count++;
             }
           }
diff --git a/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java b/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
index c5a249f..20f85a5 100644
--- a/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/SyncSliceTest.java
@@ -224,7 +224,7 @@ public class SyncSliceTest extends AbstractFullDistribZkTestBase {
       Collection<Replica> replicas = slice.getReplicas();
       boolean allActive = true;
       for (Replica replica : replicas) {
-        if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
+        if (!clusterState.liveNodesContain(replica.getNode()) || replica.getState() != Replica.State.ACTIVE) {
           allActive = false;
           break;
         }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
index 61e6dbd..04ed6a4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudConsistency.java
@@ -127,7 +127,7 @@ public class TestCloudConsistency extends SolrCloudTestCase {
     addDocs(collectionName, 3, 1);
 
     final Replica oldLeader = getCollectionState(collectionName).getSlice("shard1").getLeader();
-    assertEquals(cluster.getJettySolrRunner(0).getNodeName(), oldLeader.getNodeName());
+    assertEquals(cluster.getJettySolrRunner(0).getNodeName(), oldLeader.getNode());
 
     if (onRestart) {
       addDocToWhenOtherReplicasAreDown(collectionName, oldLeader, 4);
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
index e895fe3..fd2b41b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudDeleteByQuery.java
@@ -146,9 +146,9 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
       String shardName = slice.getName();
       Replica leader = slice.getLeader();
       assertNotNull("slice has null leader: " + slice.toString(), leader);
-      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNodeName());
-      String leaderUrl = urlMap.remove(leader.getNodeName());
-      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNodeName(),
+      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNode());
+      String leaderUrl = urlMap.remove(leader.getNode());
+      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNode(),
                     leaderUrl);
       assertEquals("expected two total replicas for: " + slice.getName(),
                    2, slice.getReplicas().size());
@@ -157,8 +157,8 @@ public class TestCloudDeleteByQuery extends SolrCloudTestCase {
       
       for (Replica replica : slice.getReplicas()) {
         if ( ! replica.equals(leader)) {
-          passiveUrl = urlMap.remove(replica.getNodeName());
-          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNodeName(),
+          passiveUrl = urlMap.remove(replica.getNode());
+          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNode(),
                         passiveUrl);
         }
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
index 8167f2c..d77cd69 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudRecovery2.java
@@ -123,7 +123,7 @@ public class TestCloudRecovery2 extends SolrCloudTestCase {
     node1.stop();
     waitForState("", COLLECTION, (liveNodes, collectionState) -> {
       Replica leader = collectionState.getLeader("shard1");
-      return leader != null && leader.getNodeName().equals(node2.getNodeName());
+      return leader != null && leader.getNode().equals(node2.getNodeName());
     });
 
     node1.start();
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java b/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
index 247313e..e35ce2d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCloudSearcherWarming.java
@@ -195,7 +195,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
     // but clusterShape will also return true if replica is not live -- which we don't want
     CollectionStatePredicate collectionStatePredicate = (liveNodes, collectionState) -> {
       for (Replica r : collectionState.getReplicas()) {
-        if (r.getNodeName().equals(oldNodeName.get())) {
+        if (r.getNode().equals(oldNodeName.get())) {
           return r.getState() == Replica.State.DOWN;
         }
       }
@@ -262,7 +262,7 @@ public class TestCloudSearcherWarming extends SolrCloudTestCase {
               if (log.isInfoEnabled()) {
                 log.info("Checking node: {}", jettySolrRunner.getNodeName());
               }
-              if (jettySolrRunner.getNodeName().equals(replica.getNodeName())) {
+              if (jettySolrRunner.getNodeName().equals(replica.getNode())) {
                 SolrDispatchFilter solrDispatchFilter = jettySolrRunner.getSolrDispatchFilter();
                 try (SolrCore core = solrDispatchFilter.getCores().getCore(coreName)) {
                   if (core.getSolrConfig().useColdSearcher) {
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
index 5e06c28..6f5ff16 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestPullReplica.java
@@ -593,10 +593,10 @@ public class TestPullReplica extends SolrCloudTestCase {
         if (r.getState() != Replica.State.DOWN && r.getState() != Replica.State.ACTIVE) {
           return false;
         }
-        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNodeName())) {
+        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNode())) {
           return false;
         }
-        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNodeName())) {
+        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNode())) {
           return false;
         }
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
index 8a0a874..b35f670 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestRebalanceLeaders.java
@@ -530,7 +530,7 @@ public class TestRebalanceLeaders extends SolrCloudTestCase {
 
       for (Slice slice : docCollection.getSlices()) {
         for (Replica rep : slice.getReplicas()) {
-          if (downJettyNodes.contains(rep.getNodeName()) == false) {
+          if (downJettyNodes.contains(rep.getNode()) == false) {
             continue; // We are on a live node
           }
           // A replica on an allegedly down node is reported as active.
@@ -587,7 +587,7 @@ public class TestRebalanceLeaders extends SolrCloudTestCase {
       Replica changedRep = null;
       int livePos = Integer.MAX_VALUE;
       for (Replica rep : slice.getReplicas()) {
-        int pos = liveNodes.indexOf(rep.getNodeName());
+        int pos = liveNodes.indexOf(rep.getNode());
         if (pos >= 0 && pos < livePos) {
           livePos = pos;
           changedRep = rep;
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestShortCircuitedRequests.java b/solr/core/src/test/org/apache/solr/cloud/TestShortCircuitedRequests.java
index 08e0eb5..9f29c4d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestShortCircuitedRequests.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestShortCircuitedRequests.java
@@ -46,7 +46,7 @@ public class TestShortCircuitedRequests extends AbstractFullDistribZkTestBase {
 
     // query shard3 directly with _route_=a! so that we trigger the short circuited request path
     Replica shard3 = cloudClient.getZkStateReader().getClusterState().getCollection(DEFAULT_COLLECTION).getLeader("shard3");
-    String nodeName = shard3.getNodeName();
+    String nodeName = shard3.getNode();
     SolrClient shard3Client = getClient(nodeName);
     QueryResponse response = shard3Client.query(new SolrQuery("*:*").add(ShardParams._ROUTE_, "a!").add(ShardParams.SHARDS_INFO, "true"));
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestStressInPlaceUpdates.java b/solr/core/src/test/org/apache/solr/cloud/TestStressInPlaceUpdates.java
index 4230ec1..4f2be63 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestStressInPlaceUpdates.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestStressInPlaceUpdates.java
@@ -594,7 +594,7 @@ public class TestStressInPlaceUpdates extends AbstractFullDistribZkTestBase {
     leader = shard1.getLeader();
 
     for (int i = 0; i < clients.size(); i++) {
-      String leaderBaseUrl = zkStateReader.getBaseUrlForNodeName(leader.getNodeName());
+      String leaderBaseUrl = zkStateReader.getBaseUrlForNodeName(leader.getNode());
       if (((HttpSolrClient) clients.get(i)).getBaseURL().startsWith(leaderBaseUrl))
         return clients.get(i);
     }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplayVsRecovery.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplayVsRecovery.java
index 8b9f1cf..ab8e67c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplayVsRecovery.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplayVsRecovery.java
@@ -148,7 +148,7 @@ public class TestTlogReplayVsRecovery extends SolrCloudTestCase {
     waitForState("Timeout waiting for 1x2 collection", COLLECTION, clusterShape(1, 2));
     
     final Replica leader = getCollectionState(COLLECTION).getSlice("shard1").getLeader();
-    assertEquals("Sanity check failed", NODE0.getNodeName(), leader.getNodeName());
+    assertEquals("Sanity check failed", NODE0.getNodeName(), leader.getNode());
 
     log.info("Add and commit {} docs...", committedDocs);
     addDocs(true, committedDocs, 1);
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
index d68f46b..e80d27f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTlogReplica.java
@@ -677,7 +677,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
           if (leader == null || !leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
             return false;
           }
-          return oldLeaderJetty == null || !leader.getNodeName().equals(oldLeaderJetty.getNodeName());
+          return oldLeaderJetty == null || !leader.getNode().equals(oldLeaderJetty.getNodeName());
         }
     );
   }
@@ -829,10 +829,10 @@ public class TestTlogReplica extends SolrCloudTestCase {
         if (r.getState() != Replica.State.DOWN && r.getState() != Replica.State.ACTIVE) {
           return false;
         }
-        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNodeName())) {
+        if (r.getState() == Replica.State.DOWN && liveNodes.contains(r.getNode())) {
           return false;
         }
-        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNodeName())) {
+        if (r.getState() == Replica.State.ACTIVE && !liveNodes.contains(r.getNode())) {
           return false;
         }
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
index 0fe45c9..bd5ac26 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestTolerantUpdateProcessorCloud.java
@@ -136,9 +136,9 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       String shardName = slice.getName();
       Replica leader = slice.getLeader();
       assertNotNull("slice has null leader: " + slice.toString(), leader);
-      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNodeName());
-      String leaderUrl = urlMap.remove(leader.getNodeName());
-      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNodeName(),
+      assertNotNull("slice leader has null node name: " + slice.toString(), leader.getNode());
+      String leaderUrl = urlMap.remove(leader.getNode());
+      assertNotNull("could not find URL for " + shardName + " leader: " + leader.getNode(),
                     leaderUrl);
       assertEquals("expected two total replicas for: " + slice.getName(),
                    2, slice.getReplicas().size());
@@ -147,8 +147,8 @@ public class TestTolerantUpdateProcessorCloud extends SolrCloudTestCase {
       
       for (Replica replica : slice.getReplicas()) {
         if ( ! replica.equals(leader)) {
-          passiveUrl = urlMap.remove(replica.getNodeName());
-          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNodeName(),
+          passiveUrl = urlMap.remove(replica.getNode());
+          assertNotNull("could not find URL for " + shardName + " replica: " + replica.getNode(),
                         passiveUrl);
         }
       }
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestUtilizeNode.java b/solr/core/src/test/org/apache/solr/cloud/TestUtilizeNode.java
index 68e1e88..9692ace 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestUtilizeNode.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestUtilizeNode.java
@@ -173,7 +173,7 @@ public class TestUtilizeNode extends SolrCloudTestCase {
     List<Replica> results = new ArrayList<>(3);
     if (collection != null) {
       collection.forEachReplica((s, replica) -> {
-        if (replica.getNodeName().equals(jettyNode.getNodeName())) {
+        if (replica.getNode().equals(jettyNode.getNodeName())) {
           results.add(replica);
         }
       });
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestWithCollection.java b/solr/core/src/test/org/apache/solr/cloud/TestWithCollection.java
index 2ee47f4..1d0aad4 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestWithCollection.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestWithCollection.java
@@ -155,7 +155,7 @@ public class TestWithCollection extends SolrCloudTestCase {
     assertNotNull(c1);
     assertEquals(abc, c1.getStr(WITH_COLLECTION));
     Replica replica = c1.getReplicas().get(0);
-    String nodeName = replica.getNodeName();
+    String nodeName = replica.getNode();
 
     assertEquals(chosenNode, nodeName);
   }
@@ -242,7 +242,7 @@ public class TestWithCollection extends SolrCloudTestCase {
     DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
     DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
 
-    assertTrue(collection.getReplicas().stream().noneMatch(replica -> withCollection.getReplicas(replica.getNodeName()).isEmpty()));
+    assertTrue(collection.getReplicas().stream().noneMatch(replica -> withCollection.getReplicas(replica.getNode()).isEmpty()));
   }
 
   @Test
@@ -273,7 +273,7 @@ public class TestWithCollection extends SolrCloudTestCase {
         .process(solrClient);
 
     DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
-    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
+    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNode());
 
 //    zkClient().printLayoutToStdOut();
 
@@ -283,8 +283,8 @@ public class TestWithCollection extends SolrCloudTestCase {
     DocCollection withCollection = solrClient.getZkStateReader().getClusterState().getCollection(abc);
 
     assertTrue(collection.getReplicas().stream().noneMatch(
-        replica -> withCollection.getReplicas(replica.getNodeName()) == null
-            || withCollection.getReplicas(replica.getNodeName()).isEmpty()));
+        replica -> withCollection.getReplicas(replica.getNode()) == null
+            || withCollection.getReplicas(replica.getNode()).isEmpty()));
   }
 
   @Test
@@ -333,8 +333,8 @@ public class TestWithCollection extends SolrCloudTestCase {
     collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz); // refresh
     DocCollection withCollectionRefreshed = solrClient.getZkStateReader().getClusterState().getCollection(abc); // refresh
     assertTrue(collection.getReplicas().stream().noneMatch(
-        replica -> withCollectionRefreshed.getReplicas(replica.getNodeName()) == null
-            || withCollectionRefreshed.getReplicas(replica.getNodeName()).isEmpty()));
+        replica -> withCollectionRefreshed.getReplicas(replica.getNode()) == null
+            || withCollectionRefreshed.getReplicas(replica.getNode()).isEmpty()));
   }
 
   @Test
@@ -366,7 +366,7 @@ public class TestWithCollection extends SolrCloudTestCase {
         .process(solrClient);
 
     DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
-    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
+    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNode());
 
     String otherNode = null;
     for (JettySolrRunner jettySolrRunner : cluster.getJettySolrRunners()) {
@@ -393,8 +393,8 @@ public class TestWithCollection extends SolrCloudTestCase {
 
     // sanity check that the failed move operation didn't actually change our co-location guarantees
     assertTrue(collection.getReplicas().stream().noneMatch(
-        replica -> withCollectionRefreshed.getReplicas(replica.getNodeName()) == null
-            || withCollectionRefreshed.getReplicas(replica.getNodeName()).isEmpty()));
+        replica -> withCollectionRefreshed.getReplicas(replica.getNode()) == null
+            || withCollectionRefreshed.getReplicas(replica.getNode()).isEmpty()));
   }
 
   /**
@@ -429,7 +429,7 @@ public class TestWithCollection extends SolrCloudTestCase {
         .process(solrClient);
 
     DocCollection collection = solrClient.getZkStateReader().getClusterState().getCollection(xyz);
-    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNodeName());
+    assertEquals(chosenNode, collection.getReplicas().iterator().next().getNode());
 
     String setTriggerCommand = "{" +
         "'set-trigger' : {" +
@@ -477,7 +477,7 @@ public class TestWithCollection extends SolrCloudTestCase {
 
     // assert that the replica of xyz collection was not moved
     assertNotNull(collection.getReplica(xyzReplica.getName()));
-    assertEquals(chosenNode, collection.getReplicas().get(0).getNodeName());
+    assertEquals(chosenNode, collection.getReplicas().get(0).getNode());
 
     // add an extra replica of xyz collection -- this should be placed on the 'otherNode'
     addReplica = CollectionAdminRequest.addReplicaToShard(xyz, "shard1");
@@ -592,12 +592,12 @@ public class TestWithCollection extends SolrCloudTestCase {
   private void assertColocated(DocCollection collection, String noneOnNode, DocCollection withCollection) {
     // sanity check
     assertTrue(collection.getReplicas().stream().noneMatch(
-        replica -> withCollection.getReplicas(replica.getNodeName()) == null
-            || withCollection.getReplicas(replica.getNodeName()).isEmpty()));
+        replica -> withCollection.getReplicas(replica.getNode()) == null
+            || withCollection.getReplicas(replica.getNode()).isEmpty()));
 
     if (noneOnNode != null) {
       assertTrue(collection.getReplicas().stream().noneMatch(
-          replica -> noneOnNode.equals(replica.getNodeName())));
+          replica -> noneOnNode.equals(replica.getNode())));
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
index 74bcd75..b806c7d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/AbstractCloudBackupRestoreTestCase.java
@@ -372,7 +372,7 @@ public abstract class AbstractCloudBackupRestoreTestCase extends SolrCloudTestCa
 
     Map<String, Integer> numReplicasByNodeName = new HashMap<>();
     restoreCollection.getReplicas().forEach(x -> {
-      numReplicasByNodeName.put(x.getNodeName(), numReplicasByNodeName.getOrDefault(x.getNodeName(), 0) + 1);
+      numReplicasByNodeName.put(x.getNode(), numReplicasByNodeName.getOrDefault(x.getNode(), 0) + 1);
     });
     numReplicasByNodeName.forEach((k, v) -> {
       assertTrue("Node " + k + " has " + v + " replicas. Expected num replicas : " + restoreMaxShardsPerNode
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionTooManyReplicasTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionTooManyReplicasTest.java
index 6489f1c..7ae5f02 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionTooManyReplicasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionTooManyReplicasTest.java
@@ -90,7 +90,7 @@ public class CollectionTooManyReplicasTest extends SolrCloudTestCase {
     Slice slice = collectionState.getSlice("shard1");
     Replica replica = getRandomReplica(slice, r -> r.getCoreName().equals("bogus2"));
     assertNotNull("Should have found a replica named 'bogus2'", replica);
-    assertEquals("Replica should have been put on correct node", nodeName, replica.getNodeName());
+    assertEquals("Replica should have been put on correct node", nodeName, replica.getNode());
 
     // Shard1 should have 2 replicas
     assertEquals("There should be 3 replicas for shard 1", 3, slice.getReplicas().size());
@@ -216,7 +216,7 @@ public class CollectionTooManyReplicasTest extends SolrCloudTestCase {
 
   private List<String> getAllNodeNames(String collectionName) throws KeeperException, InterruptedException {
     DocCollection state = getCollectionState(collectionName);
-    return state.getReplicas().stream().map(Replica::getNodeName).distinct().collect(Collectors.toList());
+    return state.getReplicas().stream().map(Replica::getNode).distinct().collect(Collectors.toList());
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIAsyncDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIAsyncDistributedZkTest.java
index a9fdcae..43b176c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIAsyncDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIAsyncDistributedZkTest.java
@@ -191,7 +191,7 @@ public class CollectionsAPIAsyncDistributedZkTest extends SolrCloudTestCase {
     Slice shard1 = client.getZkStateReader().getClusterState().getCollection(collection).getSlice("shard1");
     Replica replica = shard1.getReplicas().iterator().next();
     for (String liveNode : client.getZkStateReader().getClusterState().getLiveNodes()) {
-      if (!replica.getNodeName().equals(liveNode)) {
+      if (!replica.getNode().equals(liveNode)) {
         state = new CollectionAdminRequest.MoveReplica(collection, replica.getName(), liveNode)
             .processAndWait(client, MAX_TIMEOUT_SECONDS);
         assertSame("MoveReplica did not complete", RequestStatusState.COMPLETED, state);
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
index 8d4f441..22d6044 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/CollectionsAPIDistributedZkTest.java
@@ -312,7 +312,7 @@ public class CollectionsAPIDistributedZkTest extends SolrCloudTestCase {
     DocCollection collection = getCollectionState("nodes_used_collection");
     for (Slice slice : collection.getSlices()) {
       for (Replica replica : slice.getReplicas()) {
-        createNodeList.remove(replica.getNodeName());
+        createNodeList.remove(replica.getNode());
       }
     }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/ReplicaPropertiesBase.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/ReplicaPropertiesBase.java
index fd4a409..5d59270 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/ReplicaPropertiesBase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/ReplicaPropertiesBase.java
@@ -131,16 +131,16 @@ public abstract class ReplicaPropertiesBase extends AbstractFullDistribZkTestBas
         boolean thisSliceHasProp = false;
         int propCount = 0;
         for (Replica replica : slice.getReplicas()) {
-          uniqueNodes.add(replica.getNodeName());
+          uniqueNodes.add(replica.getNode());
           String propVal = replica.getProperty(property);
           if (StringUtils.isNotBlank(propVal)) {
             ++propCount;
-            if (counts.containsKey(replica.getNodeName()) == false) {
-              counts.put(replica.getNodeName(), 0);
+            if (counts.containsKey(replica.getNode()) == false) {
+              counts.put(replica.getNode(), 0);
             }
-            int count = counts.get(replica.getNodeName());
+            int count = counts.get(replica.getNode());
             thisSliceHasProp = true;
-            counts.put(replica.getNodeName(), count + 1);
+            counts.put(replica.getNode(), count + 1);
           }
         }
         badSlice = (propCount > 1) ? true : badSlice;
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
index aa99bf8..a5a322f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/ShardSplitTest.java
@@ -132,7 +132,7 @@ public class ShardSplitTest extends BasicDistributedZkTest {
 
     DocCollection defCol = cloudClient.getZkStateReader().getClusterState().getCollection(AbstractDistribZkTestBase.DEFAULT_COLLECTION);
     Replica replica = defCol.getReplicas().get(0);
-    String nodeName = replica.getNodeName();
+    String nodeName = replica.getNode();
 
     String collectionName = "testSplitStaticIndexReplication_" + splitMethod.toLower();
     CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collectionName, "conf1", 1, 1);
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
index ef718e0..26b33ac 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionAPI.java
@@ -586,7 +586,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
 
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set("action", CollectionParams.CollectionAction.ADDROLE.toString());
-      params.set("node", replica.getNodeName());
+      params.set("node", replica.getNode());
       params.set("role", "overseer");
       @SuppressWarnings({"rawtypes"})
       SolrRequest request = new QueryRequest(params);
@@ -610,7 +610,7 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
       List<String> overseer = (List<String>) roles.get("overseer");
       assertNotNull(overseer);
       assertEquals(1, overseer.size());
-      assertTrue(overseer.contains(replica.getNodeName()));
+      assertTrue(overseer.contains(replica.getNode()));
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
index 3d5e28d..f503ee1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/api/collections/TestCollectionsAPIViaSolrCloudCluster.java
@@ -141,9 +141,9 @@ public class TestCollectionsAPIViaSolrCloudCluster extends SolrCloudTestCase {
     Collection<Slice> slices = clusterState.getCollection(collectionName).getSlices();
     // track the servers not host replicas
     for (Slice slice : slices) {
-      jettyMap.remove(slice.getLeader().getNodeName().replace("_solr", "/solr"));
+      jettyMap.remove(slice.getLeader().getNode().replace("_solr", "/solr"));
       for (Replica replica : slice.getReplicas()) {
-        jettyMap.remove(replica.getNodeName().replace("_solr", "/solr"));
+        jettyMap.remove(replica.getNode().replace("_solr", "/solr"));
       }
     }
     assertTrue("Expected to find a node without a replica", jettyMap.size() > 0);
@@ -249,9 +249,9 @@ public class TestCollectionsAPIViaSolrCloudCluster extends SolrCloudTestCase {
       final Map<String,Boolean> shardLeaderMap = new HashMap<>();
       for (final Slice slice : clusterState.getCollection(collectionName).getSlices()) {
         for (final Replica replica : slice.getReplicas()) {
-          shardLeaderMap.put(replica.getNodeName().replace("_solr", "/solr"), Boolean.FALSE);
+          shardLeaderMap.put(replica.getNode().replace("_solr", "/solr"), Boolean.FALSE);
         }
-        shardLeaderMap.put(slice.getLeader().getNodeName().replace("_solr", "/solr"), Boolean.TRUE);
+        shardLeaderMap.put(slice.getLeader().getNode().replace("_solr", "/solr"), Boolean.TRUE);
       }
       for (int ii = 0; ii < jettys.size(); ++ii) {
         final URL jettyBaseUrl = jettys.get(ii).getBaseUrl();
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
index b744ac1..f1f17d1 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoAddReplicasIntegrationTest.java
@@ -419,7 +419,7 @@ public class AutoAddReplicasIntegrationTest extends SolrCloudTestCase {
     List<Replica> replacedHdfsReplicas = new ArrayList<>();
     for (Replica replica : zkStateReader.getClusterState().getCollection(collection).getReplicas()) {
       String dataDir = replica.getStr("dataDir");
-      if (replica.getNodeName().equals(lostNodeName) && dataDir != null) {
+      if (replica.getNode().equals(lostNodeName) && dataDir != null) {
         replacedHdfsReplicas.add(replica);
       }
     }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
index 6d83fe2..0aca93d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/AutoScalingHandlerTest.java
@@ -117,7 +117,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
     String configPayload = "{\n" +
         "  'cluster-policy': [{'replica': 0, 'node': '_NODE'}]\n" +
         "}";
-    configPayload = configPayload.replaceAll("_NODE", aReplica.getNodeName());
+    configPayload = configPayload.replaceAll("_NODE", aReplica.getNode());
     @SuppressWarnings({"rawtypes"})
     SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, "/suggestions", configPayload);
     NamedList<Object> response = solrClient.request(req);
@@ -125,7 +125,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
     String replicaName = response._getStr("suggestions[0]/operation/command/move-replica/replica", null);
     boolean[] passed = new boolean[]{false};
     collection.forEachReplica((s, replica) -> {
-      if (replica.getName().equals(replicaName) && replica.getNodeName().equals(aReplica.getNodeName())) {
+      if (replica.getName().equals(replicaName) && replica.getNode().equals(aReplica.getNode())) {
         passed[0] = true;
       }
     });
@@ -150,7 +150,7 @@ public class AutoScalingHandlerTest extends SolrCloudTestCase {
     String configPayload = "{\n" +
         "  'cluster-policy': [{'replica': 0, 'node': '_NODE'}]\n" +
         "}";
-    configPayload = configPayload.replaceAll("_NODE", aReplica.getNodeName());
+    configPayload = configPayload.replaceAll("_NODE", aReplica.getNode());
     @SuppressWarnings({"rawtypes"})
     SolrRequest req = AutoScalingRequest.create(SolrRequest.METHOD.POST, "/diagnostics", configPayload);
     NamedList<Object> response = solrClient.request(req);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerSizeEstimationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerSizeEstimationTest.java
index 528dd55..2ba74b6 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerSizeEstimationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/IndexSizeTriggerSizeEstimationTest.java
@@ -157,7 +157,7 @@ public class IndexSizeTriggerSizeEstimationTest extends SolrCloudTestCase {
     final String registry = SolrCoreMetricManager.createRegistryName(true, collectionName, "shard1", replicaName, null);
     Set<String> tags = SimUtils.COMMON_REPLICA_TAGS.stream()
         .map(s -> "metrics:" + registry + ":" + s).collect(Collectors.toSet());
-    Map<String, Object> sizes = cloudManager.getNodeStateProvider().getNodeValues(leader.getNodeName(), tags);
+    Map<String, Object> sizes = cloudManager.getNodeStateProvider().getNodeValues(leader.getNode(), tags);
     String commitSizeTag = "metrics:" + registry + ":SEARCHER.searcher.indexCommitSize";
     String numDocsTag = "metrics:" + registry + ":SEARCHER.searcher.numDocs";
     String maxDocTag = "metrics:" + registry + ":SEARCHER.searcher.maxDoc";
@@ -283,7 +283,7 @@ public class IndexSizeTriggerSizeEstimationTest extends SolrCloudTestCase {
         final String subregistry = SolrCoreMetricManager.createRegistryName(true, collectionName, subShard, replicaName, null);
         Set<String> subtags = SimUtils.COMMON_REPLICA_TAGS.stream()
             .map(s -> "metrics:" + subregistry + ":" + s).collect(Collectors.toSet());
-        sizes = cloudManager.getNodeStateProvider().getNodeValues(leader.getNodeName(), subtags);
+        sizes = cloudManager.getNodeStateProvider().getNodeValues(leader.getNode(), subtags);
         commitSizeTag = "metrics:" + subregistry + ":SEARCHER.searcher.indexCommitSize";
         numDocsTag = "metrics:" + subregistry + ":SEARCHER.searcher.numDocs";
         maxDocTag = "metrics:" + subregistry + ":SEARCHER.searcher.maxDoc";
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
index 15ab701..92ace70 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerIntegrationTest.java
@@ -248,8 +248,8 @@ public class SearchRateTriggerIntegrationTest extends SolrCloudTestCase {
     assertTrue(replicaRates.toString(), replicaRates.size() > 0);
     AtomicDouble totalReplicaRate = new AtomicDouble();
     replicaRates.forEach(r -> {
-      assertTrue(r.toString(), r.getVariable("rate") != null);
-      totalReplicaRate.addAndGet((Double) r.getVariable("rate"));
+      assertTrue(r.toString(), r.get("rate") != null);
+      totalReplicaRate.addAndGet((Double) r.get("rate"));
     });
     Map<String, Object> shardRates = (Map<String, Object>) ev.event.getProperties().get(SearchRateTrigger.HOT_SHARDS);
     assertNotNull("shardRates", shardRates);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
index 6e3f15d..5612548 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SearchRateTriggerTest.java
@@ -149,7 +149,7 @@ public class SearchRateTriggerTest extends SolrCloudTestCase {
         assertEquals(1, infos.size());
         ReplicaInfo info = infos.get(0);
         assertEquals(coreName, info.getCore());
-        assertTrue((Double)info.getVariable(AutoScalingParams.RATE) > rate);
+        assertTrue((Double)info.get(AutoScalingParams.RATE) > rate);
       }
       // close that jetty to remove the violation - alternatively wait for 1 min...
       JettySolrRunner j = cluster.stopJettySolrRunner(1);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SystemLogListenerTest.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SystemLogListenerTest.java
index 714bc34..96d1451 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/SystemLogListenerTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/SystemLogListenerTest.java
@@ -282,7 +282,7 @@ public class SystemLogListenerTest extends SolrCloudTestCase {
 
     // now find a node that is *NOT* the overseer or the leader of a .system collection shard
     for (Replica r :  getCollectionState(CollectionAdminParams.SYSTEM_COLL).getReplicas()) {
-      if ( ! (r.getBool("leader", false) || r.getNodeName().equals(overseerNodeName) ) ) {
+      if ( ! (r.getBool("leader", false) || r.getNode().equals(overseerNodeName) ) ) {
         return cluster.getReplicaJetty(r);
       }
     }
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
index e715d85..b0ddf46 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/TestPolicyCloud.java
@@ -159,7 +159,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
       SolrClientNodeStateProvider nodeStateProvider = (SolrClientNodeStateProvider) cloudManager.getNodeStateProvider();
       Map<String, Map<String, List<ReplicaInfo>>> result = nodeStateProvider.getReplicaInfo(nodeName, Collections.singleton("UPDATE./update.requests"));
       nodeStateProvider.forEachReplica(nodeName, replicaInfo -> {
-        if (replicaInfo.getVariables().containsKey("UPDATE./update.requests")) count.incrementAndGet();
+        if (replicaInfo.getProperties().containsKey("UPDATE./update.requests")) count.incrementAndGet();
       });
       assertTrue(count.get() > 0);
 
@@ -178,7 +178,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
       for (Row row : session.getSortedNodes()) {
         row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
           for (ReplicaInfo replicaInfo : replicaInfos) {
-            if (replicaInfo.getVariables().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
+            if (replicaInfo.getProperties().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
           }
         }));
       }
@@ -199,7 +199,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
       for (Slice slice : collection) {
         for (Replica replica : slice) {
           if ( ! (replica.isActive(liveNodes)
-                  && expectedNodeName.equals(replica.getNodeName())) ) {
+                  && expectedNodeName.equals(replica.getNode())) ) {
             return false;
           }
           actualReplicaCount++;
@@ -296,8 +296,8 @@ public class TestPolicyCloud extends SolrCloudTestCase {
                      // now the main check we care about: were the replicas split up on
                      // the expected nodes...
                      if (! expectedNodeNames.equals(ImmutableSet.of
-                                                  (liveReplicas.get(0).getNodeName(),
-                                                   liveReplicas.get(1).getNodeName()))) {
+                                                  (liveReplicas.get(0).getNode(),
+                                                   liveReplicas.get(1).getNode()))) {
                        return false;
                      }
                    }
@@ -334,8 +334,8 @@ public class TestPolicyCloud extends SolrCloudTestCase {
                      // now the main check we care about: were the replicas split up on
                      // the expected nodes...
                      if (! expectedNodeNames.equals(ImmutableSet.of
-                                                    (liveReplicas.get(0).getNodeName(),
-                                                     liveReplicas.get(1).getNodeName()))) {
+                                                    (liveReplicas.get(0).getNode(),
+                                                     liveReplicas.get(1).getNode()))) {
                        return false;
                      }
                    }
@@ -381,11 +381,11 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     try (SolrCloudManager provider = new SolrClientCloudManager(queueFactory, solrClient)) {
       List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
           "metrics:solr.jvm:buffers.direct.Count");
-      Map<String, Object> val = provider.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), tags);
+      Map<String, Object> val = provider.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNode(), tags);
       for (String tag : tags) {
         assertNotNull("missing : " + tag, val.get(tag));
       }
-      val = provider.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), Collections.singleton("diskType"));
+      val = provider.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNode(), Collections.singleton("diskType"));
 
       Set<String> diskTypes = ImmutableSet.of("rotational", "ssd");
       assertTrue(diskTypes.contains(val.get("diskType")));
@@ -434,15 +434,15 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
       switch (replica.getType()) {
         case NRT: {
-          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNodeName().equals(nrtNodeName));
+          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNode().equals(nrtNodeName));
           break;
         }
         case TLOG: {
-          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNodeName().equals(tlogNodeName));
+          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNode().equals(tlogNodeName));
           break;
         }
         case PULL: {
-          assertTrue("PULL replica should be in " + pullNodeName, replica.getNodeName().equals(pullNodeName));
+          assertTrue("PULL replica should be in " + pullNodeName, replica.getNode().equals(pullNodeName));
           break;
         }
       }
@@ -479,7 +479,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     DocCollection coll = getCollectionState(collectionName);
     assertEquals("c1", coll.getPolicyName());
     assertEquals(2,coll.getReplicas().size());
-    coll.forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
+    coll.forEachReplica((s, replica) -> assertEquals(jetty.getNodeName(), replica.getNode()));
     
     CollectionAdminRequest.createShard(collectionName, "s3").process(cluster.getSolrClient());
 
@@ -487,7 +487,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
 
     coll = getCollectionState(collectionName);
     assertEquals(1, coll.getSlice("s3").getReplicas().size());
-    coll.getSlice("s3").forEach(replica -> assertEquals(jetty.getNodeName(), replica.getNodeName()));
+    coll.getSlice("s3").forEach(replica -> assertEquals(jetty.getNodeName(), replica.getNode()));
   }
 
   public void testDataProvider() throws Exception {
@@ -500,7 +500,7 @@ public class TestPolicyCloud extends SolrCloudTestCase {
     DocCollection rulesCollection = getCollectionState(collectionName);
 
     try (SolrCloudManager cloudManager = new SolrClientCloudManager(new ZkDistributedQueueFactory(cluster.getZkClient()), cluster.getSolrClient())) {
-      Map<String, Object> val = cloudManager.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
+      Map<String, Object> val = cloudManager.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNode(), Arrays.asList(
           "freedisk",
           "cores",
           "host",
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
index 3d2e314..1b5e21f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/SimSolrCloudTestCase.java
@@ -219,7 +219,7 @@ public class SimSolrCloudTestCase extends SolrTestCaseJ4 {
 
   public static void assertReplicaEquals(Replica one, Replica two) {
     assertEquals(one.getName(), two.getName());
-    assertEquals(one.getNodeName(), two.getNodeName());
+    assertEquals(one.getNode(), two.getNode());
     assertEquals(one.getState(), two.getState());
     assertEquals(one.getType(), two.getType());
     assertReplicaPropsEquals(one.getProperties(), two.getProperties());
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
index edf2349..d9a412d 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimLargeCluster.java
@@ -708,7 +708,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     cluster.getSimClusterStateProvider().getClusterState().getCollection(collectionName)
         .getSlice("shard1")
         .getReplicas()
-        .forEach(r -> nodes.add(r.getNodeName()));
+        .forEach(r -> nodes.add(r.getNode()));
 
     String metricName = "QUERY./select.requestTimes:1minRate";
     // simulate search traffic
@@ -803,7 +803,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
     ClusterState clusterState = cluster.getClusterStateProvider().getClusterState();
     DocCollection coll = clusterState.getCollection(collectionName);
     Set<String> nodes = coll.getReplicas().stream()
-        .map(r -> r.getNodeName())
+        .map(r -> r.getNode())
         .collect(Collectors.toSet());
     Map<String, Number> initialFreedisk = getFreeDiskPerNode(nodes);
 
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
index 11c388c..db03368 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimPolicyCloud.java
@@ -106,7 +106,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     for (Row row : session.getSortedNodes()) {
       row.collectionVsShardVsReplicas.forEach((c, shardVsReplicas) -> shardVsReplicas.forEach((s, replicaInfos) -> {
         for (ReplicaInfo replicaInfo : replicaInfos) {
-          if (replicaInfo.getVariables().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
+          if (replicaInfo.getProperties().containsKey(Type.CORE_IDX.tagName)) count.incrementAndGet();
         }
       }));
     }
@@ -133,14 +133,14 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     CloudUtil.waitForState(cluster, collectionName, 120, TimeUnit.SECONDS,
         CloudUtil.clusterShape(1, 1, false, true));
 
-    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNode()));
 
     CollectionAdminRequest.addReplicaToShard(collectionName, "shard1").process(solrClient);
     CloudUtil.waitForState(cluster,
         collectionName, 120l, TimeUnit.SECONDS,
         (liveNodes, collectionState) -> collectionState.getReplicas().size() == 2);
 
-    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+    getCollectionState(collectionName).forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNode()));
   }
 
   public void testCreateCollectionSplitShard() throws Exception  {
@@ -230,7 +230,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     DocCollection collection = getCollectionState("metricsTest");
     List<String> tags = Arrays.asList("metrics:solr.node:ADMIN./admin/authorization.clientErrors:count",
         "metrics:solr.jvm:buffers.direct.Count");
-    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNodeName(), tags);
+    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(collection.getReplicas().get(0).getNode(), tags);
     for (String tag : tags) {
       assertNotNull( "missing : "+ tag , val.get(tag));
     }
@@ -279,15 +279,15 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     BiConsumer<String, Replica> verifyReplicas = (s, replica) -> {
       switch (replica.getType()) {
         case NRT: {
-          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNodeName().equals(nrtNodeName));
+          assertTrue("NRT replica should be in " + nrtNodeName, replica.getNode().equals(nrtNodeName));
           break;
         }
         case TLOG: {
-          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNodeName().equals(tlogNodeName));
+          assertTrue("TLOG replica should be in " + tlogNodeName, replica.getNode().equals(tlogNodeName));
           break;
         }
         case PULL: {
-          assertTrue("PULL replica should be in " + pullNodeName, replica.getNodeName().equals(pullNodeName));
+          assertTrue("PULL replica should be in " + pullNodeName, replica.getNode().equals(pullNodeName));
           break;
         }
       }
@@ -321,14 +321,14 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
     DocCollection coll = getCollectionState("policiesTest");
     assertEquals("c1", coll.getPolicyName());
     assertEquals(2,coll.getReplicas().size());
-    coll.forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNodeName()));
+    coll.forEachReplica((s, replica) -> assertEquals(nodeId, replica.getNode()));
     CollectionAdminRequest.createShard("policiesTest", "s3").process(solrClient);
     CloudUtil.waitForState(cluster, "Timeout waiting for collection to become active", "policiesTest",
         CloudUtil.clusterShape(3, 1));
 
     coll = getCollectionState("policiesTest");
     assertEquals(1, coll.getSlice("s3").getReplicas().size());
-    coll.getSlice("s3").forEach(replica -> assertEquals(nodeId, replica.getNodeName()));
+    coll.getSlice("s3").forEach(replica -> assertEquals(nodeId, replica.getNode()));
   }
 
   public void testDataProvider() throws IOException, SolrServerException, KeeperException, InterruptedException {
@@ -339,7 +339,7 @@ public class TestSimPolicyCloud extends SimSolrCloudTestCase {
         CloudUtil.clusterShape(1, 2, false, true));
     DocCollection rulesCollection = getCollectionState("policiesTest");
 
-    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNodeName(), Arrays.asList(
+    Map<String, Object> val = cluster.getNodeStateProvider().getNodeValues(rulesCollection.getReplicas().get(0).getNode(), Arrays.asList(
         "freedisk",
         "cores",
         "heapUsage",
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
index a49a739..3206f06 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSimTriggerIntegration.java
@@ -1472,8 +1472,8 @@ public class TestSimTriggerIntegration extends SimSolrCloudTestCase {
     assertTrue(replicaRates.toString(), replicaRates.size() > 0);
     AtomicDouble totalReplicaRate = new AtomicDouble();
     replicaRates.forEach(r -> {
-      assertTrue(r.toString(), r.getVariable("rate") != null);
-      totalReplicaRate.addAndGet((Double)r.getVariable("rate"));
+      assertTrue(r.toString(), r.get("rate") != null);
+      totalReplicaRate.addAndGet((Double)r.get("rate"));
     });
     Map<String, Object> shardRates = (Map<String, Object>)ev.event.getProperties().get(SearchRateTrigger.HOT_SHARDS);
     assertNotNull("shardRates", shardRates);
diff --git a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSnapshotCloudManager.java b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSnapshotCloudManager.java
index 81d2f8f..2bf345e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSnapshotCloudManager.java
+++ b/solr/core/src/test/org/apache/solr/cloud/autoscaling/sim/TestSnapshotCloudManager.java
@@ -171,7 +171,7 @@ public class TestSnapshotCloudManager extends SolrCloudTestCase {
       // get another node
       String target = null;
       for (String node : simCloudManager.getClusterStateProvider().getLiveNodes()) {
-        if (!node.equals(r.getNodeName())) {
+        if (!node.equals(r.getNode())) {
           target = node;
           break;
         }
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
index aa35de1..61dfb3b 100644
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/cdcr/BaseCdcrDistributedZkTest.java
@@ -610,7 +610,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
     List<String> nodeNames = new ArrayList<>();
     for (Slice shard : zkStateReader.getClusterState().getCollection(temporaryCollection).getSlices()) {
       for (Replica replica : shard.getReplicas()) {
-        nodeNames.add(replica.getNodeName());
+        nodeNames.add(replica.getNode());
       }
     }
 
@@ -742,7 +742,7 @@ public class BaseCdcrDistributedZkTest extends AbstractDistribZkTestBase {
       this.coreNodeName = coreNodeName;
       nodeProperties.setProperty(CoreDescriptor.CORE_NODE_NAME, this.coreNodeName);
 
-      this.nodeName = replica.getNodeName();
+      this.nodeName = replica.getNode();
 
       ZkCoreNodeProps coreNodeProps = new ZkCoreNodeProps(info);
       this.url = coreNodeProps.getCoreUrl();
diff --git a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java
index 869e5be..2f379a0 100644
--- a/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java
+++ b/solr/core/src/test/org/apache/solr/cloud/cdcr/CdcrTestsUtil.java
@@ -265,7 +265,7 @@ public class CdcrTestsUtil extends SolrTestCaseJ4 {
   public static String getLeaderNode(MiniSolrCloudCluster cluster, String collection) throws Exception {
     for (Replica replica : cluster.getSolrClient().getClusterStateProvider().getCollection(collection).getReplicas()) {
       if (cluster.getSolrClient().getClusterStateProvider().getCollection(collection).getLeader("shard1") == replica) {
-        return replica.getNodeName();
+        return replica.getNode();
       }
     }
     return "";
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
index 02cc477..e4787ca 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RuleEngineTest.java
@@ -33,6 +33,7 @@ import org.apache.solr.client.solrj.cloud.autoscaling.DelegatingCloudManager;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.rule.Snitch;
@@ -212,7 +213,7 @@ public class RuleEngineTest extends SolrTestCaseJ4{
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             return null;
           }
         };
diff --git a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
index 7401c85..fe83514 100644
--- a/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/rule/RulesTest.java
@@ -131,7 +131,7 @@ public class RulesTest extends SolrCloudTestCase {
                          return false;
                        }
                        replicaNodes.addAll(liveReplicas.stream().map
-                                           (Replica::getNodeName).collect(Collectors.toList()));
+                                           (Replica::getNode).collect(Collectors.toList()));
                      } else if (slice.getName().equals("shard2")) {
                        // for shard2, we should have 3 fully live replicas
                        final List<Replica> liveReplicas = slice.getReplicas
@@ -140,7 +140,7 @@ public class RulesTest extends SolrCloudTestCase {
                          return false;
                        }
                        replicaNodes.addAll(liveReplicas.stream().map
-                                           (Replica::getNodeName).collect(Collectors.toList()));
+                                           (Replica::getNode).collect(Collectors.toList()));
                      } else {
                        // WTF?
                        return false;
@@ -210,7 +210,7 @@ public class RulesTest extends SolrCloudTestCase {
                    // now sanity check that the rules were *obeyed*
                    // (and the contradictory policy was ignored)
                    return rulesCollection.getReplicas().stream().allMatch
-                     (replica -> (replica.getNodeName().contains(port) &&
+                     (replica -> (replica.getNode().contains(port) &&
                                   replica.isActive(liveNodes)));
                  });
   }
@@ -254,7 +254,7 @@ public class RulesTest extends SolrCloudTestCase {
                    }
                    // now sanity check that the rules were *obeyed*
                    return rulesCollection.getReplicas().stream().allMatch
-                     (replica -> (replica.getNodeName().contains(port) &&
+                     (replica -> (replica.getNode().contains(port) &&
                                   replica.isActive(liveNodes)));
                  });
   }
diff --git a/solr/core/src/test/org/apache/solr/core/BlobRepositoryCloudTest.java b/solr/core/src/test/org/apache/solr/core/BlobRepositoryCloudTest.java
index efa8e11..8cb9485 100644
--- a/solr/core/src/test/org/apache/solr/core/BlobRepositoryCloudTest.java
+++ b/solr/core/src/test/org/apache/solr/core/BlobRepositoryCloudTest.java
@@ -94,7 +94,7 @@ public class BlobRepositoryCloudTest extends SolrCloudTestCase {
   // TODO: move this up to parent class?
   private static String findLiveNodeURI() {
     ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
-    return zkStateReader.getBaseUrlForNodeName(zkStateReader.getClusterState().getCollection(".system").getSlices().iterator().next().getLeader().getNodeName());
+    return zkStateReader.getBaseUrlForNodeName(zkStateReader.getClusterState().getCollection(".system").getSlices().iterator().next().getLeader().getNode());
   }
 
   private void assertLastQueryToCollection(String collection) throws SolrServerException, IOException {
diff --git a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
index f584604..93a9dd4 100644
--- a/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
+++ b/solr/core/src/test/org/apache/solr/core/CoreSorterTest.java
@@ -185,9 +185,9 @@ public class CoreSorterTest extends SolrTestCaseJ4 {
   private CoreDescriptor newCoreDescriptor(Replica r) {
     @SuppressWarnings({"unchecked"})
     Map<String,String> props = map(
-        CoreDescriptor.CORE_SHARD, r.getSlice(),
+        CoreDescriptor.CORE_SHARD, r.getShard(),
         CoreDescriptor.CORE_COLLECTION, r.getCollection(),
-        CoreDescriptor.CORE_NODE_NAME, r.getNodeName()
+        CoreDescriptor.CORE_NODE_NAME, r.getNode()
     );
     return new CoreDescriptor(r.getCoreName(), TEST_PATH(), props , null, mock(ZkController.class));
   }
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/AutoscalingHistoryHandlerTest.java b/solr/core/src/test/org/apache/solr/handler/admin/AutoscalingHistoryHandlerTest.java
index 9886e89..7c4f503 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/AutoscalingHistoryHandlerTest.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/AutoscalingHistoryHandlerTest.java
@@ -348,7 +348,7 @@ public class AutoscalingHistoryHandlerTest extends SolrCloudTestCase {
     if (system != null) {
       systemLeaderNodes = system.getReplicas().stream()
           .filter(r -> r.getBool("leader", false))
-          .map(r -> r.getNodeName())
+          .map(r -> r.getNode())
           .collect(Collectors.toSet());
     } else {
       systemLeaderNodes = Collections.emptySet();
@@ -356,12 +356,12 @@ public class AutoscalingHistoryHandlerTest extends SolrCloudTestCase {
     String nodeToKill = null;
     for (Replica r : coll.getReplicas()) {
       if (r.isActive(state.getLiveNodes()) &&
-          !r.getNodeName().equals(overseerLeader)) {
-        if (systemLeaderNodes.contains(r.getNodeName())) {
+          !r.getNode().equals(overseerLeader)) {
+        if (systemLeaderNodes.contains(r.getNode())) {
           log.info("--skipping .system leader replica {}", r);
           continue;
         }
-        nodeToKill = r.getNodeName();
+        nodeToKill = r.getNode();
         break;
       }
     }
@@ -431,7 +431,7 @@ public class AutoscalingHistoryHandlerTest extends SolrCloudTestCase {
       hasLeaders = true;
       if (replicas != null && !replicas.isEmpty()) {
         for (Replica r : replicas) {
-          if (state.getLiveNodes().contains(r.getNodeName())) {
+          if (state.getLiveNodes().contains(r.getNode())) {
             if (!r.isActive(state.getLiveNodes())) {
               log.info("Not active: {}", r);
               allActive = false;
diff --git a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
index cd3f874..c70c075 100644
--- a/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
+++ b/solr/core/src/test/org/apache/solr/update/TestInPlaceUpdatesDistrib.java
@@ -199,7 +199,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
     Slice shard1 = clusterState.getCollection(DEFAULT_COLLECTION).getSlice(SHARD1);
     leader = shard1.getLeader();
 
-    String leaderBaseUrl = zkStateReader.getBaseUrlForNodeName(leader.getNodeName());
+    String leaderBaseUrl = zkStateReader.getBaseUrlForNodeName(leader.getNode());
     for (int i=0; i<clients.size(); i++) {
       if (((HttpSolrClient)clients.get(i)).getBaseURL().startsWith(leaderBaseUrl))
         LEADER = clients.get(i);
@@ -210,7 +210,7 @@ public class TestInPlaceUpdatesDistrib extends AbstractFullDistribZkTestBase {
       if (rep.equals(leader)) {
         continue;
       }
-      String baseUrl = zkStateReader.getBaseUrlForNodeName(rep.getNodeName());
+      String baseUrl = zkStateReader.getBaseUrlForNodeName(rep.getNode());
       for (int i=0; i<clients.size(); i++) {
         if (((HttpSolrClient)clients.get(i)).getBaseURL().startsWith(baseUrl))
           NONLEADERS.add(clients.get(i));
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/NodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/NodeStateProvider.java
index 0f5f1e5..bb1b553 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/NodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/NodeStateProvider.java
@@ -20,8 +20,8 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.common.SolrCloseable;
+import org.apache.solr.common.cloud.Replica;
 
 /**
  * This interface models the access to node and replica information.
@@ -40,6 +40,7 @@ public interface NodeStateProvider extends SolrCloseable {
    * Get the details of each replica in a node. It attempts to fetch as much details about
    * the replica as mentioned in the keys list. It is not necessary to give all details
    * <p>The format is {collection:shard :[{replicadetails}]}.</p>
+   * @return
    */
-  Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys);
+  Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys);
 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
index b5c956a..06f49e6 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Clause.java
@@ -700,7 +700,7 @@ public class Clause implements MapWriter, Comparable<Clause> {
   }
 
 
-  public boolean isMatch(ReplicaInfo r, String collection, String shard) {
+  public boolean isMatch(Replica r, String collection, String shard) {
     if (type != null && r.getType() != type) return false;
     if (r.getCollection().equals(collection)) {
       if (this.shard == null || this.shard.val.equals(Policy.ANY)) return true;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
index a468385..2e2c911 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/CoresVariable.java
@@ -21,6 +21,8 @@ import java.util.Collection;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.function.Consumer;
 
+import org.apache.solr.common.cloud.Replica;
+
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MOVEREPLICA;
 
 public class CoresVariable extends VariableBase {
@@ -59,12 +61,12 @@ public class CoresVariable extends VariableBase {
   }
 
   @Override
-  public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
+  public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
     cell.val = cell.val == null ? 0 : ((Number) cell.val).doubleValue() + 1;
   }
 
   @Override
-  public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
+  public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
     cell.val = cell.val == null ? 0 : ((Number) cell.val).doubleValue() - 1;
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
index 72b2819..2a30d37 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/DelegatingNodeStateProvider.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
+import org.apache.solr.common.cloud.Replica;
 
 /**
  * Base class for overriding some behavior of {@link NodeStateProvider}.
@@ -40,7 +41,7 @@ public class DelegatingNodeStateProvider implements NodeStateProvider {
   }
 
   @Override
-  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+  public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
     return delegate.getReplicaInfo(node, keys);
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
index 2193ea3..fe5b164 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/FreeDiskVariable.java
@@ -26,6 +26,7 @@ import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 import org.apache.solr.client.solrj.cloud.autoscaling.Suggester.Hint;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.Pair;
 
 import static org.apache.solr.client.solrj.cloud.autoscaling.Suggestion.suggestNegativeViolations;
@@ -97,24 +98,24 @@ public class FreeDiskVariable extends VariableBase {
 
       for (Row node : matchingNodes) {
         //lets try to start moving the smallest cores off of the node
-        ArrayList<ReplicaInfo> replicas = new ArrayList<>();
+        ArrayList<Replica> replicas = new ArrayList<>();
         node.forEachReplica(replicas::add);
         replicas.sort((r1, r2) -> {
-          Long s1 = Clause.parseLong(CORE_IDX.tagName, r1.getVariables().get(CORE_IDX.tagName));
-          Long s2 = Clause.parseLong(CORE_IDX.tagName, r2.getVariables().get(CORE_IDX.tagName));
+          Long s1 = Clause.parseLong(CORE_IDX.tagName, r1.getProperties().get(CORE_IDX.tagName));
+          Long s2 = Clause.parseLong(CORE_IDX.tagName, r2.getProperties().get(CORE_IDX.tagName));
           if (s1 != null && s2 != null) return s1.compareTo(s2);
           return 0;
         });
         double currentDelta = ctx.violation.getClause().tag.delta(node.getVal(DISK));
-        for (ReplicaInfo replica : replicas) {
+        for (Replica replica : replicas) {
           if (currentDelta < 1) break;
-          if (replica.getVariables().get(CORE_IDX.tagName) == null) continue;
+          if (replica.getProperties().get(CORE_IDX.tagName) == null) continue;
           Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
               .hint(Hint.COLL_SHARD, new Pair<>(replica.getCollection(), replica.getShard()))
               .hint(Hint.SRC_NODE, node.node)
               .forceOperation(true);
           ctx.addSuggestion(suggester);
-          currentDelta -= Clause.parseLong(CORE_IDX.tagName, replica.getVariable(CORE_IDX.tagName));
+          currentDelta -= Clause.parseLong(CORE_IDX.tagName, replica.get(CORE_IDX.tagName));
         }
       }
     } else if (ctx.violation.replicaCountDelta < 0) {
@@ -131,7 +132,7 @@ public class FreeDiskVariable extends VariableBase {
             node.forEachShard(coll, (s, ri) -> {
               if (result.get() != null) return;
               if (s.equals(shard1) && ri.size() > 0) {
-                Number sz = ((Number) ri.get(0).getVariable(CORE_IDX.tagName));
+                Number sz = ((Number) ri.get(0).get(CORE_IDX.tagName));
                 if (sz != null) result.set(new Pair<>(shard1, sz.longValue()));
               }
             });
@@ -146,28 +147,28 @@ public class FreeDiskVariable extends VariableBase {
 
   //When a replica is added, freedisk should be incremented
   @Override
-  public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
+  public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> ops, boolean strictMode) {
     //go through other replicas of this shard and copy the index size value into this
     for (Row row : cell.getRow().session.matrix) {
       row.forEachReplica(replicaInfo -> {
         if (ri != replicaInfo &&
             ri.getCollection().equals(replicaInfo.getCollection()) &&
             ri.getShard().equals(replicaInfo.getShard()) &&
-            ri.getVariable(CORE_IDX.tagName) == null &&
-            replicaInfo.getVariable(CORE_IDX.tagName) != null) {
-          ri.getVariables().put(CORE_IDX.tagName, validate(CORE_IDX.tagName, replicaInfo.getVariable(CORE_IDX.tagName), false));
+            ri.get(CORE_IDX.tagName) == null &&
+            replicaInfo.get(CORE_IDX.tagName) != null) {
+          ri.getProperties().put(CORE_IDX.tagName, validate(CORE_IDX.tagName, replicaInfo.get(CORE_IDX.tagName), false));
         }
       });
     }
-    Double idxSize = (Double) validate(CORE_IDX.tagName, ri.getVariable(CORE_IDX.tagName), false);
+    Double idxSize = (Double) validate(CORE_IDX.tagName, ri.get(CORE_IDX.tagName), false);
     if (idxSize == null) return;
     Double currFreeDisk = cell.val == null ? 0.0d : (Double) cell.val;
     cell.val = currFreeDisk - idxSize;
   }
 
   @Override
-  public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
-    Double idxSize = (Double) validate(CORE_IDX.tagName, ri.getVariable(CORE_IDX.tagName), false);
+  public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
+    Double idxSize = (Double) validate(CORE_IDX.tagName, ri.get(CORE_IDX.tagName), false);
     if (idxSize == null) return;
     Double currFreeDisk = cell.val == null ? 0.0d : (Double) cell.val;
     cell.val = currFreeDisk + idxSize;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggester.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggester.java
index 49be0e3..42fbcc1 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggester.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/MoveReplicaSuggester.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.Pair;
 import org.slf4j.Logger;
@@ -47,14 +48,14 @@ public class MoveReplicaSuggester extends Suggester {
     List<Violation> leastSeriousViolation = null;
     Row bestSrcRow = null;
     Row bestTargetRow = null;
-    ReplicaInfo sourceReplicaInfo = null;
-    List<Pair<ReplicaInfo, Row>> validReplicas = getValidReplicas(true, true, -1);
+    Replica sourceReplicaInfo = null;
+    List<Pair<Replica, Row>> validReplicas = getValidReplicas(true, true, -1);
     validReplicas.sort(leaderLast);
     for (int i1 = 0; i1 < validReplicas.size(); i1++) {
       lastBestDeviation = null;
-      Pair<ReplicaInfo, Row> fromReplica = validReplicas.get(i1);
+      Pair<Replica, Row> fromReplica = validReplicas.get(i1);
       Row fromRow = fromReplica.second();
-      ReplicaInfo ri = fromReplica.first();
+      Replica ri = fromReplica.first();
       if (ri == null) continue;
       final int i = session.indexOf(fromRow.node);
       int stopAt = force ? 0 : i;
@@ -97,7 +98,7 @@ public class MoveReplicaSuggester extends Suggester {
     return null;
   }
 
-  static Comparator<Pair<ReplicaInfo, Row>> leaderLast = (r1, r2) -> {
+  static Comparator<Pair<Replica, Row>> leaderLast = (r1, r2) -> {
     int leaderCompare = Boolean.compare(r1.first().isLeader, r2.first().isLeader);
     if ( leaderCompare != 0 ) {
       return leaderCompare;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
index 4066eab..bf7c8cb 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/PolicyHelper.java
@@ -144,12 +144,12 @@ public class PolicyHelper {
         if (coll != null) {
           for (String shardName : shardNames) {
             Replica ldr = coll.getLeader(shardName);
-            if (ldr != null && cloudManager.getClusterStateProvider().getLiveNodes().contains(ldr.getNodeName())) {
-              Map<String, Map<String, List<ReplicaInfo>>> details = cloudManager.getNodeStateProvider().getReplicaInfo(ldr.getNodeName(),
+            if (ldr != null && cloudManager.getClusterStateProvider().getLiveNodes().contains(ldr.getNode())) {
+              Map<String, Map<String, List<Replica>>> details = cloudManager.getNodeStateProvider().getReplicaInfo(ldr.getNode(),
                   Collections.singleton(FREEDISK.perReplicaValue));
-              ReplicaInfo replicaInfo = details.getOrDefault(collName, emptyMap()).getOrDefault(shardName, singletonList(null)).get(0);
+              Replica replicaInfo = details.getOrDefault(collName, emptyMap()).getOrDefault(shardName, singletonList(null)).get(0);
               if (replicaInfo != null) {
-                Object idxSz = replicaInfo.getVariables().get(FREEDISK.perReplicaValue);
+                Object idxSz = replicaInfo.getProperties().get(FREEDISK.perReplicaValue);
                 if (idxSz != null) {
                   diskSpaceReqd.put(shardName, 1.5 * (Double) Variable.Type.FREEDISK.validate(null, idxSz, false));
                 }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
index 642b565..c41e48c 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaCount.java
@@ -63,14 +63,14 @@ class ReplicaCount  implements MapWriter {
     return total();
   }
 
-  public void increment(List<ReplicaInfo> infos) {
+  public void increment(List<Replica> infos) {
     if (infos == null) return;
-    for (ReplicaInfo info : infos) {
+    for (Replica info : infos) {
       increment(info);
     }
   }
 
-  void increment(ReplicaInfo info) {
+  void increment(Replica info) {
     increment(info.getType());
   }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfoXX.java
similarity index 89%
rename from solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
rename to solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfoXX.java
index 9b1f78c..5b5420f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfo.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/ReplicaInfoXX.java
@@ -35,7 +35,7 @@ import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
 import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
 import static org.apache.solr.common.cloud.ZkStateReader.LEADER_PROP;
 
-public class ReplicaInfo implements MapWriter {
+public class ReplicaInfoXX implements MapWriter {
   private final String name;
   private final String core, collection, shard;
   private final Replica.Type type;
@@ -43,13 +43,13 @@ public class ReplicaInfo implements MapWriter {
   public final boolean isLeader;
   private final Map<String, Object> variables = new HashMap<>();
 
-  public ReplicaInfo(String coll, String shard, Replica r, Map<String, Object> vals) {
+  public ReplicaInfoXX(String coll, String shard, Replica r, Map<String, Object> vals) {
     this.name = r.getName();
     this.core = r.getCoreName();
     this.collection = coll;
     this.shard = shard;
     this.type = r.getType();
-    this.node = r.getNodeName();
+    this.node = r.getNode();
     boolean maybeLeader = r.getBool(LEADER_PROP, false);
     if (vals != null) {
       this.variables.putAll(vals);
@@ -59,7 +59,7 @@ public class ReplicaInfo implements MapWriter {
     validate();
   }
 
-  public ReplicaInfo(String name, String core, String coll, String shard, Replica.Type type, String node, Map<String, Object> vals) {
+  public ReplicaInfoXX(String name, String core, String coll, String shard, Replica.Type type, String node, Map<String, Object> vals) {
     if (vals == null) vals = Collections.emptyMap();
     this.name = name;
     if (vals != null) {
@@ -75,7 +75,7 @@ public class ReplicaInfo implements MapWriter {
   }
 
   @SuppressWarnings({"unchecked"})
-  public ReplicaInfo(Map<String, Object> map) {
+  public ReplicaInfoXX(Map<String, Object> map) {
     this.name = map.keySet().iterator().next();
     @SuppressWarnings({"rawtypes"})Map details = (Map) map.get(name);
     details = Utils.getDeepCopy(details, 4);
@@ -101,7 +101,7 @@ public class ReplicaInfo implements MapWriter {
   }
 
   public Object clone() {
-    return new ReplicaInfo(name, core, collection, shard, type, node, new HashMap<>(variables));
+    return new ReplicaInfoXX(name, core, collection, shard, type, node, new HashMap<>(variables));
   }
 
   @Override
@@ -158,15 +158,15 @@ public class ReplicaInfo implements MapWriter {
     }
   }
 
-  public Map<String, Object> getVariables() {
+  public Map<String, Object> getProperties() {
     return variables;
   }
 
-  public Object getVariable(String name) {
+  public Object get(String name) {
     return variables.get(name);
   }
 
-  public Object getVariable(String name, Object defValue) {
+  public Object get(String name, Object defValue) {
     Object o = variables.get(name);
     if (o != null) {
       return o;
@@ -176,7 +176,7 @@ public class ReplicaInfo implements MapWriter {
   }
 
   public boolean getBool(String name, boolean defValue) {
-    Object o = getVariable(name, defValue);
+    Object o = get(name, defValue);
     if (o instanceof Boolean) {
       return (Boolean)o;
     } else {
@@ -189,10 +189,10 @@ public class ReplicaInfo implements MapWriter {
     if (o == null) {
       return false;
     }
-    if (!(o instanceof ReplicaInfo)) {
+    if (!(o instanceof ReplicaInfoXX)) {
       return false;
     }
-    ReplicaInfo other = (ReplicaInfo)o;
+    ReplicaInfoXX other = (ReplicaInfoXX)o;
     if (
         name.equals(other.name) &&
         collection.equals(other.collection) &&
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Row.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Row.java
index e2b6a55..db94ed0 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Row.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Row.java
@@ -55,7 +55,7 @@ public class Row implements MapWriter {
   public final String node;
   final Cell[] cells;
   //this holds the details of each replica in the node
-  public Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas;
+  public Map<String, Map<String, List<Replica>>> collectionVsShardVsReplicas;
 
   boolean anyValueMissing = false;
   boolean isLive = true;
@@ -130,7 +130,7 @@ public class Row implements MapWriter {
   }
 
 
-  public void forEachShard(String collection, BiConsumer<String, List<ReplicaInfo>> consumer) {
+  public void forEachShard(String collection, BiConsumer<String, List<Replica>> consumer) {
     collectionVsShardVsReplicas
         .getOrDefault(collection, Collections.emptyMap())
         .forEach(consumer);
@@ -172,7 +172,7 @@ public class Row implements MapWriter {
 
   public Row(String node, Cell[] cells, boolean anyValueMissing,
              @SuppressWarnings({"rawtypes"}) Map<String,
-                     Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, boolean isLive, Policy.Session session,
+                     Map<String, List<Replica>>> collectionVsShardVsReplicas, boolean isLive, Policy.Session session,
              @SuppressWarnings({"rawtypes"}) Map perRowCache,
              @SuppressWarnings({"rawtypes"})Map globalCache) {
     this.session = session;
@@ -251,11 +251,11 @@ public class Row implements MapWriter {
     row = session.copy().getNode(this.node);
     if (row == null) throw new RuntimeException("couldn't get a row");
     row.lazyCopyReplicas(coll, shard);
-    Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.computeIfAbsent(coll, k -> new HashMap<>());
-    List<ReplicaInfo> replicas = c.computeIfAbsent(shard, k -> new ArrayList<>());
+    Map<String, List<Replica>> c = row.collectionVsShardVsReplicas.computeIfAbsent(coll, k -> new HashMap<>());
+    List<Replica> replicas = c.computeIfAbsent(shard, k -> new ArrayList<>());
     String replicaname = "SYNTHETIC." + new Random().nextInt(1000) + 1000;
-    ReplicaInfo ri = new ReplicaInfo(replicaname, replicaname, coll, shard, type, this.node,
-        Utils.makeMap(ZkStateReader.REPLICA_TYPE, type != null ? type.toString() : Replica.Type.NRT.toString()));
+    Replica ri = new Replica(replicaname, this.node, coll, shard, replicaname, false,
+        Replica.State.ACTIVE, type != null ? type : Replica.Type.NRT, Collections.emptyMap());
     replicas.add(ri);
     for (Cell cell : row.cells) {
       cell.type.projectAddReplica(cell, ri, opCollector, strictMode);
@@ -281,8 +281,8 @@ public class Row implements MapWriter {
     perCollCache = cacheCopy;
     if (isAlreadyCopied) return;//caches need to be invalidated but the rest can remain as is
 
-    Map<String, Map<String, List<ReplicaInfo>>> replicasCopy = new HashMap<>(collectionVsShardVsReplicas);
-    Map<String, List<ReplicaInfo>> oneColl = replicasCopy.get(coll);
+    Map<String, Map<String, List<Replica>>> replicasCopy = new HashMap<>(collectionVsShardVsReplicas);
+    Map<String, List<Replica>> oneColl = replicasCopy.get(coll);
     if (oneColl != null) {
       replicasCopy.put(coll, Utils.getDeepCopy(oneColl, 2));
     }
@@ -296,7 +296,7 @@ public class Row implements MapWriter {
 
   @SuppressWarnings({"unchecked"})
   public void createCollShard(Pair<String, String> collShard) {
-    Map<String, List<ReplicaInfo>> shardInfo = collectionVsShardVsReplicas.computeIfAbsent(collShard.first(), Utils.NEW_HASHMAP_FUN);
+    Map<String, List<Replica>> shardInfo = collectionVsShardVsReplicas.computeIfAbsent(collShard.first(), Utils.NEW_HASHMAP_FUN);
     if (collShard.second() != null) shardInfo.computeIfAbsent(collShard.second(), Utils.NEW_ARRAYLIST_FUN);
   }
 
@@ -318,14 +318,14 @@ public class Row implements MapWriter {
   }
 
 
-  public ReplicaInfo getReplica(String coll, String shard, Replica.Type type) {
-    Map<String, List<ReplicaInfo>> c = collectionVsShardVsReplicas.get(coll);
+  public Replica getReplica(String coll, String shard, Replica.Type type) {
+    Map<String, List<Replica>> c = collectionVsShardVsReplicas.get(coll);
     if (c == null) return null;
-    List<ReplicaInfo> r = c.get(shard);
+    List<Replica> r = c.get(shard);
     if (r == null) return null;
     int idx = -1;
     for (int i = 0; i < r.size(); i++) {
-      ReplicaInfo info = r.get(i);
+      Replica info = r.get(i);
       if (type == null || info.getType() == type) {
         idx = i;
         break;
@@ -350,20 +350,20 @@ public class Row implements MapWriter {
     Consumer<OperationInfo> opCollector = it -> furtherOps.add(it);
     Row row = session.copy().getNode(this.node);
     row.lazyCopyReplicas(coll, shard);
-    Map<String, List<ReplicaInfo>> c = row.collectionVsShardVsReplicas.get(coll);
+    Map<String, List<Replica>> c = row.collectionVsShardVsReplicas.get(coll);
     if (c == null) return null;
-    List<ReplicaInfo> r = c.get(shard);
+    List<Replica> r = c.get(shard);
     if (r == null) return null;
     int idx = -1;
     for (int i = 0; i < r.size(); i++) {
-      ReplicaInfo info = r.get(i);
+      Replica info = r.get(i);
       if (type == null || info.getType() == type) {
         idx = i;
         break;
       }
     }
     if (idx == -1) return null;
-    ReplicaInfo removed = r.remove(idx);
+    Replica removed = r.remove(idx);
     for (Cell cell : row.cells) {
       cell.type.projectRemoveReplica(cell, removed, opCollector);
     }
@@ -379,23 +379,23 @@ public class Row implements MapWriter {
     return isLive;
   }
 
-  public void forEachReplica(Consumer<ReplicaInfo> consumer) {
+  public void forEachReplica(Consumer<Replica> consumer) {
     forEachReplica(collectionVsShardVsReplicas, consumer);
   }
 
-  public void forEachReplica(String coll, Consumer<ReplicaInfo> consumer) {
+  public void forEachReplica(String coll, Consumer<Replica> consumer) {
     collectionVsShardVsReplicas.getOrDefault(coll, Collections.emptyMap()).forEach((shard, replicaInfos) -> {
-      for (ReplicaInfo replicaInfo : replicaInfos) {
+      for (Replica replicaInfo : replicaInfos) {
         consumer.accept(replicaInfo);
       }
     });
   }
 
-  public static void forEachReplica(Map<String, Map<String, List<ReplicaInfo>>> collectionVsShardVsReplicas, Consumer<ReplicaInfo> consumer) {
+  public static void forEachReplica(Map<String, Map<String, List<Replica>>> collectionVsShardVsReplicas, Consumer<Replica> consumer) {
     collectionVsShardVsReplicas.forEach((coll, shardVsReplicas) -> shardVsReplicas
         .forEach((shard, replicaInfos) -> {
           for (int i = 0; i < replicaInfos.size(); i++) {
-            ReplicaInfo r = replicaInfos.get(i);
+            Replica r = replicaInfos.get(i);
             consumer.accept(r);
           }
         }));
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
index b9d5faf..48f36e8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Suggester.java
@@ -296,8 +296,8 @@ public abstract class Suggester implements MapWriter {
     return false;
   }
 
-  List<Pair<ReplicaInfo, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
-    List<Pair<ReplicaInfo, Row>> allPossibleReplicas = new ArrayList<>();
+  List<Pair<Replica, Row>> getValidReplicas(boolean sortDesc, boolean isSource, int until) {
+    List<Pair<Replica, Row>> allPossibleReplicas = new ArrayList<>();
 
     if (sortDesc) {
       if (until == -1) until = getMatrix().size();
@@ -310,14 +310,14 @@ public abstract class Suggester implements MapWriter {
     return allPossibleReplicas;
   }
 
-  void addReplicaToList(Row r, boolean isSource, List<Pair<ReplicaInfo, Row>> replicaList) {
+  void addReplicaToList(Row r, boolean isSource, List<Pair<Replica, Row>> replicaList) {
     if (!isAllowed(r.node, isSource ? Hint.SRC_NODE : Hint.TARGET_NODE)) return;
-    for (Map.Entry<String, Map<String, List<ReplicaInfo>>> e : r.collectionVsShardVsReplicas.entrySet()) {
+    for (Map.Entry<String, Map<String, List<Replica>>> e : r.collectionVsShardVsReplicas.entrySet()) {
       if (!isAllowed(e.getKey(), Hint.COLL)) continue;
-      for (Map.Entry<String, List<ReplicaInfo>> shard : e.getValue().entrySet()) {
+      for (Map.Entry<String, List<Replica>> shard : e.getValue().entrySet()) {
         if (!isAllowed(new Pair<>(e.getKey(), shard.getKey()), Hint.COLL_SHARD)) continue;//todo fix
         if (shard.getValue() == null || shard.getValue().isEmpty()) continue;
-        for (ReplicaInfo replicaInfo : shard.getValue()) {
+        for (Replica replicaInfo : shard.getValue()) {
           if (replicaInfo.getName().startsWith("SYNTHETIC.")) continue;
           replicaList.add(new Pair<>(shard.getValue().get(0), r));
           break;
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
index 4b45107..86af567 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Variable.java
@@ -26,6 +26,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.function.Consumer;
 
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 
 import static java.util.Collections.emptySet;
@@ -48,7 +49,7 @@ public interface Variable {
     return val;
   }
 
-  default void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
+  default void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
   }
 
   default boolean addViolatingReplicas(Violation.Ctx ctx) {
@@ -77,7 +78,7 @@ public interface Variable {
 
   int compareViolation(Violation v1, Violation v2);
 
-  default void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
+  default void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
   }
 
   default String postValidate(Condition condition) {
@@ -329,11 +330,11 @@ public interface Variable {
     /**
      * Simulate a replica addition to a node in the cluster
      */
-    public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
+    public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
       impl.projectAddReplica(cell, ri, opCollector, strictMode);
     }
 
-    public void projectRemoveReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector) {
+    public void projectRemoveReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector) {
       impl.projectRemoveReplica(cell, ri, opCollector);
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
index e0d2048..41221a2 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/Violation.java
@@ -27,6 +27,7 @@ import java.util.function.Function;
 
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.util.Utils;
 
 public class Violation implements MapWriter {
@@ -109,10 +110,10 @@ public class Violation implements MapWriter {
   }
 
   static class ReplicaInfoAndErr implements MapWriter{
-    final ReplicaInfo replicaInfo;
+    final Replica replicaInfo;
     Double delta;
 
-    ReplicaInfoAndErr(ReplicaInfo replicaInfo) {
+    ReplicaInfoAndErr(Replica replicaInfo) {
       this.replicaInfo = replicaInfo;
     }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/WithCollectionVariable.java b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/WithCollectionVariable.java
index 8cab901..f0a9e04 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/WithCollectionVariable.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/cloud/autoscaling/WithCollectionVariable.java
@@ -56,7 +56,7 @@ public class WithCollectionVariable extends VariableBase {
     return true;
   }
 
-  public void projectAddReplica(Cell cell, ReplicaInfo ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
+  public void projectAddReplica(Cell cell, Replica ri, Consumer<Row.OperationInfo> opCollector, boolean strictMode) {
     if (strictMode) {
       // we do not want to add a replica of the 'withCollection' in strict mode
       return;
@@ -138,14 +138,14 @@ public class WithCollectionVariable extends VariableBase {
 
         if (hostedCollections.contains(withCollection) && !hostedCollections.contains(collection))  {
           // find the candidate replicas that we can move
-          List<ReplicaInfo> movableReplicas = new ArrayList<>();
+          List<Replica> movableReplicas = new ArrayList<>();
           row.forEachReplica(replicaInfo -> {
             if (replicaInfo.getCollection().equals(withCollection)) {
               movableReplicas.add(replicaInfo);
             }
           });
 
-          for (ReplicaInfo toMove : movableReplicas) {
+          for (Replica toMove : movableReplicas) {
             // candidate source node for a move replica operation
             Suggester suggester = ctx.session.getSuggester(MOVEREPLICA)
                 .forceOperation(true)
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
index 6a38f7c..0b8a831 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/BaseCloudSolrClient.java
@@ -1116,7 +1116,7 @@ public abstract class BaseCloudSolrClient extends SolrClient {
       for (Slice slice : slices.values()) {
         Replica leader = slice.getLeader();
         for (Replica replica : slice.getReplicas()) {
-          String node = replica.getNodeName();
+          String node = replica.getNode();
           if (!liveNodes.contains(node) // Must be a live node to continue
               || replica.getState() != Replica.State.ACTIVE) // Must be an ACTIVE replica to continue
             continue;
@@ -1139,7 +1139,7 @@ public abstract class BaseCloudSolrClient extends SolrClient {
       String joinedInputCollections = StrUtils.join(inputCollections, ',');
       Set<String> seenNodes = new HashSet<>();
       sortedReplicas.forEach( replica -> {
-        if (seenNodes.add(replica.getNodeName())) {
+        if (seenNodes.add(replica.getNode())) {
           theUrlList.add(ZkCoreNodeProps.getCoreUrl(replica.getBaseUrl(), joinedInputCollections));
         }
       });
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
index 4f4b7d9..0c5bb21 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/SolrClientNodeStateProvider.java
@@ -34,7 +34,6 @@ import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
-import org.apache.solr.client.solrj.cloud.autoscaling.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.autoscaling.Row;
 import org.apache.solr.client.solrj.cloud.autoscaling.Variable.Type;
 import org.apache.solr.client.solrj.cloud.autoscaling.VariableBase;
@@ -45,6 +44,7 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.rule.ImplicitSnitch;
 import org.apache.solr.common.cloud.rule.SnitchContext;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -76,7 +76,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
 
 
   private final CloudSolrClient solrClient;
-  protected final Map<String, Map<String, Map<String, List<ReplicaInfo>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
+  protected final Map<String, Map<String, Map<String, List<Replica>>>> nodeVsCollectionVsShardVsReplicaInfo = new HashMap<>();
   private Map<String, Object> snitchSession = new HashMap<>();
   @SuppressWarnings({"rawtypes"})
   private Map<String, Map> nodeVsTags = new HashMap<>();
@@ -110,10 +110,10 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
         withCollectionsMap.put(coll.getName(), (String) coll.getProperties().get(CollectionAdminParams.WITH_COLLECTION));
       }
       coll.forEachReplica((shard, replica) -> {
-        Map<String, Map<String, List<ReplicaInfo>>> nodeData = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(replica.getNodeName(), k -> new HashMap<>());
-        Map<String, List<ReplicaInfo>> collData = nodeData.computeIfAbsent(collName, k -> new HashMap<>());
-        List<ReplicaInfo> replicas = collData.computeIfAbsent(shard, k -> new ArrayList<>());
-        replicas.add(new ReplicaInfo(collName, shard, replica, new HashMap<>(replica.getProperties())));
+        Map<String, Map<String, List<Replica>>> nodeData = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(replica.getNode(), k -> new HashMap<>());
+        Map<String, List<Replica>> collData = nodeData.computeIfAbsent(collName, k -> new HashMap<>());
+        List<Replica> replicas = collData.computeIfAbsent(shard, k -> new ArrayList<>());
+        replicas.add((Replica) replica.clone());
       });
     });
   }
@@ -141,21 +141,21 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     return ctx.getTags();
   }
 
-  public void forEachReplica(String node, Consumer<ReplicaInfo> consumer){
+  public void forEachReplica(String node, Consumer<Replica> consumer){
     Row.forEachReplica(nodeVsCollectionVsShardVsReplicaInfo.get(node), consumer);
   }
 
 
   @Override
-  public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+  public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
     @SuppressWarnings({"unchecked"})
-    Map<String, Map<String, List<ReplicaInfo>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
+    Map<String, Map<String, List<Replica>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
     if (!keys.isEmpty()) {
-      Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica = new HashMap<>();
+      Map<String, Pair<String, Replica>> metricsKeyVsTagReplica = new HashMap<>();
       Row.forEachReplica(result, r -> {
         for (String key : keys) {
-          if (r.getVariables().containsKey(key)) continue;// it's already collected
-          String perReplicaMetricsKey = "solr.core." + r.getCollection() + "." + r.getShard() + "." + Utils.parseMetricsReplicaName(r.getCollection(), r.getCore()) + ":";
+          if (r.getProperties().containsKey(key)) continue;// it's already collected
+          String perReplicaMetricsKey = "solr.core." + r.getCollection() + "." + r.getShard() + "." + Utils.parseMetricsReplicaName(r.getCollection(), r.getCoreName()) + ":";
           Type tagType = VariableBase.getTagType(key);
           String perReplicaValue = key;
           if (tagType != null) {
@@ -170,10 +170,10 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
       if (!metricsKeyVsTagReplica.isEmpty()) {
         Map<String, Object> tagValues = fetchReplicaMetrics(node, metricsKeyVsTagReplica);
         tagValues.forEach((k, o) -> {
-          Pair<String, ReplicaInfo> p = metricsKeyVsTagReplica.get(k);
+          Pair<String, Replica> p = metricsKeyVsTagReplica.get(k);
           Type validator = VariableBase.getTagType(p.first());
           if (validator != null) o = validator.convertVal(o);
-          if (p.second() != null) p.second().getVariables().put(p.first(), o);
+          if (p.second() != null) p.second().getProperties().put(p.first(), o);
         });
 
       }
@@ -181,7 +181,7 @@ public class SolrClientNodeStateProvider implements NodeStateProvider, MapWriter
     return result;
   }
 
-  protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+  protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
     Map<String, Object> collect = metricsKeyVsTagReplica.entrySet().stream()
         .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getKey));
     ClientSnitchCtx ctx = new ClientSnitchCtx(null, null, emptyMap(), solrClient);
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
index 9ed805c..f014b92 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/FeaturesSelectionStream.java
@@ -270,7 +270,7 @@ public class FeaturesSelectionStream extends TupleStream implements Expressible{
         Collection<Replica> replicas = slice.getReplicas();
         List<Replica> shuffler = new ArrayList<>();
         for(Replica replica : replicas) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())) {
             shuffler.add(replica);
           }
         }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
index 6949d80..429cba0 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TextLogitStream.java
@@ -352,7 +352,7 @@ public class TextLogitStream extends TupleStream implements Expressible {
         Collection<Replica> replicas = slice.getReplicas();
         List<Replica> shuffler = new ArrayList<>();
         for(Replica replica : replicas) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())) {
             shuffler.add(replica);
           }
         }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
index b1769a6..bab8c66 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TopicStream.java
@@ -423,7 +423,7 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
     params.set(DISTRIB, "false");
     params.set("rows", 1);
     for(Replica replica : replicas) {
-      if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+      if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())) {
         String coreUrl = replica.getCoreUrl();
         SolrStream solrStream = new SolrStream(coreUrl, params);
 
@@ -484,7 +484,7 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
     for(Slice slice : slices) {
       Collection<Replica> replicas = slice.getReplicas();
       for(Replica replica : replicas) {
-        if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())){
+        if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())){
           HttpSolrClient httpClient = streamContext.getSolrClientCache().getHttpSolrClient(replica.getCoreUrl());
           try {
             SolrDocument doc = httpClient.getById(id);
@@ -531,7 +531,7 @@ public class TopicStream extends CloudSolrStream implements Expressible  {
         Collection<Replica> replicas = slice.getReplicas();
         List<Replica> shuffler = new ArrayList<>();
         for(Replica replica : replicas) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName()))
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode()))
             shuffler.add(replica);
         }
 
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
index 90bfb0e..46c133b 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/io/stream/TupleStream.java
@@ -160,7 +160,7 @@ public abstract class TupleStream implements Closeable, Serializable, MapWriter
       for(Slice slice : slices) {
         List<Replica> sortedReplicas = new ArrayList<>();
         for(Replica replica : slice.getReplicas()) {
-          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNodeName())) {
+          if(replica.getState() == Replica.State.ACTIVE && liveNodes.contains(replica.getNode())) {
             sortedReplicas.add(replica);
           }
         }
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparator.java b/solr/solrj/src/java/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparator.java
index bb8cecb..19a4fbaa 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparator.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparator.java
@@ -139,7 +139,7 @@ public class NodePreferenceRulesComparator implements Comparator<Object> {
     }
 
     Collection<String> tags = Collections.singletonList(metricTag);
-    String otherNodeName = ((Replica) o).getNodeName();
+    String otherNodeName = ((Replica) o).getNode();
     Map<String, Object> currentNodeMetric = sysPropsCache.getSysProps(nodeName, tags);
     Map<String, Object> otherNodeMetric = sysPropsCache.getSysProps(otherNodeName, tags);
     return currentNodeMetric.equals(otherNodeMetric);
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
index 5e61bc1..1f90ada 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterStateUtil.java
@@ -80,7 +80,7 @@ public class ClusterStateUtil {
               Collection<Replica> replicas = slice.getReplicas();
               for (Replica replica : replicas) {
                 // on a live node?
-                final boolean live = clusterState.liveNodesContain(replica.getNodeName());
+                final boolean live = clusterState.liveNodesContain(replica.getNode());
                 final boolean isActive = replica.getState() == Replica.State.ACTIVE;
                 if (!live || !isActive) {
                   // fail
@@ -139,7 +139,7 @@ public class ClusterStateUtil {
             Collection<Replica> replicas = slice.getReplicas();
             for (Replica replica : replicas) {
               // on a live node?
-              boolean live = clusterState.liveNodesContain(replica.getNodeName());
+              boolean live = clusterState.liveNodesContain(replica.getNode());
               String rcoreNodeName = replica.getName();
               String rbaseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
               if (live && coreNodeName.equals(rcoreNodeName)
@@ -193,7 +193,7 @@ public class ClusterStateUtil {
               for (Replica replica : replicas) {
                 // on a live node?
                 boolean live = clusterState.liveNodesContain(replica
-                    .getNodeName());
+                    .getNode());
                 if (live) {
                   // fail
                   success = false;
@@ -222,7 +222,7 @@ public class ClusterStateUtil {
     int liveAndActive = 0;
     for (Slice slice : slices) {
       for (Replica replica : slice.getReplicas()) {
-        boolean live = zkStateReader.getClusterState().liveNodesContain(replica.getNodeName());
+        boolean live = zkStateReader.getClusterState().liveNodesContain(replica.getNode());
         boolean active = replica.getState() == Replica.State.ACTIVE;
         if (live && active) {
           liveAndActive++;
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index 3a4a28a..847bc01 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -117,18 +117,18 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   }
 
   private void addNodeNameReplica(Replica replica) {
-    List<Replica> replicas = nodeNameReplicas.get(replica.getNodeName());
+    List<Replica> replicas = nodeNameReplicas.get(replica.getNode());
     if (replicas == null) {
       replicas = new ArrayList<>();
-      nodeNameReplicas.put(replica.getNodeName(), replicas);
+      nodeNameReplicas.put(replica.getNode(), replicas);
     }
     replicas.add(replica);
 
     if (replica.getStr(Slice.LEADER) != null) {
-      List<Replica> leaderReplicas = nodeNameLeaderReplicas.get(replica.getNodeName());
+      List<Replica> leaderReplicas = nodeNameLeaderReplicas.get(replica.getNode());
       if (leaderReplicas == null) {
         leaderReplicas = new ArrayList<>();
-        nodeNameLeaderReplicas.put(replica.getNodeName(), leaderReplicas);
+        nodeNameLeaderReplicas.put(replica.getNode(), leaderReplicas);
       }
       leaderReplicas.add(replica);
     }
@@ -353,7 +353,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   public String getShardId(String nodeName, String coreName) {
     for (Slice slice : this) {
       for (Replica replica : slice) {
-        if (Objects.equals(replica.getNodeName(), nodeName) && Objects.equals(replica.getCoreName(), coreName))
+        if (Objects.equals(replica.getNode(), nodeName) && Objects.equals(replica.getCoreName(), coreName))
           return slice.getName();
       }
     }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
index a022ed2..9f32775 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Replica.java
@@ -16,13 +16,22 @@
  */
 package org.apache.solr.common.cloud;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
+import java.util.function.BiPredicate;
 
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.Utils;
-public class Replica extends ZkNodeProps {
+
+import static org.apache.solr.common.ConditionalMapWriter.NON_NULL_VAL;
+import static org.apache.solr.common.ConditionalMapWriter.dedupeKeyPredicate;
+
+public class Replica extends ZkNodeProps implements MapWriter {
   
   /**
    * The replica's state. In general, if the node the replica is hosted on is
@@ -36,7 +45,7 @@ public class Replica extends ZkNodeProps {
      * <p>
      * <b>NOTE</b>: when the node the replica is hosted on crashes, the
      * replica's state may remain ACTIVE in ZK. To determine if the replica is
-     * truly active, you must also verify that its {@link Replica#getNodeName()
+     * truly active, you must also verify that its {@link Replica#getNode()
      * node} is under {@code /live_nodes} in ZK (or use
      * {@link ClusterState#liveNodesContain(String)}).
      * </p>
@@ -106,40 +115,93 @@ public class Replica extends ZkNodeProps {
     }
   }
 
-  private final String name;
-  private final String nodeName;
-  private final String core;
-  private final State state;
-  private final Type type;
-  public final String slice, collection;
+  // coreNode name
+  public final String name;
+  public final String node;
+  public final String core;
+  public final State state;
+  public final Type type;
+  public final String shard, collection;
+  public final boolean isLeader;
 
-  public Replica(String name, Map<String,Object> propMap, String collection, String slice) {
-    super(propMap);
+  public Replica(String name, Map<String,Object> map, String collection, String shard) {
+    super(new HashMap<>());
     this.collection = collection;
-    this.slice = slice;
+    map.remove(ZkStateReader.COLLECTION_PROP);
+    this.shard = shard;
+    map.remove(ZkStateReader.SHARD_ID_PROP);
     this.name = name;
-    this.nodeName = (String) propMap.get(ZkStateReader.NODE_NAME_PROP);
-    this.core = (String) propMap.get(ZkStateReader.CORE_NAME_PROP);
-    type = Type.get((String) propMap.get(ZkStateReader.REPLICA_TYPE));
-    Objects.requireNonNull(this.collection, "'collection' must not be null");
-    Objects.requireNonNull(this.slice, "'slice' must not be null");
-    Objects.requireNonNull(this.name, "'name' must not be null");
-    Objects.requireNonNull(this.nodeName, "'node_name' must not be null");
-    Objects.requireNonNull(this.core, "'core' must not be null");
-    Objects.requireNonNull(this.type, "'type' must not be null");
+    map.remove(ZkStateReader.CORE_NODE_NAME_PROP);
+    this.node = (String) map.get(ZkStateReader.NODE_NAME_PROP);
+    map.remove(ZkStateReader.NODE_NAME_PROP);
+    this.core = (String) map.get(ZkStateReader.CORE_NAME_PROP);
+    map.remove(ZkStateReader.CORE_NAME_PROP);
+    type = Type.get((String) map.remove(ZkStateReader.REPLICA_TYPE));
     if (propMap.get(ZkStateReader.STATE_PROP) != null) {
-      this.state = State.getState((String) propMap.get(ZkStateReader.STATE_PROP));
+      this.state = State.getState((String) map.remove(ZkStateReader.STATE_PROP));
     } else {
       this.state = State.ACTIVE;                         //Default to ACTIVE
-      propMap.put(ZkStateReader.STATE_PROP, state.toString());
     }
+    this.isLeader = Boolean.parseBoolean(String.valueOf(map.getOrDefault(ZkStateReader.LEADER_PROP, "false")));
+    this.propMap.putAll(map);
+    validate();
   }
 
-  public String getCollection(){
+  // clone constructor
+  public Replica(String name, String node, String collection, String shard, String core,
+                  boolean isLeader, State state, Type type, Map<String, Object> props) {
+    super(new HashMap<>());
+    this.name = name;
+    this.node = node;
+    this.state = state;
+    this.type = type;
+    this.isLeader = isLeader;
+    this.collection = collection;
+    this.shard = shard;
+    this.core = core;
+    if (props != null) {
+      this.propMap.putAll(props);
+    }
+  }
+
+  /**
+   * This constructor uses a map with one key (coreNode name) and a value that
+   * is a map containing all replica properties.
+   * @param nestedMap nested map containing replica properties
+   */
+  public Replica(Map<String, Object> nestedMap) {
+    this.name = nestedMap.keySet().iterator().next();
+    Map<String, Object> details = (Map<String, Object>) nestedMap.get(name);
+    Objects.requireNonNull(details);
+    details = Utils.getDeepCopy(details, 4);
+    this.collection = (String) details.get("collection");
+    this.shard = (String) details.get("shard");
+    this.core = (String) details.get("core");
+    this.node = (String) details.get("node_name");
+    this.isLeader = Boolean.parseBoolean(String.valueOf(details.getOrDefault(ZkStateReader.LEADER_PROP, "false")));
+    type = Replica.Type.valueOf(String.valueOf(details.getOrDefault(ZkStateReader.REPLICA_TYPE, "NRT")));
+    state = State.getState(String.valueOf(details.getOrDefault(ZkStateReader.STATE_PROP, "active")));
+    this.propMap.putAll(details);
+    validate();
+
+  }
+
+  private final void validate() {
+    Objects.requireNonNull(this.name, "'name' must not be null");
+    Objects.requireNonNull(this.core, "'core' must not be null");
+    Objects.requireNonNull(this.collection, "'collection' must not be null");
+    Objects.requireNonNull(this.shard, "'shard' must not be null");
+    Objects.requireNonNull(this.type, "'type' must not be null");
+    Objects.requireNonNull(this.state, "'state' must not be null");
+    Objects.requireNonNull(this.node, "'node' must not be null");
+  }
+
+  public String getCollection() {
     return collection;
   }
-  public String getSlice(){
-    return slice;
+
+  public String getShard() {
+    return shard;
   }
 
   @Override
@@ -148,14 +210,26 @@ public class Replica extends ZkNodeProps {
     if (o == null || getClass() != o.getClass()) return false;
     if (!super.equals(o)) return false;
 
-    Replica replica = (Replica) o;
+    Replica other = (Replica) o;
 
-    return name.equals(replica.name);
+    if (
+        name.equals(other.name) &&
+            collection.equals(other.collection) &&
+            core.equals(other.core) &&
+            isLeader == other.isLeader &&
+            node.equals(other.node) &&
+            shard.equals(other.shard) &&
+            type == other.type &&
+            propMap.equals(other.propMap)) {
+      return true;
+    } else {
+      return false;
+    }
   }
 
   @Override
   public int hashCode() {
-    return Objects.hash(name);
+    return Objects.hash(name, core, collection, shard, type, state, propMap);
   }
 
   /** Also known as coreNodeName. */
@@ -166,7 +240,8 @@ public class Replica extends ZkNodeProps {
   public String getCoreUrl() {
     return ZkCoreNodeProps.getCoreUrl(getStr(ZkStateReader.BASE_URL_PROP), core);
   }
-  public String getBaseUrl(){
+
+  public String getBaseUrl() {
     return getStr(ZkStateReader.BASE_URL_PROP);
   }
 
@@ -176,8 +251,8 @@ public class Replica extends ZkNodeProps {
   }
 
   /** The name of the node this replica resides on */
-  public String getNodeName() {
-    return nodeName;
+  public String getNode() {
+    return node;
   }
   
   /** Returns the {@link State} of this replica. */
@@ -186,7 +261,7 @@ public class Replica extends ZkNodeProps {
   }
 
   public boolean isActive(Set<String> liveNodes) {
-    return this.nodeName != null && liveNodes.contains(this.nodeName) && this.state == State.ACTIVE;
+    return this.node != null && liveNodes.contains(this.node) && this.state == State.ACTIVE;
   }
   
   public Type getType() {
@@ -204,8 +279,42 @@ public class Replica extends ZkNodeProps {
     return propertyValue;
   }
 
+  public Object get(String propName) {
+    return propMap.get(propName);
+  }
+
+  public Object get(String propName, Object defValue) {
+    Object o = propMap.get(propName);
+    if (o != null) {
+      return o;
+    } else {
+      return defValue;
+    }
+  }
+
+  public Object clone() {
+    return new Replica(name, node, collection, shard, core, isLeader, state, type,
+        propMap);
+  }
+
+  @Override
+  public void writeMap(MapWriter.EntryWriter ew) throws IOException {
+    BiPredicate<CharSequence, Object> p = dedupeKeyPredicate(new HashSet<>())
+        .and(NON_NULL_VAL);
+    ew.put(name, (MapWriter) ew1 -> {
+      ew1.put(ZkStateReader.CORE_NAME_PROP, core, p)
+          .put(ZkStateReader.SHARD_ID_PROP, shard, p)
+          .put(ZkStateReader.COLLECTION_PROP, collection, p)
+          .put(ZkStateReader.NODE_NAME_PROP, node, p)
+          .put(ZkStateReader.REPLICA_TYPE, type.toString(), p)
+          .put(ZkStateReader.STATE_PROP, state.toString(), p)
+          .put(ZkStateReader.LEADER_PROP, isLeader, p);
+      for (Map.Entry<String, Object> e : propMap.entrySet()) ew1.put(e.getKey(), e.getValue(), p);
+    });
+  }
+
   @Override
   public String toString() {
-    return name + ':' + Utils.toJSONString(propMap); // small enough, keep it on one line (i.e. no indent)
+    return Utils.toJSONString(this); // small enough, keep it on one line (i.e. no indent)
   }
 }
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 5139259..e0851ad 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -852,7 +852,7 @@ public class ZkStateReader implements SolrCloseable {
 
   public Replica getLeader(Set<String> liveNodes, DocCollection docCollection, String shard) {
     Replica replica = docCollection != null ? docCollection.getLeader(shard) : null;
-    if (replica != null && liveNodes.contains(replica.getNodeName())) {
+    if (replica != null && liveNodes.contains(replica.getNode())) {
       return replica;
     }
     return null;
@@ -862,7 +862,7 @@ public class ZkStateReader implements SolrCloseable {
     if (clusterState != null) {
       DocCollection docCollection = clusterState.getCollectionOrNull(collection);
       Replica replica = docCollection != null ? docCollection.getLeader(shard) : null;
-      if (replica != null && getClusterState().liveNodesContain(replica.getNodeName())) {
+      if (replica != null && getClusterState().liveNodesContain(replica.getNode())) {
         return replica;
       }
     }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
index 9bc1d61..ad14620 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy.java
@@ -119,11 +119,11 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
   public static String clusterState = Utils.toJSONString(loadFromResource("testPolicy.json"));
 
-  public static Map<String, Map<String, List<ReplicaInfo>>> getReplicaDetails(String node,
+  public static Map<String, Map<String, List<Replica>>> getReplicaDetails(String node,
                                                                               @SuppressWarnings({"rawtypes"})Map clusterState) {
     ValidatingJsonMap m = ValidatingJsonMap
         .getDeepCopy(clusterState, 6, true);
-    Map<String, Map<String, List<ReplicaInfo>>> result = new LinkedHashMap<>();
+    Map<String, Map<String, List<Replica>>> result = new LinkedHashMap<>();
 
     m.forEach((collName, o) -> {
       ValidatingJsonMap coll = (ValidatingJsonMap) o;
@@ -133,10 +133,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
           ValidatingJsonMap r = (ValidatingJsonMap) o2;
           String node_name = (String) r.get("node_name");
           if (!node_name.equals(node)) return;
-          Map<String, List<ReplicaInfo>> shardVsReplicaStats = result.computeIfAbsent(collName, k -> new HashMap<>());
-          List<ReplicaInfo> replicaInfos = shardVsReplicaStats.computeIfAbsent(shard, k -> new ArrayList<>());
-          replicaInfos.add(new ReplicaInfo(replicaName, (String) r.get("core"), collName, shard,
-              Replica.Type.get((String) r.get(ZkStateReader.REPLICA_TYPE)), node, r));
+          Map<String, List<Replica>> shardVsReplicaStats = result.computeIfAbsent(collName, k -> new HashMap<>());
+          List<Replica> replicaInfos = shardVsReplicaStats.computeIfAbsent(shard, k -> new ArrayList<>());
+          replicaInfos.add(new Replica(replicaName, node, collName, shard, (String) r.get("core"),
+              false, Replica.State.ACTIVE, Replica.Type.get((String) r.get(ZkStateReader.REPLICA_TYPE)), r));
         });
       });
     });
@@ -173,7 +173,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
         //e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
         Map<String, Object> result = new HashMap<>();
         metricsKeyVsTagReplica.forEach((k, v) -> {
@@ -270,7 +270,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
         //e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
         Map<String, Object> result = new HashMap<>();
         metricsKeyVsTagReplica.forEach((k, v) -> {
@@ -365,7 +365,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
         //e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
         Map<String, Object> result = new HashMap<>();
         metricsKeyVsTagReplica.forEach((k, v) -> {
@@ -475,7 +475,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+      protected Map<String, Object> fetchReplicaMetrics(String solrNode, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
         //e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
         Map<String, Object> result = new HashMap<>();
         metricsKeyVsTagReplica.forEach((k, v) -> {
@@ -854,7 +854,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
       }
 
       @Override
-      protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, ReplicaInfo>> metricsKeyVsTagReplica) {
+      protected Map<String, Object> fetchReplicaMetrics(String node, Map<String, Pair<String, Replica>> metricsKeyVsTagReplica) {
         //e.g: solr.core.perReplicaDataColl.shard1.replica_n4:INDEX.sizeInBytes
         Map<String, Object> result = new HashMap<>();
         metricsKeyVsTagReplica.forEach((k, v) -> {
@@ -905,7 +905,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
         assertEquals(3, violation.getViolatingReplicas().size());
         Set<String> expected = ImmutableSet.of("r1", "r3", "r5");
         for (Violation.ReplicaInfoAndErr replicaInfoAndErr : violation.getViolatingReplicas()) {
-          assertTrue(expected.contains(replicaInfoAndErr.replicaInfo.getCore()));
+          assertTrue(expected.contains(replicaInfoAndErr.replicaInfo.getCoreName()));
         }
       } else if (violation.node.equals("node5")) {
         assertEquals(-1, violation.replicaCountDelta.doubleValue(), 0.01);
@@ -1191,8 +1191,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
             String name = m3.keySet().iterator().next().toString();
             m3 = (Map) m3.get(name);
             Replica.Type type = Replica.Type.get((String) m3.get("type"));
-            l3.set(i, new ReplicaInfo(name, name
-                , coll.toString(), shard.toString(), type, (String) node, m3));
+            l3.set(i, new Replica(name, (String) node, coll.toString(), shard.toString(), name, false, Replica.State.ACTIVE, type, m3));
           }
         });
 
@@ -1239,9 +1238,9 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             @SuppressWarnings({"unchecked"})
-            Map<String, Map<String, List<ReplicaInfo>>> result = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
+            Map<String, Map<String, List<Replica>>> result = (Map<String, Map<String, List<Replica>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
             return result == null ? new HashMap<>() : result;
           }
         };
@@ -1481,10 +1480,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
       public NodeStateProvider getNodeStateProvider() {
         return new DelegatingNodeStateProvider(null) {
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             @SuppressWarnings({"unchecked"})
-            Map<String, Map<String, List<ReplicaInfo>>> o = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.fromJSONString("{c1: {s0:[{}]}}");
-            Utils.setObjectByPath(o, "c1/s0[0]", new ReplicaInfo("r0", "c1.s0", "c1", "s0", Replica.Type.NRT, "nodex", new HashMap<>()));
+            Map<String, Map<String, List<Replica>>> o = (Map<String, Map<String, List<Replica>>>) Utils.fromJSONString("{c1: {s0:[{}]}}");
+            Utils.setObjectByPath(o, "c1/s0[0]", new Replica("r0", "nodex", "c1", "s0", "c1.s0", false, Replica.State.ACTIVE, Replica.Type.NRT, new HashMap<>()));
             return o;
           }
 
@@ -1516,8 +1515,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
     Row r2 = r1.addReplica("c1", "s1", Replica.Type.NRT);
     assertEquals(1, r1.collectionVsShardVsReplicas.get("c1").get("s1").size());
     assertEquals(2, r2.collectionVsShardVsReplicas.get("c1").get("s1").size());
-    assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(0) instanceof ReplicaInfo);
-    assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(1) instanceof ReplicaInfo);
+    assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(0) instanceof Replica);
+    assertTrue(r2.collectionVsShardVsReplicas.get("c1").get("s1").get(1) instanceof Replica);
   }
 
   public void testMerge() {
@@ -2019,8 +2018,10 @@ public class TestPolicy extends SolrTestCaseJ4 {
         "      {'core_node2':{}}]}}}");
     Map m = (Map) Utils.getObjectByPath(replicaInfoMap, false, "127.0.0.1:60089_solr/compute_plan_action_test");
     m.put("shard1", Arrays.asList(
-        new ReplicaInfo("core_node1", "core_node1", "compute_plan_action_test", "shard1", Replica.Type.NRT, "127.0.0.1:60089_solr", Collections.emptyMap()),
-        new ReplicaInfo("core_node2", "core_node2", "compute_plan_action_test", "shard1", Replica.Type.NRT, "127.0.0.1:60089_solr", Collections.emptyMap())));
+        new Replica("core_node1", "127.0.0.1:60089_solr", "compute_plan_action_test", "shard1", "core_node1",
+            false, Replica.State.ACTIVE, Replica.Type.NRT, Collections.emptyMap()),
+        new Replica("core_node2", "127.0.0.1:60089_solr", "compute_plan_action_test", "shard1", "core_node2",
+            false, Replica.State.ACTIVE, Replica.Type.NRT, Collections.emptyMap())));
 
     @SuppressWarnings({"unchecked", "rawtypes"})
     Map<String, Map<String, Object>> tagsMap = (Map) Utils.fromJSONString("{" +
@@ -2056,8 +2057,8 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
           @Override
           @SuppressWarnings({"unchecked"})
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
-            return (Map<String, Map<String, List<ReplicaInfo>>>) replicaInfoMap.get(node);
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
+            return (Map<String, Map<String, List<Replica>>>) replicaInfoMap.get(node);
           }
         };
       }
@@ -2136,7 +2137,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             return cloudManager.getNodeStateProvider().getReplicaInfo(node, keys);
           }
         };
@@ -2214,7 +2215,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             return getReplicaDetails(node, clusterS);
           }
         };
@@ -2277,7 +2278,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             return getReplicaDetails(node, (Map)Utils.fromJSONString(clusterState));
           }
         };
@@ -2351,7 +2352,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             return getReplicaDetails(node, (Map)Utils.fromJSONString(clusterState));
           }
         };
@@ -2805,16 +2806,16 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
           @Override
           @SuppressWarnings({"unchecked", "rawtypes"})
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             if (node.equals("node1")) {
               Map m = Utils.makeMap("newColl",
-                  Utils.makeMap("shard1", Collections.singletonList(new ReplicaInfo("r1", "shard1",
+                  Utils.makeMap("shard1", Collections.singletonList(new Replica("r1", "shard1",
                       new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node1", ZkStateReader.CORE_NAME_PROP, "core1"), "newColl", "shard1"),
                       Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
               return m;
             } else if (node.equals("node2")) {
               Map m = Utils.makeMap("newColl",
-                  Utils.makeMap("shard2", Collections.singletonList(new ReplicaInfo("r1", "shard2",
+                  Utils.makeMap("shard2", Collections.singletonList(new Replica("r1", "shard2",
                       new Replica("r1", Utils.makeMap(ZkStateReader.NODE_NAME_PROP, "node2", ZkStateReader.CORE_NAME_PROP, "core2"),"newColl", "shard2"),
                       Utils.makeMap(FREEDISK.perReplicaValue, 200)))));
               return m;
@@ -2857,27 +2858,27 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
   public void testMoveReplicaLeaderlast() {
 
-    List<Pair<ReplicaInfo, Row>> validReplicas = new ArrayList<>();
+    List<Pair<Replica, Row>> validReplicas = new ArrayList<>();
     Map<String, Object> propMap = Utils.makeMap(
         "leader", "true",
         ZkStateReader.NODE_NAME_PROP, "node1",
         ZkStateReader.REPLICA_TYPE, Replica.Type.NRT.toString(),
         ZkStateReader.CORE_NAME_PROP, "core1");
     Replica replica = new Replica("r1", propMap, "c1", "s1");
-    ReplicaInfo replicaInfo = new ReplicaInfo(replica.collection, replica.slice ,replica, new HashMap<>());
+    Replica replicaInfo = new Replica(replica.collection, replica.shard,replica, new HashMap<>());
     validReplicas.add(new Pair<>(replicaInfo, null));
 
-    replicaInfo = new ReplicaInfo("r4", "c1_s2_r1", "c1", "s2", Replica.Type.NRT, "n1", Collections.singletonMap("leader", "true"));
+    replicaInfo = new Replica("r4", "n1", "c1_s2_r1", "c1", "s2", true, Replica.State.ACTIVE, Replica.Type.NRT, null);
     validReplicas.add(new Pair<>(replicaInfo, null));
 
 
     propMap.put("leader", false);
     replica = new Replica("r2", propMap,"c1","s1");
-    replicaInfo = new ReplicaInfo(replica.collection, replica.slice, replica, new HashMap<>());
+    replicaInfo = new Replica(replica.collection, replica.shard, replica, new HashMap<>());
     validReplicas.add(new Pair<>(replicaInfo, null));
 
     replica = new Replica("r3", propMap,"c1","s1");
-    replicaInfo = new ReplicaInfo(replica.collection,replica.slice, replica, new HashMap<>());
+    replicaInfo = new Replica(replica.collection,replica.shard, replica, new HashMap<>());
     validReplicas.add(new Pair<>(replicaInfo, null));
 
 
@@ -3181,7 +3182,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             @SuppressWarnings({"unchecked"})
             Map<String, Map<String, List<ReplicaInfo>>> result = (Map<String, Map<String, List<ReplicaInfo>>>) Utils.getObjectByPath(m, false, Arrays.asList("replicaInfo", node));
             return result == null ? new HashMap<>() : result;
@@ -3262,7 +3263,7 @@ public class TestPolicy extends SolrTestCaseJ4 {
 
           @Override
           @SuppressWarnings({"unchecked"})
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             //return Collections.EMPTY_MAP;
             return replicaInfoMap;
           }
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
index b617193..ade0314 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/cloud/autoscaling/TestPolicy2.java
@@ -42,6 +42,7 @@ import org.apache.solr.client.solrj.impl.ClusterStateProvider;
 import org.apache.solr.client.solrj.impl.SolrClientNodeStateProvider;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ReplicaPosition;
 import org.apache.solr.common.util.Utils;
 import org.junit.Ignore;
@@ -185,8 +186,8 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
     Set<String> nodes = new HashSet<>(nodeVals.keySet());
     clusterState.getCollectionStates().forEach((s, collectionRef) -> collectionRef.get()
         .forEachReplica((s12, replica) -> {
-          nodes.add(replica.getNodeName());
-          coreCount.computeIfAbsent(replica.getNodeName(), s1 -> new AtomicInteger(0))
+          nodes.add(replica.getNode());
+          coreCount.computeIfAbsent(replica.getNode(), s1 -> new AtomicInteger(0))
               .incrementAndGet();
         }));
 
@@ -230,17 +231,17 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
           }
 
           @Override
-          public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+          public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
             @SuppressWarnings({"unchecked"})
             Map<String, Map<String, List<ReplicaInfo>>> result = nodeVsCollectionVsShardVsReplicaInfo.computeIfAbsent(node, Utils.NEW_HASHMAP_FUN);
             if (!keys.isEmpty()) {
               Row.forEachReplica(result, replicaInfo -> {
                 for (String key : keys) {
-                  if (!replicaInfo.getVariables().containsKey(key)) {
+                  if (!replicaInfo.getProperties().containsKey(key)) {
                     replicaVals.stream()
                         .filter(it -> replicaInfo.getCore().equals(it.get("core")))
                         .findFirst()
-                        .ifPresent(map -> replicaInfo.getVariables().put(key, map.get(key)));
+                        .ifPresent(map -> replicaInfo.getProperties().put(key, map.get(key)));
                   }
                 }
               });
@@ -289,7 +290,7 @@ public class TestPolicy2 extends SolrTestCaseJ4 {
       }
 
       @Override
-      public Map<String, Map<String, List<ReplicaInfo>>> getReplicaInfo(String node, Collection<String> keys) {
+      public Map<String, Map<String, List<Replica>>> getReplicaInfo(String node, Collection<String> keys) {
         return nodeVsCollectionVsShardVsReplicaInfo.get(node) == null ?
             Collections.emptyMap() :
             nodeVsCollectionVsShardVsReplicaInfo.get(node);
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
index edef269..768f307 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/MathExpressionTest.java
@@ -4044,7 +4044,7 @@ public void testCache() throws Exception {
     ClusterState clusterState = cluster.getSolrClient().getClusterStateProvider().getClusterState();
     String collection = useAlias ? COLLECTIONORALIAS + "_collection" : COLLECTIONORALIAS;
     DocCollection coll = clusterState.getCollection(collection);
-    String node = coll.getReplicas().iterator().next().getNodeName();
+    String node = coll.getReplicas().iterator().next().getNode();
     String url = null;
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       if (jetty.getNodeName().equals(node)) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
index 6c88ffe..5bd7b8c 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/io/stream/StreamDecoratorTest.java
@@ -3663,7 +3663,7 @@ public class StreamDecoratorTest extends SolrCloudTestCase {
     // find a node with a replica
     ClusterState clusterState = cluster.getSolrClient().getClusterStateProvider().getClusterState();
     DocCollection coll = clusterState.getCollection(COLLECTIONORALIAS);
-    String node = coll.getReplicas().iterator().next().getNodeName();
+    String node = coll.getReplicas().iterator().next().getNode();
     String url = null;
     for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
       if (jetty.getNodeName().equals(node)) {
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
index c663e99..8aa3500 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV2Request.java
@@ -158,7 +158,7 @@ public class TestV2Request extends SolrCloudTestCase {
     System.out.println("livenodes: " + cs.getLiveNodes());
 
     String[] node = new String[1];
-    cs.getCollection("v2forward").forEachReplica((s, replica) -> node[0] = replica.getNodeName());
+    cs.getCollection("v2forward").forEachReplica((s, replica) -> node[0] = replica.getNode());
 
     //find a node that does not have a replica for this collection
     final String[] testNode = new String[1];
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
index 3998dd0..fec51fe 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/NodePreferenceRulesComparatorTest.java
@@ -36,8 +36,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
     List<PreferenceRule> rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_LOCATION + ":http://host2:8983");
     NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
     replicas.sort(comparator);
-    assertEquals("node2", replicas.get(0).getNodeName());
-    assertEquals("node1", replicas.get(1).getNodeName());
+    assertEquals("node2", replicas.get(0).getNode());
+    assertEquals("node1", replicas.get(1).getNode());
 
   }
 
@@ -49,8 +49,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
     NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
 
     replicas.sort(comparator);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node2", replicas.get(1).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node2", replicas.get(1).getNode());
 
     // reversed rule
     rules = PreferenceRule.from(ShardParams.SHARDS_PREFERENCE_REPLICA_TYPE + ":TLOG," +
@@ -58,8 +58,8 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
     comparator = new NodePreferenceRulesComparator(rules, null);
 
     replicas.sort(comparator);
-    assertEquals("node2", replicas.get(0).getNodeName());
-    assertEquals("node1", replicas.get(1).getNodeName());
+    assertEquals("node2", replicas.get(0).getNode());
+    assertEquals("node1", replicas.get(1).getNode());
   }
 
   @SuppressWarnings("unchecked")
@@ -86,10 +86,10 @@ public class NodePreferenceRulesComparatorTest extends SolrTestCaseJ4 {
     NodePreferenceRulesComparator comparator = new NodePreferenceRulesComparator(rules, null);
 
     replicas.sort(comparator);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node4", replicas.get(1).getNodeName());
-    assertEquals("node2", replicas.get(2).getNodeName());
-    assertEquals("node3", replicas.get(3).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node4", replicas.get(1).getNode());
+    assertEquals("node2", replicas.get(2).getNode());
+    assertEquals("node3", replicas.get(3).getNode());
   }
 
   @Test(expected = IllegalArgumentException.class)
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
index c0ebad3..bdab386 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/routing/RequestReplicaListTransformerGeneratorTest.java
@@ -42,30 +42,30 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
 
     ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node2", replicas.get(1).getNodeName());
-    assertEquals("node3", replicas.get(2).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node2", replicas.get(1).getNode());
+    assertEquals("node3", replicas.get(2).getNode());
 
     params.set("routingPreference", "1");
     rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node2", replicas.get(0).getNodeName());
-    assertEquals("node3", replicas.get(1).getNodeName());
-    assertEquals("node1", replicas.get(2).getNodeName());
+    assertEquals("node2", replicas.get(0).getNode());
+    assertEquals("node3", replicas.get(1).getNode());
+    assertEquals("node1", replicas.get(2).getNode());
 
     params.set("routingPreference", "2");
     rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node3", replicas.get(0).getNodeName());
-    assertEquals("node1", replicas.get(1).getNodeName());
-    assertEquals("node2", replicas.get(2).getNodeName());
+    assertEquals("node3", replicas.get(0).getNode());
+    assertEquals("node1", replicas.get(1).getNode());
+    assertEquals("node2", replicas.get(2).getNode());
 
     params.set("routingPreference", "3");
     rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node2", replicas.get(1).getNodeName());
-    assertEquals("node3", replicas.get(2).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node2", replicas.get(1).getNode());
+    assertEquals("node3", replicas.get(2).getNode());
   }
 
   @SuppressWarnings("unchecked")
@@ -110,20 +110,20 @@ public class RequestReplicaListTransformerGeneratorTest extends SolrTestCaseJ4 {
     params.add(ShardParams.SHARDS_PREFERENCE, rulesParam);
     ReplicaListTransformer rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node2", replicas.get(1).getNodeName());
-    assertEquals("node4", replicas.get(2).getNodeName());
-    assertEquals("node3", replicas.get(3).getNodeName());
-    assertEquals("node5", replicas.get(4).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node2", replicas.get(1).getNode());
+    assertEquals("node4", replicas.get(2).getNode());
+    assertEquals("node3", replicas.get(3).getNode());
+    assertEquals("node5", replicas.get(4).getNode());
 
     params.set("routingPreference", "1");
     rlt = generator.getReplicaListTransformer(params);
     rlt.transform(replicas);
-    assertEquals("node1", replicas.get(0).getNodeName());
-    assertEquals("node4", replicas.get(1).getNodeName());
-    assertEquals("node2", replicas.get(2).getNodeName());
-    assertEquals("node5", replicas.get(3).getNodeName());
-    assertEquals("node3", replicas.get(4).getNodeName());
+    assertEquals("node1", replicas.get(0).getNode());
+    assertEquals("node4", replicas.get(1).getNode());
+    assertEquals("node2", replicas.get(2).getNode());
+    assertEquals("node5", replicas.get(3).getNode());
+    assertEquals("node3", replicas.get(4).getNode());
   }
 
   @SuppressWarnings("unchecked")
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
index 1a5bf98..f088e93 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractDistribZkTestBase.java
@@ -175,7 +175,7 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
             if (verbose) System.out.println("replica:" + shard.getValue().getName() + " rstate:"
                 + shard.getValue().getStr(ZkStateReader.STATE_PROP)
                 + " live:"
-                + liveNodes.contains(shard.getValue().getNodeName()));
+                + liveNodes.contains(shard.getValue().getNode()));
             final Replica.State state = shard.getValue().getState();
             if ((state == Replica.State.RECOVERING || state == Replica.State.DOWN
                 || state == Replica.State.RECOVERY_FAILED)
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
index 5c01018..d79ed1a 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/AbstractFullDistribZkTestBase.java
@@ -747,7 +747,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
   }
 
   protected int getReplicaPort(Replica replica) {
-    String replicaNode = replica.getNodeName();
+    String replicaNode = replica.getNode();
     String tmp = replicaNode.substring(replicaNode.indexOf(':')+1);
     if (tmp.indexOf('_') != -1)
       tmp = tmp.substring(0,tmp.indexOf('_'));
@@ -1933,7 +1933,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       for (String sliceName : slices.keySet()) {
         for (Replica replica : slices.get(sliceName).getReplicas()) {
           if (nodesAllowedToRunShards != null && !nodesAllowedToRunShards.contains(replica.getStr(ZkStateReader.NODE_NAME_PROP))) {
-            return "Shard " + replica.getName() + " created on node " + replica.getNodeName() + " not allowed to run shards for the created collection " + collectionName;
+            return "Shard " + replica.getName() + " created on node " + replica.getNode() + " not allowed to run shards for the created collection " + collectionName;
           }
         }
         totalShards += slices.get(sliceName).getReplicas().size();
@@ -2117,7 +2117,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
       assertNotNull(leader);
       if (log.isInfoEnabled()) {
         log.info("Found {}  replicas and leader on {} for {} in {}"
-            , replicas.size(), leader.getNodeName(), shardId, testCollectionName);
+            , replicas.size(), leader.getNode(), shardId, testCollectionName);
       }
 
       // ensure all replicas are "active" and identify the non-leader replica
@@ -2261,7 +2261,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
         fail("Unable to get leader indexVersion");
       }
       for (Replica pullReplica:s.getReplicas(EnumSet.of(Replica.Type.PULL,Replica.Type.TLOG))) {
-        if (!zkStateReader.getClusterState().liveNodesContain(pullReplica.getNodeName())) {
+        if (!zkStateReader.getClusterState().liveNodesContain(pullReplica.getNode())) {
           continue;
         }
         while (true) {
@@ -2273,7 +2273,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
             }
             
             // Make sure the host is serving the correct version
-            try (SolrCore core = containers.get(pullReplica.getNodeName()).getCore(pullReplica.getCoreName())) {
+            try (SolrCore core = containers.get(pullReplica.getNode()).getCore(pullReplica.getCoreName())) {
               RefCounted<SolrIndexSearcher> ref = core.getRegisteredSearcher();
               try {
                 SolrIndexSearcher searcher = ref.get();
diff --git a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
index 7dfdc36..30b8e13 100644
--- a/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
+++ b/solr/test-framework/src/java/org/apache/solr/cloud/ChaosMonkey.java
@@ -659,7 +659,7 @@ public class ChaosMonkey {
         m.find();
         String jettyPort = m.group(1);
         builder.append(String.format(Locale.ROOT, "%s(%s): {state: %s, type: %s, leader: %s, Live: %s}, ", 
-            replica.getName(), jettyPort, replica.getState(), replica.getType(), (replica.get("leader")!= null), zkStateReader.getClusterState().liveNodesContain(replica.getNodeName())));
+            replica.getName(), jettyPort, replica.getState(), replica.getType(), (replica.get("leader")!= null), zkStateReader.getClusterState().liveNodesContain(replica.getNode())));
       }
       if (slice.getReplicas().size() > 0) {
         builder.setLength(builder.length() - 2);