You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2020/11/09 04:08:03 UTC

[lucene-solr] 01/04: @1101 Overseer overhaul wip.

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

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

commit ce9077d401c6840d5ee59a4a22fef18bd0c04984
Author: markrmiller@gmail.com <ma...@gmail.com>
AuthorDate: Sun Nov 8 20:47:13 2020 -0600

    @1101 Overseer overhaul wip.
---
 .../stream/AnalyticsShardRequestManager.java       |   3 +-
 .../solr/prometheus/scraper/SolrCloudScraper.java  |   4 +-
 .../prometheus/scraper/SolrCloudScraperTest.java   |   4 +-
 .../org/apache/solr/cloud/CloudDescriptor.java     |  15 -
 .../src/java/org/apache/solr/cloud/CloudUtil.java  |  14 +-
 .../java/org/apache/solr/cloud/DistributedMap.java |  25 +-
 .../org/apache/solr/cloud/ElectionContext.java     |   6 +-
 .../java/org/apache/solr/cloud/LeaderElector.java  | 147 ++--
 .../src/java/org/apache/solr/cloud/Overseer.java   | 909 ++++++++++++---------
 .../OverseerCollectionConfigSetProcessor.java      |  65 +-
 .../cloud/OverseerConfigSetMessageHandler.java     |   9 +-
 .../apache/solr/cloud/OverseerElectionContext.java |  44 +-
 .../solr/cloud/OverseerTaskExecutorTask.java       | 157 ++++
 .../apache/solr/cloud/OverseerTaskProcessor.java   |  28 +-
 .../org/apache/solr/cloud/OverseerTaskQueue.java   | 248 +++---
 .../solr/cloud/RecoveringCoreTermWatcher.java      |  10 +-
 .../org/apache/solr/cloud/RecoveryStrategy.java    | 123 +--
 .../solr/cloud/ShardLeaderElectionContext.java     | 175 ++--
 .../solr/cloud/ShardLeaderElectionContextBase.java |  14 +-
 .../java/org/apache/solr/cloud/SyncStrategy.java   |  89 +-
 .../java/org/apache/solr/cloud/ZkController.java   | 662 +++++----------
 .../org/apache/solr/cloud/ZkDistributedQueue.java  | 730 +++++++----------
 .../java/org/apache/solr/cloud/ZkShardTerms.java   |   2 +-
 .../solr/cloud/api/collections/AddReplicaCmd.java  | 312 ++++---
 .../solr/cloud/api/collections/AliasCmd.java       |   2 +-
 .../apache/solr/cloud/api/collections/Assign.java  |  97 +--
 .../solr/cloud/api/collections/BackupCmd.java      |  19 +-
 .../solr/cloud/api/collections/CreateAliasCmd.java |   2 +-
 .../cloud/api/collections/CreateCollectionCmd.java | 283 +++----
 .../solr/cloud/api/collections/CreateShardCmd.java | 120 ++-
 .../cloud/api/collections/CreateSnapshotCmd.java   |   6 +-
 .../solr/cloud/api/collections/DeleteAliasCmd.java |   2 +-
 .../cloud/api/collections/DeleteCollectionCmd.java |  43 +-
 .../solr/cloud/api/collections/DeleteNodeCmd.java  |  13 +-
 .../cloud/api/collections/DeleteReplicaCmd.java    | 254 +++---
 .../solr/cloud/api/collections/DeleteShardCmd.java |  80 +-
 .../cloud/api/collections/DeleteSnapshotCmd.java   |   6 +-
 .../api/collections/MaintainRoutedAliasCmd.java    |   2 +-
 .../solr/cloud/api/collections/MigrateCmd.java     |  31 +-
 .../cloud/api/collections/ModifyCollectionCmd.java |  28 +
 .../solr/cloud/api/collections/MoveReplicaCmd.java | 220 ++---
 .../OverseerCollectionMessageHandler.java          | 223 +++--
 .../cloud/api/collections/OverseerRoleCmd.java     |   3 +-
 .../cloud/api/collections/OverseerStatusCmd.java   |  18 +-
 .../api/collections/ReindexCollectionCmd.java      |   6 +-
 .../solr/cloud/api/collections/RenameCmd.java      |   2 +-
 .../solr/cloud/api/collections/ReplaceNodeCmd.java | 213 +++--
 .../solr/cloud/api/collections/RestoreCmd.java     |  33 +-
 .../cloud/api/collections/SetAliasPropCmd.java     |   2 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  | 506 ++++++++----
 .../solr/cloud/overseer/ClusterStateMutator.java   |  33 +-
 .../solr/cloud/overseer/CollectionMutator.java     |  19 +-
 .../apache/solr/cloud/overseer/NodeMutator.java    |  10 +-
 .../apache/solr/cloud/overseer/ReplicaMutator.java | 274 ++-----
 .../apache/solr/cloud/overseer/SliceMutator.java   | 119 ++-
 .../apache/solr/cloud/overseer/ZkStateWriter.java  | 443 +++++-----
 .../apache/solr/cloud/overseer/ZkWriteCommand.java |  16 +-
 .../java/org/apache/solr/core/CoreContainer.java   | 108 +--
 .../org/apache/solr/core/HdfsDirectoryFactory.java |   2 +-
 .../src/java/org/apache/solr/core/PluginInfo.java  |   4 -
 .../src/java/org/apache/solr/core/SolrCore.java    |  12 +-
 .../src/java/org/apache/solr/core/ZkContainer.java |  28 +-
 .../core/snapshots/CollectionSnapshotMetaData.java |   2 +-
 .../solr/core/snapshots/SolrSnapshotsTool.java     |   2 +-
 .../java/org/apache/solr/handler/IndexFetcher.java | 177 ++--
 .../apache/solr/handler/ReplicationHandler.java    |  33 +-
 .../apache/solr/handler/admin/BackupCoreOp.java    |   8 +
 .../solr/handler/admin/CollectionsHandler.java     |  22 +-
 .../solr/handler/admin/CoreAdminHandler.java       |   1 -
 .../solr/handler/admin/CoreAdminOperation.java     |   4 +-
 .../solr/handler/admin/MetricsHistoryHandler.java  |   2 +-
 .../apache/solr/handler/admin/PrepRecoveryOp.java  | 104 +--
 .../solr/handler/admin/RebalanceLeaders.java       |   2 -
 .../solr/handler/admin/RequestSyncShardOp.java     |   5 +-
 .../org/apache/solr/handler/admin/SplitOp.java     |  13 +-
 .../handler/component/RealTimeGetComponent.java    |   4 +-
 .../org/apache/solr/logging/MDCLoggingContext.java |   2 +-
 .../apache/solr/metrics/SolrCoreMetricManager.java |   6 +-
 .../metrics/reporters/solr/SolrShardReporter.java  |   2 +-
 .../org/apache/solr/schema/ManagedIndexSchema.java |  11 +-
 .../java/org/apache/solr/schema/SchemaManager.java |   2 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |   5 +-
 .../apache/solr/servlet/SolrDispatchFilter.java    |   4 +-
 .../org/apache/solr/update/SolrCmdDistributor.java |  33 +-
 .../src/java/org/apache/solr/update/UpdateLog.java |   3 +
 .../org/apache/solr/update/UpdateShardHandler.java |   1 -
 .../processor/DistributedZkUpdateProcessor.java    |  72 +-
 .../DocExpirationUpdateProcessorFactory.java       |   4 +-
 .../processor/RoutedAliasUpdateProcessor.java      |   2 +-
 .../src/java/org/apache/solr/util/ExportTool.java  |  12 +-
 .../src/java/org/apache/solr/util/SolrCLI.java     |  10 +-
 .../java/org/apache/solr/util/SolrLogLayout.java   |   2 +-
 .../apache/solr/HelloWorldSolrCloudTestCase.java   |   5 +-
 .../test/org/apache/solr/cloud/AddReplicaTest.java |  57 +-
 .../cloud/AssignBackwardCompatibilityTest.java     |   4 +-
 .../apache/solr/cloud/BasicDistributedZkTest.java  |   6 +-
 .../solr/cloud/ChaosMonkeyShardSplitTest.java      |   2 +-
 .../org/apache/solr/cloud/CleanupOldIndexTest.java |   1 +
 .../solr/cloud/ClusterStateMockUtilTest.java       |   2 +-
 .../apache/solr/cloud/ClusterStateUpdateTest.java  |  10 +-
 .../org/apache/solr/cloud/CollectionPropsTest.java |   2 +-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java |  80 +-
 .../solr/cloud/CreateCollectionCleanupTest.java    |   2 +
 .../test/org/apache/solr/cloud/DeleteNodeTest.java |   1 +
 .../org/apache/solr/cloud/DeleteReplicaTest.java   |  81 +-
 .../org/apache/solr/cloud/DeleteShardTest.java     |  28 +-
 .../DistribDocExpirationUpdateProcessorTest.java   |   4 +-
 .../apache/solr/cloud/DistributedQueueTest.java    |  66 +-
 .../solr/cloud/DistributedVersionInfoTest.java     |   4 +-
 .../org/apache/solr/cloud/ForceLeaderTest.java     |   3 +-
 .../solr/cloud/FullSolrCloudDistribCmdsTest.java   |   4 +-
 .../solr/cloud/HttpPartitionOnCommitTest.java      |   5 +-
 .../org/apache/solr/cloud/HttpPartitionTest.java   |  18 +-
 .../solr/cloud/LeaderElectionContextKeyTest.java   |   2 +-
 .../org/apache/solr/cloud/LeaderElectionTest.java  |  24 +-
 .../cloud/LeaderFailoverAfterPartitionTest.java    |   7 +-
 .../solr/cloud/MissingSegmentRecoveryTest.java     |   6 +-
 .../solr/cloud/MoveReplicaHDFSFailoverTest.java    |   1 -
 .../org/apache/solr/cloud/MoveReplicaTest.java     |  31 +-
 .../org/apache/solr/cloud/NodeMutatorTest.java     |  47 +-
 .../test/org/apache/solr/cloud/OverseerTest.java   |  98 ++-
 .../test/org/apache/solr/cloud/RecoveryZkTest.java |   6 +-
 .../org/apache/solr/cloud/ReplaceNodeTest.java     |   1 +
 .../apache/solr/cloud/ReplicationFactorTest.java   |  11 +-
 .../apache/solr/cloud/ShardRoutingCustomTest.java  |   2 +-
 .../org/apache/solr/cloud/ShardRoutingTest.java    |  10 +-
 .../cloud/SharedFSAutoReplicaFailoverTest.java     |   2 +-
 .../apache/solr/cloud/SolrCloudBridgeTestCase.java |  33 +-
 .../apache/solr/cloud/SolrCloudExampleTest.java    |   4 +-
 .../test/org/apache/solr/cloud/SplitShardTest.java |  20 +-
 .../solr/cloud/TestAuthenticationFramework.java    |   1 +
 .../apache/solr/cloud/TestBaseStatsCacheCloud.java |   4 +-
 .../apache/solr/cloud/TestCloudDeleteByQuery.java  |  12 +-
 .../org/apache/solr/cloud/TestCloudRecovery2.java  |   7 +-
 .../solr/cloud/TestCloudSearcherWarming.java       |  21 +-
 .../solr/cloud/TestConfigSetsAPIExclusivity.java   |   1 +
 .../cloud/TestLeaderElectionWithEmptyReplica.java  |   6 +-
 .../solr/cloud/TestLocalStatsCacheCloud.java       |   1 -
 .../solr/cloud/TestOnReconnectListenerSupport.java |   4 +-
 .../org/apache/solr/cloud/TestPrepRecovery.java    |  11 +-
 .../org/apache/solr/cloud/TestPullReplica.java     |   2 +-
 .../solr/cloud/TestShortCircuitedRequests.java     |  19 +-
 .../org/apache/solr/cloud/TestTlogReplica.java     |  32 +-
 .../cloud/TestTolerantUpdateProcessorCloud.java    |  14 +-
 .../apache/solr/cloud/UnloadDistributedZkTest.java |   8 +-
 .../org/apache/solr/cloud/ZkControllerTest.java    |   4 +-
 .../org/apache/solr/cloud/ZkShardTermsTest.java    |  48 +-
 .../org/apache/solr/cloud/ZkSolrClientTest.java    |   2 +-
 .../CollectionsAPIAsyncDistributedZkTest.java      |  32 +-
 .../CollectionsAPIDistClusterPerZkTest.java        |  47 +-
 .../solr/cloud/api/collections/ShardSplitTest.java |  11 +-
 .../SimpleCollectionCreateDeleteTest.java          |   1 +
 .../cloud/api/collections/SplitByPrefixTest.java   | 143 ++--
 .../cloud/overseer/TestClusterStateMutator.java    |   6 +-
 .../solr/cloud/overseer/ZkStateReaderTest.java     | 174 ----
 .../solr/cloud/overseer/ZkStateWriterTest.java     | 224 -----
 .../solr/core/ConfigureRecoveryStrategyTest.java   |   3 +-
 .../test/org/apache/solr/core/CoreSorterTest.java  |   2 +-
 .../core/snapshots/TestSolrCloudSnapshots.java     |  10 +-
 .../solr/handler/TestStressThreadBackup.java       |   2 +-
 .../solr/handler/admin/IndexSizeEstimatorTest.java |   5 +-
 .../handler/admin/ZookeeperStatusHandlerTest.java  |  34 +-
 .../handler/component/CloudReplicaSourceTest.java  |   2 +
 .../DistributedQueryComponentOptimizationTest.java |   4 +-
 .../handler/component/ShardsWhitelistTest.java     |  34 +-
 .../component/TestTrackingShardHandlerFactory.java |   8 +-
 .../reporters/solr/SolrShardReporterTest.java      |   6 +-
 .../transform/TestSubQueryTransformerDistrib.java  |   1 +
 .../apache/solr/search/stats/TestDistribIDF.java   |   7 +-
 .../processor/RoutedAliasUpdateProcessorTest.java  |   2 +-
 .../test/org/apache/solr/util/TestExportTool.java  |   5 +-
 solr/solrj/build.gradle                            |   2 +
 .../solr/client/solrj/cloud/DistributedQueue.java  |  12 +-
 .../solr/client/solrj/cloud/ReplicaInfo.java       |  18 +-
 .../client/solrj/impl/BaseCloudSolrClient.java     |  84 +-
 .../solr/client/solrj/impl/CloudSolrClient.java    |   1 +
 .../solr/client/solrj/impl/Http2SolrClient.java    |  74 +-
 .../solr/client/solrj/impl/LBHttp2SolrClient.java  |  19 +-
 .../solrj/impl/SolrClientNodeStateProvider.java    |   2 +-
 .../client/solrj/impl/ZkDistribStateManager.java   |   2 +-
 .../solr/client/solrj/io/sql/StatementImpl.java    |   3 +-
 .../solrj/io/stream/FeaturesSelectionStream.java   |   3 +-
 .../client/solrj/io/stream/TextLogitStream.java    |   3 +-
 .../solr/client/solrj/io/stream/TopicStream.java   |   2 +-
 .../client/solrj/request/CoreAdminRequest.java     |  43 +-
 .../solr/client/solrj/request/CoreStatus.java      |   4 +
 .../src/java/org/apache/solr/common/ParWork.java   |  37 +-
 .../org/apache/solr/common/cloud/ClusterState.java |  56 +-
 .../solr/common/cloud/CompositeIdRouter.java       |   5 +-
 .../solr/common/cloud/ConnectionManager.java       |   4 +-
 .../apache/solr/common/cloud/DocCollection.java    |  19 +-
 .../java/org/apache/solr/common/cloud/Replica.java |  36 +-
 .../java/org/apache/solr/common/cloud/Slice.java   |  18 +-
 .../org/apache/solr/common/cloud/SolrZkClient.java | 199 +++--
 .../apache/solr/common/cloud/ZkCmdExecutor.java    |  12 +-
 .../apache/solr/common/cloud/ZkCoreNodeProps.java  |  13 +-
 .../org/apache/solr/common/cloud/ZkNodeProps.java  |   4 +-
 .../apache/solr/common/cloud/ZkStateReader.java    | 274 +++++--
 .../apache/solr/common/params/CoreAdminParams.java |   2 -
 .../solr/common/params/RequiredSolrParams.java     |   2 +-
 .../solr/common/util/SolrInternalHttpClient.java   |   6 +
 .../java/org/apache/solr/common/util/TimeOut.java  |   2 +-
 .../java/org/apache/solr/common/util/Utils.java    |   9 +-
 .../solrj/impl/BaseSolrClientWireMockTest.java     |  20 +-
 .../solrj/impl/CloudHttp2SolrClientTest.java       |   5 +-
 .../impl/CloudHttp2SolrClientWireMockTest.java     |   4 +-
 .../solr/client/solrj/io/sql/JdbcDriverTest.java   |   3 +-
 .../solr/client/solrj/io/stream/StreamingTest.java |   2 +-
 .../apache/solr/BaseDistributedSearchTestCase.java |  22 +-
 .../src/java/org/apache/solr/SolrTestCase.java     |  10 +-
 .../solr/cloud/AbstractFullDistribZkTestBase.java  |   9 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |   9 +-
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |   2 +-
 .../component/TrackingShardHandlerFactory.java     |   2 +-
 .../src/resources/logconf/log4j2-startup-debug.xml |  13 +-
 versions.props                                     |   3 +-
 216 files changed, 4915 insertions(+), 5436 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 3bc02a1..f6f27db 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
@@ -119,8 +119,7 @@ public class AnalyticsShardRequestManager {
 
         Collections.shuffle(shuffler, new Random());
         Replica rep = shuffler.get(0);
-        ZkCoreNodeProps zkProps = new ZkCoreNodeProps(rep);
-        String url = zkProps.getCoreUrl();
+        String url = rep.getCoreUrl();
         replicaUrls.add(url);
       }
     } catch (Exception e) {
diff --git a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
index 5880778..eb50f3d 100644
--- a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
+++ b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/scraper/SolrCloudScraper.java
@@ -65,12 +65,12 @@ public class SolrCloudScraper extends SolrScraper {
 
     List<String> coreNames = replicas
         .stream()
-        .map(Replica::getCoreName)
+        .map(Replica::getName)
         .collect(Collectors.toList());
 
     Map<String, Http2SolrClient> coreToClient = replicas
         .stream()
-        .map(replica -> new Pair<>(replica.getCoreName(), httpSolrClients.get(replica.getBaseUrl())))
+        .map(replica -> new Pair<>(replica.getName(), httpSolrClients.get(replica.getBaseUrl())))
         .collect(Collectors.toMap(Pair::first, Pair::second));
 
     return sendRequestsInParallel(coreNames, core -> {
diff --git a/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java b/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
index ed52b35..0602536 100644
--- a/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
+++ b/solr/contrib/prometheus-exporter/src/test/org/apache/solr/prometheus/scraper/SolrCloudScraperTest.java
@@ -134,7 +134,7 @@ public class SolrCloudScraperTest extends PrometheusExporterTestBase {
     assertEquals(coreCount, allCoreMetrics.size());
 
     for (Map.Entry<String, DocCollection> entry : collectionStates.entrySet()) {
-      String coreName = entry.getValue().getReplicas().get(0).getCoreName();
+      String coreName = entry.getValue().getReplicas().get(0).getName();
       assertTrue(allCoreMetrics.containsKey(coreName));
       List<Collector.MetricFamilySamples> coreMetrics = allCoreMetrics.get(coreName).asList();
       assertEquals(1, coreMetrics.size());
@@ -168,7 +168,7 @@ public class SolrCloudScraperTest extends PrometheusExporterTestBase {
     Collection<Slice> slices = getCollectionState().getSlices();
 
     Set<String> leaderCoreNames = slices.stream()
-        .map(slice -> collection.getLeader(slice.getName()).getCoreName())
+        .map(slice -> collection.getLeader(slice.getName()).getName())
         .collect(Collectors.toSet());
 
     for (Collector.MetricFamilySamples.Sample sample : shardLeaderSamples.samples) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
index cec8dbd..4180280 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudDescriptor.java
@@ -37,7 +37,6 @@ public class CloudDescriptor {
   private String collectionName;
   private String roles = null;
   private Integer numShards;
-  private String nodeName = null;
   private Map<String,String> collectionParams = new HashMap<>();
 
   private volatile boolean isLeader = false;
@@ -64,9 +63,6 @@ public class CloudDescriptor {
     // If no collection name is specified, we default to the core name
     this.collectionName = props.getProperty(CoreDescriptor.CORE_COLLECTION, coreName);
     this.roles = props.getProperty(CoreDescriptor.CORE_ROLES, null);
-    this.nodeName = props.getProperty(CoreDescriptor.CORE_NODE_NAME);
-    if (Strings.isNullOrEmpty(nodeName))
-      this.nodeName = null;
     this.numShards = PropertiesUtil.toInteger(props.getProperty(CloudDescriptor.NUM_SHARDS), null);
     String replicaTypeStr = props.getProperty(CloudDescriptor.REPLICA_TYPE);
     if (Strings.isNullOrEmpty(replicaTypeStr)) {
@@ -146,16 +142,6 @@ public class CloudDescriptor {
   public void setNumShards(int numShards) {
     this.numShards = numShards;
   }
-  
-  public String getCoreNodeName() {
-    return nodeName;
-  }
-
-  public void setCoreNodeName(String nodeName) {
-    this.nodeName = nodeName;
-    if(nodeName==null) cd.getPersistableStandardProperties().remove(CoreDescriptor.CORE_NODE_NAME);
-    else cd.getPersistableStandardProperties().setProperty(CoreDescriptor.CORE_NODE_NAME, nodeName);
-  }
 
   public void reload(CloudDescriptor reloadFrom) {
     if (reloadFrom == null) return;
@@ -166,7 +152,6 @@ public class CloudDescriptor {
     if (reloadFrom.getNumShards() != null) {
       setNumShards(reloadFrom.getNumShards());
     }
-    setCoreNodeName(StringUtils.isEmpty(reloadFrom.getCoreNodeName()) ? getCoreNodeName() : reloadFrom.getCoreNodeName());
     setLeader(reloadFrom.isLeader);
     setHasRegistered(reloadFrom.hasRegistered);
     setLastPublished(reloadFrom.getLastPublished());
diff --git a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
index 9415e5f..0f5cc48 100644
--- a/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
+++ b/solr/core/src/java/org/apache/solr/cloud/CloudUtil.java
@@ -63,10 +63,10 @@ public class CloudUtil {
   public static void checkSharedFSFailoverReplaced(CoreContainer cc, CoreDescriptor desc) {
     if (desc.getCloudDescriptor() == null) return;
     ZkController zkController = cc.getZkController();
-    String thisCnn = zkController.getCoreNodeName(desc);
+    String coreName = desc.getName();
     String thisBaseUrl = zkController.getBaseUrl();
 
-    log.debug("checkSharedFSFailoverReplaced running for coreNodeName={} baseUrl={}", thisCnn, thisBaseUrl);
+    log.debug("checkSharedFSFailoverReplaced running for coreName={} baseUrl={}", coreName, thisBaseUrl);
 
     // if we see our core node name on a different base url, unload
     final DocCollection docCollection = zkController.getClusterState().getCollectionOrNull(desc.getCloudDescriptor().getCollectionName());
@@ -75,11 +75,11 @@ public class CloudUtil {
       for (Slice slice : slicesMap.values()) {
         for (Replica replica : slice.getReplicas()) {
 
-          String cnn = replica.getName();
+          String replicaName = replica.getName();
           String baseUrl = replica.getStr(ZkStateReader.BASE_URL_PROP);
-          log.debug("compare against coreNodeName={} baseUrl={}", cnn, baseUrl);
+          log.debug("compare against coreName={} baseUrl={}", replicaName, baseUrl);
 
-          if (thisCnn != null && thisCnn.equals(cnn)
+          if (coreName != null && coreName.equals(replicaName)
               && !thisBaseUrl.equals(baseUrl)) {
             if (cc.getLoadedCoreNames().contains(desc.getName())) {
               try {
@@ -106,12 +106,12 @@ public class CloudUtil {
     }
   }
 
-  public static boolean replicaExists(ClusterState clusterState, String collection, String shard, String coreNodeName) {
+  public static boolean replicaExists(ClusterState clusterState, String collection, String shard, String coreName) {
     DocCollection docCollection = clusterState.getCollectionOrNull(collection);
     if (docCollection != null) {
       Slice slice = docCollection.getSlice(shard);
       if (slice != null) {
-        return slice.getReplica(coreNodeName) != null;
+        return slice.getReplica(coreName) != null;
       }
     }
     return false;
diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
index 2fa75fe..2bfafe0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
@@ -17,12 +17,17 @@
 package org.apache.solr.cloud;
 
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.NodeExistsException;
 import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 
 /**
@@ -31,6 +36,8 @@ import java.util.List;
  * don't have to be ordered i.e. DistributedQueue.
  */
 public class DistributedMap {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   protected final String dir;
 
   protected SolrZkClient zookeeper;
@@ -38,13 +45,14 @@ public class DistributedMap {
   protected static final String PREFIX = "mn-";
 
   public DistributedMap(SolrZkClient zookeeper, String dir) throws KeeperException {
+    log.info("create DistributedMap dir={}", dir);
     this.dir = dir;
     this.zookeeper = zookeeper;
   }
 
 
   public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
-    zookeeper.mkdir(dir + "/" + PREFIX + trackingId, data);
+    zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, false, true);
   }
   
   /**
@@ -53,7 +61,7 @@ public class DistributedMap {
    */
   public boolean putIfAbsent(String trackingId, byte[] data) throws KeeperException, InterruptedException {
     try {
-      zookeeper.mkdir(dir + "/" + PREFIX + trackingId, data);
+      zookeeper.makePath(dir + "/" + PREFIX + trackingId, data, CreateMode.PERSISTENT, null, true, true);
       return true;
     } catch (NodeExistsException e) {
       return false;
@@ -61,16 +69,16 @@ public class DistributedMap {
   }
 
   public byte[] get(String trackingId) throws KeeperException, InterruptedException {
-    return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null);
+    return zookeeper.getData(dir + "/" + PREFIX + trackingId, null, null, true);
   }
 
   public boolean contains(String trackingId) throws KeeperException, InterruptedException {
-    return zookeeper.exists(dir + "/" + PREFIX + trackingId);
+    return zookeeper.exists(dir + "/" + PREFIX + trackingId, true);
   }
 
   public int size() throws KeeperException, InterruptedException {
     Stat stat = new Stat();
-    zookeeper.getData(dir, null, stat);
+    zookeeper.getData(dir, null, stat, true);
     return stat.getNumChildren();
   }
 
@@ -81,7 +89,7 @@ public class DistributedMap {
    */
   public boolean remove(String trackingId) throws KeeperException, InterruptedException {
     try {
-      zookeeper.delete(dir + "/" + PREFIX + trackingId, -1);
+      zookeeper.delete(dir + "/" + PREFIX + trackingId, -1, true);
     } catch (KeeperException.NoNodeException e) {
       return false;
     }
@@ -94,9 +102,10 @@ public class DistributedMap {
   public void clear() throws KeeperException, InterruptedException {
     List<String> childNames = zookeeper.getChildren(dir, null, true);
     for(String childName: childNames) {
-      zookeeper.delete(dir + "/" + childName, -1);
+      zookeeper.delete(dir + "/" + childName, -1, true);
     }
-
+//nocommit
+    //zookeeper.delete(paths, true);
   }
   
   /**
diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index 308f3ba..1441e2e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -20,6 +20,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.zookeeper.KeeperException;
@@ -29,12 +30,12 @@ import org.slf4j.LoggerFactory;
 public abstract class ElectionContext implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   protected final String electionPath;
-  protected final ZkNodeProps leaderProps;
+  protected final Replica leaderProps;
   protected final String id;
   protected final String leaderPath;
   protected volatile String leaderSeqPath;
 
-  public ElectionContext(final String id, final String electionPath, final String leaderPath, final ZkNodeProps leaderProps) {
+  public ElectionContext(final String id, final String electionPath, final String leaderPath, final Replica leaderProps) {
     this.id = id;
     this.electionPath = electionPath;
     this.leaderPath = leaderPath;
@@ -44,7 +45,6 @@ public abstract class ElectionContext implements Closeable {
   }
 
   public void close() {
-    leaderSeqPath = null;
     assert ObjectReleaseTracker.release(this);
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 7b25968..a54aea7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.cloud;
 
+import java.io.Closeable;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Collections;
@@ -54,7 +55,7 @@ import org.slf4j.LoggerFactory;
  * starts the whole process over by checking if it's the lowest sequential node, etc.
  *
  */
-public  class LeaderElector {
+public class LeaderElector implements Closeable {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   public static final String ELECTION_NODE = "/election";
@@ -63,6 +64,7 @@ public  class LeaderElector {
   private final static Pattern SESSION_ID = Pattern.compile(".*?/?(.*?-.*?)-n_\\d+");
 
   protected final SolrZkClient zkClient;
+  private final ZkController zkController;
 
   private volatile ElectionContext context;
 
@@ -70,6 +72,7 @@ public  class LeaderElector {
 
   private final Map<ContextKey,ElectionContext> electionContexts;
   private final ContextKey contextKey;
+  private volatile boolean isClosed;
 
   //  public LeaderElector(SolrZkClient zkClient) {
 //    this.zkClient = zkClient;
@@ -77,9 +80,10 @@ public  class LeaderElector {
 //    this.electionContexts = new ConcurrentHashMap<>(132, 0.75f, 50);
 //  }
 
-  public LeaderElector(SolrZkClient zkClient, ContextKey key, Map<ContextKey,ElectionContext> electionContexts) {
+  public LeaderElector(ZkController zkController, ContextKey key, Map<ContextKey,ElectionContext> electionContexts) {
 
-    this.zkClient = zkClient;
+    this.zkClient = zkController.getZkClient();
+    this.zkController = zkController;
     this.electionContexts = electionContexts;
     this.contextKey = key;
   }
@@ -98,11 +102,17 @@ public  class LeaderElector {
    */
   private boolean checkIfIamLeader(final ElectionContext context, boolean replacement) throws KeeperException,
           InterruptedException, IOException {
-    log.info("Check if I am leader");
-    if (context.isClosed()) {
-      throw new AlreadyClosedException();
+    if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+      if (log.isDebugEnabled()) log.debug("Will not checkIfIamLeader, elector is closed");
+      return false;
     }
-    context.checkIfIamLeaderFired();
+
+    log.info("Check if I am leader {}", context.getClass().getSimpleName());
+
+    ParWork.getRootSharedExecutor().submit(() -> {
+      context.checkIfIamLeaderFired();
+    });
+
     boolean checkAgain = false;
 
     // get all other numbers...
@@ -114,6 +124,7 @@ public  class LeaderElector {
     try {
       leaderSeqNodeName = context.leaderSeqPath.substring(context.leaderSeqPath.lastIndexOf('/') + 1);
     } catch (NullPointerException e) {
+      if (log.isDebugEnabled()) log.debug("leaderSeqPath has been removed, bailing");
       return false;
     }
     if (!seqs.contains(leaderSeqNodeName)) {
@@ -123,12 +134,21 @@ public  class LeaderElector {
 
     if (leaderSeqNodeName.equals(seqs.get(0))) {
       // I am the leader
-      log.info("I am the leader");
-
-      runIamLeaderProcess(context, replacement);
+      log.info("I am the potential leader {}, running leader process", context.leaderProps);
+      ParWork.getRootSharedExecutor().submit(() -> {
+        try {
+          if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+            if (log.isDebugEnabled()) log.debug("Elector is closed, will not try and run leader processes");
+            return;
+          }
+          runIamLeaderProcess(context, replacement);
+        } catch (Exception e) {
+          log.error("", e);
+        }
+      });
 
     } else {
-      log.info("I am not the leader - watch the node below me");
+      log.info("I am not the leader - watch the node below me {}", context.getClass().getSimpleName());
       String toWatch = seqs.get(0);
       for (String node : seqs) {
         if (leaderSeqNodeName.equals(node)) {
@@ -140,23 +160,33 @@ public  class LeaderElector {
         String watchedNode = holdElectionPath + "/" + toWatch;
 
         ElectionWatcher oldWatcher = watcher;
-        if (oldWatcher != null) oldWatcher.cancel();
-        zkClient.getData(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath, watchedNode, getSeq(context.leaderSeqPath), context), null);
+        if (oldWatcher != null) {
+          oldWatcher.cancel();
+        }
+        zkClient.exists(watchedNode, watcher = new ElectionWatcher(context.leaderSeqPath,
+            watchedNode, getSeq(context.leaderSeqPath), context));
         if (log.isDebugEnabled()) log.debug("Watching path {} to know if I could be the leader", watchedNode);
       } catch (KeeperException.SessionExpiredException e) {
         log.error("ZooKeeper session has expired");
         throw e;
       } catch (KeeperException.NoNodeException e) {
-        // the previous node disappeared, check if we are the leader again
+        log.info("the previous node disappeared, check if we are the leader again");
         checkAgain = true;
       } catch (KeeperException e) {
         // we couldn't set our watch for some other reason, retry
-        log.warn("Failed setting watch", e);
+        log.info("Failed setting election watch, retrying {} {}", e.getClass().getName(), e.getMessage());
         checkAgain = true;
+      } catch (Exception e) {
+        // we couldn't set our watch for some other reason, retry
+        log.info("Failed setting election watch {} {}", e.getClass().getName(), e.getMessage());
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       }
-    }
 
-    return checkAgain;
+      if (checkAgain) {
+        return true;
+      }
+    }
+    return false;
   }
 
   // TODO: get this core param out of here
@@ -200,7 +230,7 @@ public  class LeaderElector {
 
   }
 
-  public int joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
+  public boolean joinElection(ElectionContext context, boolean replacement) throws KeeperException, InterruptedException, IOException {
     return joinElection(context,replacement, false);
   }
 
@@ -213,13 +243,15 @@ public  class LeaderElector {
    *
    * @return sequential node number
    */
-  public int joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
-    if (context.isClosed() || zkClient.isClosed()) {
-      if (log.isDebugEnabled()) log.debug("Already closed");
-      throw new AlreadyClosedException();
+  public boolean joinElection(ElectionContext context, boolean replacement,boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
+    if (isClosed || (zkController != null && zkController.getCoreContainer().isShutDown())) {
+      if (log.isDebugEnabled()) log.debug("Will not join election, elector is closed");
+      return false;
     }
 
-    context.joinedElectionFired();
+    ParWork.getRootSharedExecutor().submit(() -> {
+      context.joinedElectionFired();
+    });
 
     final String shardsElectZkPath = context.electionPath + LeaderElector.ELECTION_NODE;
 
@@ -248,17 +280,9 @@ public  class LeaderElector {
             zkClient.create(leaderSeqPath, null, CreateMode.EPHEMERAL, false);
           }
         } else {
-          while (true) {
-            if (log.isDebugEnabled()) log.debug("create ephem election node {}", shardsElectZkPath + "/" + id + "-n_");
-              try {
+          if (log.isDebugEnabled()) log.debug("create ephem election node {}", shardsElectZkPath + "/" + id + "-n_");
               leaderSeqPath = zkClient.create(shardsElectZkPath + "/" + id + "-n_", null,
                       CreateMode.EPHEMERAL_SEQUENTIAL, false);
-              break;
-            } catch (ConnectionLossException e) {
-              log.warn("Connection loss during leader election, trying again ...");
-              continue;
-            }
-          }
         }
 
         if (log.isDebugEnabled()) log.debug("Joined leadership election with path: {}", leaderSeqPath);
@@ -288,25 +312,43 @@ public  class LeaderElector {
       } catch (KeeperException.NoNodeException e) {
         // we must have failed in creating the election node - someone else must
         // be working on it, lets try again
+        log.info("No node found during election {} " + e.getMessage(), e.getPath());
         if (tries++ > 5) {
-          context = null;
+          log.error("No node found during election {} " + e.getMessage(), e.getPath());
           throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-                  "", e);
+              "", e);
         }
         cont = true;
       }
     }
-    while(checkIfIamLeader(context, replacement)) {
-      if (context.isClosed()) {
-        throw new AlreadyClosedException();
-      }
+
+    int seq = getSeq(context.leaderSeqPath);
+
+    boolean tryagain = checkIfIamLeader(context, replacement);
+
+    if (tryagain) {
+      checkIfIamLeader(context, replacement);
     }
 
-    return getSeq(context.leaderSeqPath);
+
+//    boolean tryagain = false;
+//    while (tryagain) {
+//      tryagain = checkIfIamLeader(context, replacement);
+//      if (tryagain) {
+//        Thread.sleep(250);
+//      }
+//    }
+
+    return false;
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.isClosed = true;
   }
 
   private class ElectionWatcher implements Watcher {
-    final String myNode,watchedNode;
+    final String myNode, watchedNode;
     final ElectionContext context;
 
     private volatile boolean canceled = false;
@@ -319,7 +361,6 @@ public  class LeaderElector {
 
     void cancel() {
       canceled = true;
-
     }
 
     @Override
@@ -329,29 +370,20 @@ public  class LeaderElector {
         return;
       }
       if (canceled) {
-        log.debug("This watcher is not active anymore {}", myNode);
-        try {
-          zkClient.delete(myNode, -1);
-        } catch (AlreadyClosedException | InterruptedException e) {
-          log.info("Already shutting down");
-          return;
-        } catch (KeeperException.NoNodeException nne) {
-          log.info("No znode found to delete at {}", myNode);
-          // expected . don't do anything
-        } catch (Exception e) {
-          log.error("Exception canceling election", e);
-          return;
-        }
+        if (log.isDebugEnabled()) log.debug("This watcher is not active anymore {}", myNode);
         return;
       }
       try {
         // am I the next leader?
-        checkIfIamLeader(context, true);
+        boolean tryagain = checkIfIamLeader(context, true);
+        if (tryagain) {
+          checkIfIamLeader(context, true);
+        }
       } catch (AlreadyClosedException | InterruptedException e) {
         log.info("Already shutting down");
         return;
       } catch (Exception e) {
-        log.error("Exception canceling election", e);
+        log.error("Exception in election", e);
         return;
       }
     }
@@ -372,9 +404,6 @@ public  class LeaderElector {
   }
 
   void retryElection(ElectionContext context, boolean joinAtHead) throws KeeperException, InterruptedException, IOException {
-    if (context.isClosed()) {
-      throw new AlreadyClosedException();
-    }
     ElectionWatcher watcher = this.watcher;
     if (electionContexts != null) {
       ElectionContext prevContext = electionContexts.put(contextKey, context);
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 45f4ebe..463ac1a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -17,25 +17,20 @@
 package org.apache.solr.cloud;
 
 import java.io.Closeable;
-import java.io.File;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.LongAdder;
 import java.util.function.BiConsumer;
 
 import org.apache.lucene.util.Version;
 import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.impl.CloudHttp2SolrClient;
 import org.apache.solr.client.solrj.impl.Http2SolrClient;
@@ -44,14 +39,8 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.api.collections.CreateCollectionCmd;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler;
-import org.apache.solr.cloud.overseer.ClusterStateMutator;
-import org.apache.solr.cloud.overseer.CollectionMutator;
-import org.apache.solr.cloud.overseer.NodeMutator;
 import org.apache.solr.cloud.overseer.OverseerAction;
-import org.apache.solr.cloud.overseer.ReplicaMutator;
-import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.cloud.overseer.ZkStateWriter;
-import org.apache.solr.cloud.overseer.ZkWriteCommand;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrCloseable;
@@ -60,32 +49,29 @@ import org.apache.solr.common.SolrThread;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.ConnectionManager;
 import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
-import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.util.IOUtils;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.ObjectReleaseTracker;
 import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.core.CloudConfig;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.handler.component.HttpShardHandler;
 import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.update.UpdateShardHandler;
-import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.cloud.OverseerConfigSetMessageHandler.CONFIGSETS_ACTION_PREFIX;
+import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
+
 /**
  * <p>Cluster leader. Responsible for processing state updates, node assignments, creating/deleting
  * collections, shards, replicas and setting various properties.</p>
@@ -156,7 +142,6 @@ public class Overseer implements SolrCloseable {
 
   public static final String OVERSEER_COLLECTION_MAP_RUNNING = "/overseer/collection-map-running";
 
-  public static final String OVERSEER_QUEUE_WORK = "/overseer/queue-work";
 
   // System properties are used in tests to make them run fast
   public static final int STATE_UPDATE_DELAY = ZkStateReader.STATE_UPDATE_DELAY;
@@ -167,13 +152,21 @@ public class Overseer implements SolrCloseable {
   public static final String OVERSEER_ELECT = "/overseer/overseer_elect";
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private volatile OverseerElectionContext context;
+
   private volatile boolean closeAndDone;
+  private volatile boolean initedHttpClient = false;
+  private volatile QueueWatcher queueWatcher;
+  private volatile CollectionWorkQueueWatcher collectionQueueWatcher;
 
   public boolean isDone() {
     return closeAndDone;
   }
 
+
+  //  public ExecutorService getTaskExecutor() {
+//    return taskExecutor;
+//  }
+
   private static class StringBiConsumer implements BiConsumer<String, Object> {
     boolean firstPair = true;
 
@@ -201,6 +194,7 @@ public class Overseer implements SolrCloseable {
     //queue where everybody can throw tasks
     private final ZkDistributedQueue stateUpdateQueue;
 
+
     private volatile boolean isClosed = false;
 
     public ClusterStateUpdater(final ZkStateReader reader, final String myId, Stats zkStats) {
@@ -219,114 +213,92 @@ public class Overseer implements SolrCloseable {
       MDCLoggingContext.setNode(zkController.getNodeName() );
       try {
 
-      log.info("Starting to work on the main queue : {}", LeaderElector.getNodeName(myId));
 
-        ZkStateWriter zkStateWriter = new ZkStateWriter(reader, stats);;
-        try {
-          reader.forciblyRefreshAllClusterStateSlow();
-        } catch (KeeperException e) {
-          log.error("", e);
-          return;
-        } catch (InterruptedException e) {
-          ParWork.propagateInterrupt(e);
-          return;
-        }
-        ClusterState clusterState = reader.getClusterState();
-        assert clusterState != null;
+        log.info("Starting to work on the main queue : {}", LeaderElector.getNodeName(myId));
+
 
+//        ClusterState clusterState = reader.getClusterState();
+//        assert clusterState != null;
 
-        while (!checkClosed()) {
-          if (log.isDebugEnabled()) log.debug("Start of Overseer loop ...");
+        // we write updates in batch, but if an exception is thrown when writing new clusterstate,
+        // we do not sure which message is bad message, therefore we will re-process node one by one
 
-          LinkedList<Pair<String, byte[]>> queue = null;
-          try {
-            // We do not need to filter any nodes here cause all processed nodes are removed once we flush clusterstate
+        while (!this.isClosed) {
+
+            LinkedList<Pair<String,byte[]>> queue = null;
+            try {
+              // We do not need to filter any nodes here cause all processed nodes are removed once we flush clusterstate
+              queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, 3000L, (x) -> true));
 
-            long wait = 1000;
-            queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, wait, (x) -> x.startsWith(OverseerTaskQueue.RESPONSE_PREFIX)));
-          } catch (AlreadyClosedException e) {
-            if (isClosed()) {
-              log.info("Overseer closed (AlreadyClosedException), exiting loop");
+              log.info("peeking at the status update queue {}", printQueue(queue));
+            } catch (InterruptedException | AlreadyClosedException e) {
+              ParWork.propagateInterrupt(e, true);
+              log.error("Unexpected error in Overseer state update loop", e);
               return;
-            }
-            return;
-          } catch (KeeperException.SessionExpiredException e) {
-            log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
-            return;
-          } catch (Exception e) {
-            ParWork.propagateInterrupt("Unexpected error in Overseer state update loop", e, true);
-            log.error("Unexpected error in Overseer state update loop", e);
-            if (!isClosed()) {
-              continue;
-            } else {
+            } catch (KeeperException.SessionExpiredException e) {
+              log.error("run()", e);
+
+              log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
+              return;
+            } catch (Exception e) {
+              ParWork.propagateInterrupt(e, true);
+              log.error("Unexpected error in Overseer state update loop", e);
               return;
             }
-          }
-          try {
-            Set<String> processedNodes = new HashSet<>();
-            TimeOut timeout = new TimeOut(3, TimeUnit.SECONDS, TimeSource.NANO_TIME);
-            while (queue != null && !queue.isEmpty()) {
-              if (isClosed()) {
-                log.info("Closing");
-                return;
-              }
-              for (Pair<String, byte[]> head : queue) {
-                byte[] data = head.second();
+            try {
+              Set<String> processedNodes = new HashSet<>();
+              while (queue != null && !queue.isEmpty()) {
+                for (Pair<String,byte[]> head : queue) {
+                  byte[] data = head.second();
+                  if (log.isDebugEnabled()) log.debug("look at node {} data={}", head.first(), head.second() == null ? null : head.second().length);
 
-                if (log.isDebugEnabled()) log.debug("look at node {} data={}", head.first(), head.second() == null ? null : head.second().length);
-                if (head.second() != null && head.second().length > 0) {
                   final ZkNodeProps message = ZkNodeProps.load(data);
                   if (log.isDebugEnabled()) log.debug("processMessage: queueSize: {}, message = {}", stateUpdateQueue.getZkStats().getQueueLength(), message);
                   if (log.isDebugEnabled()) log.debug("add processed node: {}, processedNodes = {}", head.first(), stateUpdateQueue.getZkStats().getQueueLength(), processedNodes);
-                  processedNodes.add(new File(head.first()).getName());
+                  processedNodes.add(head.first());
+
                   // The callback always be called on this thread
-                  processQueueItem(message, reader.getClusterState(), zkStateWriter, true, null);
-                } else {
-                  log.warn("Found queue item with no data, removing it {} : {}", head.first(), new File(head.first()).getName());
-                  processedNodes.add(new File(head.first()).getName());
-                }
-              }
-              if (isClosed()) {
-                log.info("Overseer closed, exiting loop");
-                return;
-              }
+                  boolean success = processQueueItem(message);
+                  if (success) {
+                    // nocommit
+                    stateUpdateQueue.remove(processedNodes);
+                    processedNodes.clear();
+                  }
 
-              if (timeout.hasTimedOut()) {
-                if (log.isDebugEnabled()) log.debug("timeout, skipping out on tight loop {}", timeout.getInterval(TimeUnit.MILLISECONDS));
-                break;
+                }
+                //    if (isClosed) break;
+                // if an event comes in the next 100ms batch it together
+                log.info("peekElements");
+                queue = new LinkedList<>(stateUpdateQueue.peekElements(1000, 100, node -> !processedNodes.contains(node)));
               }
-              // if an event comes in the next *ms batch it together
-              int wait = 10;
-              if (log.isDebugEnabled()) log.debug("going to peekElements processedNodes={}", processedNodes);
-              queue = new LinkedList<>(stateUpdateQueue.peekElements(10, wait, node -> processedNodes.contains(node) || node.startsWith(OverseerTaskQueue.RESPONSE_PREFIX)));
-            }
 
-            clusterState = zkStateWriter.writePendingUpdates(clusterState,  () -> {
-              if (log.isDebugEnabled()) log.debug("clear processedNodes={}", processedNodes);
+              // we should force write all pending updates because the next iteration might sleep until there
+              // are more items in the main queue
+              log.info("writePending updates");
+              writePendingUpdates();
+              // clean work queue
+              log.info("clean work queue");
               stateUpdateQueue.remove(processedNodes);
               processedNodes.clear();
-            });
-
-          } catch (InterruptedException | AlreadyClosedException e) {
-            ParWork.propagateInterrupt(e, true);
-            return;
-          } catch (KeeperException.SessionExpiredException e) {
-            log.error("run()", e);
-
-            log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
-            return;
-          } catch (Exception e) {
-            log.error("Unexpected error in Overseer state update loop, exiting ...", e);
-            return;
+            } catch (InterruptedException | AlreadyClosedException e) {
+              Thread.currentThread().interrupt();
+              return;
+            } catch (KeeperException.SessionExpiredException e) {
+              log.error("run()", e);
+
+              log.warn("Solr cannot talk to ZK, exiting Overseer work queue loop", e);
+              return;
+            } catch (Exception e) {
+              log.error("", e);
+              throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+            }
           }
-        }
-      } finally {
+        } finally {
         log.info("Overseer Loop exiting : {}", LeaderElector.getNodeName(myId));
 
-        if (!isClosed && !closeAndDone) { // if we have not been closed, close so that we stop the other threads
-          Overseer.this.close(true);
+        if (!isClosed) {
+          Overseer.this.close();
         }
-        itemsQueued.reset();
       }
 
       if (log.isDebugEnabled()) {
@@ -363,173 +335,6 @@ public class Overseer implements SolrCloseable {
       return false;
     }
 
-    private LongAdder itemsQueued = new LongAdder();
-
-    private ClusterState processQueueItem(ZkNodeProps message, ClusterState clusterState, ZkStateWriter zkStateWriter,
-                                          boolean enableBatching, ZkStateWriter.ZkWriteCallback callback) throws Exception {
-      if (log.isDebugEnabled()) log.debug("Consume state update from queue {}", message);
-      // assert clusterState != null;
-
-      ClusterState cs = null;
-      //  if (clusterState.getZNodeVersion() == 0 || clusterState.getZNodeVersion() > lastVersion) {
-
-      final String operation = message.getStr(QUEUE_OPERATION);
-      if (operation == null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Message missing " + QUEUE_OPERATION + ":" + message);
-      }
-      itemsQueued.increment();
-      Map<String,DocCollection> updatesToWrite = zkStateWriter.getUpdatesToWrite();
-      ClusterState state;
-      LinkedHashMap<String,ClusterState.CollectionRef> collStates;
-      ClusterState prevState = null;
-
-      state = clusterState;
-
-      collStates = new LinkedHashMap<>(state.getCollectionStates());
-      for (DocCollection docCollection : updatesToWrite.values()) {
-        Map<String,Slice> slicesMap = docCollection.getSlicesMap();
-        DocCollection oldDoc = state.getCollectionOrNull(docCollection.getName());
-        Map<String,Slice> newSlicesMap;
-        if (oldDoc != null) {
-          newSlicesMap = new HashMap(oldDoc.getSlicesMap());
-        } else {
-          newSlicesMap = new HashMap();
-        }
-
-        for (Slice slice : slicesMap.values()) {
-          Slice oldSlice = null;
-          if (oldSlice != null && oldDoc.getSlicesMap() != null) {
-            oldSlice = oldDoc.getSlicesMap().get(slice.getName());
-          }
-          Map<String,Replica> existingReplicas;
-          if (oldSlice != null && oldSlice.getReplicasMap() != null) {
-            existingReplicas = new HashMap<>(oldSlice.getReplicasMap());
-          } else {
-            existingReplicas = new HashMap<>();
-          }
-
-          for (Replica ereplica : slice.getReplicas()) {
-            existingReplicas.put(ereplica.getName(), ereplica);
-          }
-
-          newSlicesMap.put(slice.getName(), new Slice(slice.getName(), existingReplicas, slice.getProperties(), docCollection.getName()));
-        }
-        collStates.put(docCollection.getName(), new ClusterState.CollectionRef(
-            new DocCollection(docCollection.getName(), newSlicesMap, docCollection.getProperties(), docCollection.getRouter(), docCollection.getZNodeVersion())));
-
-      }
-      prevState = new ClusterState(state.getLiveNodes(), collStates, state.getZNodeVersion());
-
-      List<ZkWriteCommand> zkWriteOps = processMessage(prevState, message, operation);
-
-      cs = zkStateWriter.enqueueUpdate(prevState, zkWriteOps, callback);
-      //  }
-      if (log.isDebugEnabled()) log.debug("State update consumed from queue {}", message);
-      return cs;
-    }
-
-    private List<ZkWriteCommand> processMessage(ClusterState clusterState,
-                                                final ZkNodeProps message, final String operation) {
-      if (log.isDebugEnabled()) {
-        log.debug("processMessage(ClusterState clusterState={}, ZkNodeProps message={}, String operation={}) - start", clusterState, message, operation);
-      }
-
-      CollectionParams.CollectionAction collectionAction = CollectionParams.CollectionAction.get(operation);
-      if (collectionAction != null) {
-        switch (collectionAction) {
-          case CREATE:
-            return Collections.singletonList(new ClusterStateMutator(getSolrCloudManager()).createCollection(clusterState, message));
-          case DELETE:
-            return Collections.singletonList(new ClusterStateMutator(getSolrCloudManager()).deleteCollection(clusterState, message));
-          case CREATESHARD:
-            List<ZkWriteCommand> ret = Collections.singletonList(new CollectionMutator(getSolrCloudManager()).createShard(clusterState, message));
-            String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
-            String shardName = message.getStr(ZkStateReader.SHARD_ID_PROP);
-            try {
-              // TODO: remove this makePath calls and require making each path individually correctly
-              getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + shardName, null, CreateMode.PERSISTENT, false);
-              // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect", null, CreateMode.PERSISTENT, false);
-              getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
-              getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
-              getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
-              getSolrCloudManager().getDistribStateManager().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
-
-            } catch (KeeperException | IOException | InterruptedException | AlreadyExistsException e) {
-              ParWork.propagateInterrupt(e);
-              throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-            }
-            return ret;
-          case DELETESHARD:
-            return Collections.singletonList(new CollectionMutator(getSolrCloudManager()).deleteShard(clusterState, message));
-          case ADDREPLICA:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).addReplica(clusterState, message));
-          case ADDREPLICAPROP:
-            return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).addReplicaProperty(clusterState, message));
-          case DELETEREPLICAPROP:
-            return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).deleteReplicaProperty(clusterState, message));
-          case BALANCESHARDUNIQUE:
-            ExclusiveSliceProperty dProp = new ExclusiveSliceProperty(clusterState, message);
-            if (dProp.balanceProperty()) {
-              String collName = message.getStr(ZkStateReader.COLLECTION_PROP);
-              List<ZkWriteCommand> returnList = Collections.singletonList(new ZkWriteCommand(collName, dProp.getDocCollection()));
-              if (log.isDebugEnabled()) {
-                log.debug("processMessage(ClusterState, ZkNodeProps, String) - end");
-              }
-              return returnList;
-            }
-            break;
-          case MODIFYCOLLECTION:
-            ZkWriteCommand zkwrite = new CollectionMutator(getSolrCloudManager()).modifyCollection(clusterState, message);
-            return Collections.singletonList(zkwrite);
-          default:
-            throw new RuntimeException("unknown operation:" + operation
-                    + " contents:" + message.getProperties());
-        }
-      } else {
-        OverseerAction overseerAction = OverseerAction.get(operation);
-        if (overseerAction == null) {
-          throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
-        }
-        switch (overseerAction) {
-          case STATE:
-            return Collections.singletonList(new ReplicaMutator(getSolrCloudManager()).setState(clusterState, message));
-          case LEADER:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).setShardLeader(clusterState, message));
-          case DELETECORE:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).removeReplica(clusterState, message));
-          case ADDROUTINGRULE:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).addRoutingRule(clusterState, message));
-          case REMOVEROUTINGRULE:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).removeRoutingRule(clusterState, message));
-          case UPDATESHARDSTATE:
-            return Collections.singletonList(new SliceMutator(getSolrCloudManager()).updateShardState(clusterState, message));
-//          case QUIT:
-//            if (myId.equals(message.get(ID))) {
-//              log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
-//              try {
-//                overseerCollectionConfigSetProcessor.close();
-//              } catch (IOException e) {
-//                log.error("IOException", e);
-//              }
-//              close();
-//            } else {
-//              log.warn("Overseer received wrong QUIT message {}", message);
-//            }
-//            break;
-          case DOWNNODE:
-            return new NodeMutator().downNode(clusterState, message);
-          default:
-            throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
-        }
-      }
-
-      List<ZkWriteCommand> returnList = Collections.singletonList(ZkStateWriter.NO_OP);
-      if (log.isDebugEnabled()) {
-        log.debug("processMessage(ClusterState, ZkNodeProps, String) - end");
-      }
-      return returnList;
-    }
-
     @Override
     public void close() {
       if (log.isDebugEnabled()) {
@@ -542,6 +347,16 @@ public class Overseer implements SolrCloseable {
     }
   }
 
+  private String printQueue(LinkedList<Pair<String,byte[]>> queue) {
+
+    StringBuilder sb = new StringBuilder("Queue[");
+    for (Pair<String,byte[]> item : queue) {
+      sb.append(item.first()).append(":").append(ZkNodeProps.load(item.second())).append(", ");
+    }
+    sb.append("]");
+    return sb.toString();
+  }
+
   public static class OverseerThread extends SolrThread implements Closeable {
 
     protected volatile boolean isClosed;
@@ -573,11 +388,14 @@ public class Overseer implements SolrCloseable {
 
   }
 
-  private volatile OverseerThread ccThread;
 
-  private volatile OverseerThread updaterThread;
+ // private volatile OverseerThread updaterThread;
+
+//  private volatile ExecutorService stateManagmentExecutor;
+//
+//  private volatile ExecutorService taskExecutor;
 
-  private volatile OverseerThread triggerThread;
+  private volatile ZkStateWriter zkStateWriter;
 
   private final ZkStateReader reader;
 
@@ -591,7 +409,7 @@ public class Overseer implements SolrCloseable {
 
   private volatile Stats stats;
   private volatile String id;
-  private volatile boolean closed;
+  private volatile boolean closed = true;
   private volatile boolean systemCollCompatCheck = true;
 
   private final CloudConfig config;
@@ -608,15 +426,16 @@ public class Overseer implements SolrCloseable {
     this.zkController = zkController;
     this.stats = new Stats();
     this.config = config;
-
   }
 
   public synchronized void start(String id, ElectionContext context) throws KeeperException {
-    doClose();
+    log.info("Staring Overseer");
     if (getCoreContainer().isShutDown() || closeAndDone) {
       if (log.isDebugEnabled()) log.debug("Already closed, exiting");
       return;
     }
+
+    doClose();
     closed = false;
 
     MDCLoggingContext.setNode(zkController == null ?
@@ -624,70 +443,65 @@ public class Overseer implements SolrCloseable {
         zkController.getNodeName());
 
     this.id = id;
-    this.context = (OverseerElectionContext) context;
+//
+//     stateManagmentExecutor = ParWork.getParExecutorService("stateManagmentExecutor",
+//        1, 1, 3000, new SynchronousQueue());
+//     taskExecutor = ParWork.getParExecutorService("overseerTaskExecutor",
+//        4, 16, 3000, new SynchronousQueue());
 
 //    try {
 //      if (context != null) context.close();
 //    } catch (Exception e) {
 //      log.error("", e);
 //    }
-    Http2SolrClient.Builder overseerOnlyClientBuilder = new Http2SolrClient.Builder();
-    overseerOnlyClientBuilder = overseerOnlyClientBuilder.connectionTimeout(15000).idleTimeout(500000);
-
+    if (overseerOnlyClient == null && !closeAndDone && !initedHttpClient) {
+      overseerOnlyClient = new Http2SolrClient.Builder().idleTimeout(500000).markInternalRequest().build();
+      overseerOnlyClient.enableCloseLock();
+      this.overseerLbClient = new LBHttp2SolrClient(overseerOnlyClient);
+      initedHttpClient = true;
+    }
 
-    overseerOnlyClient = overseerOnlyClientBuilder.markInternalRequest().build();
-    overseerOnlyClient.enableCloseLock();
-    this.overseerLbClient = new LBHttp2SolrClient(overseerOnlyClient);
 
-    try {
-      if (log.isDebugEnabled()) {
-        log.debug("set watch on leader znode");
-      }
-      zkController.getZkClient().exists(Overseer.OVERSEER_ELECT + "/leader", new Watcher() {
+//    try {
+//      if (log.isDebugEnabled()) {
+//        log.debug("set watch on leader znode");
+//      }
+//      zkController.getZkClient().exists(Overseer.OVERSEER_ELECT + "/leader", new Watcher() {
+//
+//        @Override
+//        public void process(WatchedEvent event) {
+//          if (Event.EventType.None.equals(event.getType())) {
+//            return;
+//          }
+//          if (!isClosed()) {
+//            log.info("Overseer leader has changed, closing ...");
+//            Overseer.this.close();
+//          }
+//        }}, true);
+//    } catch (KeeperException.SessionExpiredException e) {
+//      log.warn("ZooKeeper session expired");
+//      return;
+//    } catch (InterruptedException | AlreadyClosedException e) {
+//      log.info("Already closed");
+//      return;
+//    } catch (Exception e) {
+//      log.error("Unexpected error in Overseer state update loop", e);
+//      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+//    }
 
-        @Override
-        public void process(WatchedEvent event) {
-          if (Event.EventType.None.equals(event.getType())) {
-            return;
-          }
-          if (!isClosed()) {
-            log.info("Overseer leader has changed, closing ...");
-            Overseer.this.close();
-          }
-        }}, true);
-    } catch (KeeperException.SessionExpiredException e) {
-      log.warn("ZooKeeper session expired");
-      return;
-    } catch (InterruptedException | AlreadyClosedException e) {
-      log.info("Already closed");
-      return;
-    } catch (Exception e) {
-      log.error("Unexpected error in Overseer state update loop", e);
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-    }
 
     stats = new Stats();
     log.info("Overseer (id={}) starting", id);
     //launch cluster state updater thread
-    ThreadGroup tg = new ThreadGroup("Overseer state updater.");
-    updaterThread = new OverseerThread(tg, new ClusterStateUpdater(reader, id, stats), "OverseerStateUpdate-" + id);
-    updaterThread.setDaemon(true);
 
     ThreadGroup ccTg = new ThreadGroup("Overseer collection creation process.");
 
-    // nocommit - I don't know about this guy..
-    OverseerNodePrioritizer overseerPrioritizer = null; // new OverseerNodePrioritizer(reader, getStateUpdateQueue(), adminPath, shardHandler.getShardHandlerFactory(), updateShardHandler.getUpdateOnlyHttpClient());
-    overseerCollectionConfigSetProcessor = new OverseerCollectionConfigSetProcessor(zkController.getCoreContainer(), reader, id, overseerLbClient, adminPath, stats, Overseer.this, overseerPrioritizer);
-    ccThread = new OverseerThread(ccTg, overseerCollectionConfigSetProcessor, "OverseerCollectionConfigSetProcessor-" + id);
-    ccThread.setDaemon(true);
 
-    updaterThread.start();
-    ccThread.start();
-    if (triggerThread != null) {
-      triggerThread.start();
-    }
+    this.zkStateWriter = new ZkStateWriter(reader, stats);
+    //systemCollectionCompatCheck(new StringBiConsumer());
 
-    systemCollectionCompatCheck(new StringBiConsumer());
+    queueWatcher = new WorkQueueWatcher(getCoreContainer());
+    collectionQueueWatcher = new CollectionWorkQueueWatcher(getCoreContainer(), id, overseerLbClient, adminPath, stats, Overseer.this);
 
     // TODO: don't track for a moment, can leak out of collection api tests
     // assert ObjectReleaseTracker.track(this);
@@ -816,46 +630,41 @@ public class Overseer implements SolrCloseable {
     return zkController.getSolrCloudManager();
   }
 
-  /**
-   * For tests.
-   * 
-   * @lucene.internal
-   * @return state updater thread
-   */
-  public synchronized OverseerThread getUpdaterThread() {
-    return updaterThread;
-  }
-
-  /**
-   * For tests.
-   * @lucene.internal
-   * @return trigger thread
-   */
-  public synchronized OverseerThread getTriggerThread() {
-    return triggerThread;
-  }
-
 
   public void closeAndDone() {
     this.closeAndDone = true;
     this.closed = true;
   }
 
+  public boolean isCloseAndDone() {
+    return closeAndDone;
+  }
+
   public void close() {
     close(false);
   }
 
   public void close(boolean fromCSUpdateThread) {
-    if (this.id != null) {
-      log.info("Overseer (id={}) closing", id);
-    }
-    if (context != null) {
-      try {
-        context.close(fromCSUpdateThread);
-      } finally {
-        if (!closeAndDone) {
-          zkController.rejoinOverseerElection(context.electionPath, false);
-        }
+    log.info("Overseer (id={}) closing closeAndDone={} frp,CSUpdateThread={}", id, closeAndDone, fromCSUpdateThread);
+
+    if (closeAndDone) {
+      if (overseerOnlyClient != null) {
+        overseerOnlyClient.disableCloseLock();
+      }
+
+      if (overseerLbClient != null) {
+        overseerLbClient.close();
+        overseerLbClient = null;
+      }
+
+      if (overseerOnlyClient != null) {
+        overseerOnlyClient.close();
+        overseerOnlyClient = null;
+      }
+    } else {
+      if (!zkController.getCoreContainer().isShutDown() && !zkController.isShudownCalled() && !zkController.isClosed()) {
+        log.info("rejoining the overseer election after closing");
+        zkController.rejoinOverseerElection( false);
       }
     }
   }
@@ -871,59 +680,41 @@ public class Overseer implements SolrCloseable {
 
   void doClose(boolean fromCSUpdateThread) {
     closed = true;
-    if (log.isDebugEnabled()) {
-      log.debug("doClose() - start");
-    }
 
-    if (ccThread != null) {
-      ((OverseerCollectionConfigSetProcessor) ccThread.getThread()).closing();
-    }
+    log.info("doClose() - start fromCSUpdateThread={}  closeAndDone={}", fromCSUpdateThread, closeAndDone);
+    this.zkStateWriter  = null;
 
-    if (overseerLbClient != null) {
-      overseerLbClient.close();
-      overseerLbClient = null;
+    if (queueWatcher != null) {
+      queueWatcher.close();
     }
 
-    if (overseerOnlyClient != null) {
-      overseerOnlyClient.disableCloseLock();
-      overseerOnlyClient.close();
-      overseerLbClient = null;
+    if (collectionQueueWatcher != null) {
+      collectionQueueWatcher.close();
     }
 
-    if (ccThread != null) {
-      ccThread.interrupt();
-      ((OverseerCollectionConfigSetProcessor) ccThread.getThread()).close(closeAndDone);
-    }
 
-    if (updaterThread != null) {
-      updaterThread.interrupt();
-      try {
-        updaterThread.getThread().close();
-      } catch (Exception e) {
-        log.warn("", e);
-      }
-    }
+//    if (stateManagmentExecutor != null) {
+//      log.info("shutdown stateManagmentExecutor");
+//      stateManagmentExecutor.shutdown();
+//    }
+//
+//    if (taskExecutor != null) {
+//      log.info("shutdown taskExecutor");
+//      taskExecutor.shutdown();
+//    }
 
-    if (ccThread != null) {
-      while (true) {
-        try {
-          ccThread.join();
-          break;
-        } catch (InterruptedException e) {
-          // okay
-        }
-      }
-    }
-    if (updaterThread != null && !fromCSUpdateThread) {
-      while (true) {
-        try {
-          updaterThread.join();
-          break;
-        } catch (InterruptedException e) {
-          // okay
-        }
-      }
-    }
+//    if (stateManagmentExecutor != null) {
+//      stateManagmentExecutor.shutdownNow();
+//    }
+
+ //   ExecutorUtil.shutdownAndAwaitTermination(stateManagmentExecutor );
+
+
+//    if (taskExecutor != null) {
+//      taskExecutor.shutdownNow();
+//    }
+
+ //   ExecutorUtil.shutdownAndAwaitTermination(taskExecutor );
 
     if (log.isDebugEnabled()) {
       log.debug("doClose() - end");
@@ -962,7 +753,7 @@ public class Overseer implements SolrCloseable {
   ZkDistributedQueue getStateUpdateQueue(Stats zkStats) {
     return new ZkDistributedQueue(reader.getZkClient(), "/overseer/queue", zkStats, STATE_UPDATE_MAX_QUEUE, new ConnectionManager.IsClosed(){
       public boolean isClosed() {
-        return Overseer.this.isClosed() || zkController.getCoreContainer().isShutDown();
+        return Overseer.this.isClosed() || zkController.getCoreContainer().isShutDown(); // nocommit use
       }
     });
   }
@@ -1076,8 +867,326 @@ public class Overseer implements SolrCloseable {
     return reader;
   }
 
+  public ZkStateWriter getZkStateWriter() {
+    return zkStateWriter;
+  }
+
   public void offerStateUpdate(byte[] data) throws KeeperException, InterruptedException {
     getStateUpdateQueue().offer(data);
   }
 
+  public boolean processQueueItem(ZkNodeProps message) throws InterruptedException {
+    log.info("processQueueItem {}", message);
+    // nocommit - may not need this now
+   new OverseerTaskExecutorTask(getCoreContainer(), zkStateWriter, message).run();
+//    try {
+//      future.get();
+//    } catch (ExecutionException e) {
+//      log.error("", e);
+//      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+//    }
+    return true;
+  }
+
+  public void writePendingUpdates() throws InterruptedException {
+    log.info("writePendingUpdates ");
+
+    new OverseerTaskExecutorTask.WriteTask(getCoreContainer(), zkStateWriter).run();
+//    try {
+//      future.get();
+//    } catch (ExecutionException e) {
+//      log.error("", e);
+//      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+//    }
+
+  }
+
+  // nocommit use
+  private static class OverseerWatcher implements Watcher, Closeable {
+
+    private final CoreContainer cc;
+    private final ZkController zkController;
+    private final String path = Overseer.OVERSEER_ELECT + "/leader";
+    private final Overseer overseer;
+
+    public OverseerWatcher(CoreContainer cc) {
+      this.cc = cc;
+      this.zkController = cc.getZkController();
+      this.overseer = zkController.getOverseer();
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+      try {
+        if (log.isDebugEnabled()) {
+          log.debug("set watch on leader znode");
+        }
+        zkController.getZkClient().exists(path, new Watcher() {
+
+          @Override
+          public void process(WatchedEvent event) {
+            if (Event.EventType.None.equals(event.getType())) {
+              return;
+            }
+
+            log.info("Overseer leader has changed, closing ...");
+            overseer.close();
+
+          }}, true);
+      } catch (KeeperException.SessionExpiredException e) {
+        log.warn("ZooKeeper session expired");
+        overseer.doClose(false);
+        return;
+      } catch (InterruptedException | AlreadyClosedException e) {
+        log.info("Already closed");
+        overseer.doClose(false);
+        return;
+      } catch (Exception e) {
+        log.error("Unexpected error in Overseer state update loop", e);
+        overseer.doClose(false);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      try {
+        zkController.getZkClient().getSolrZooKeeper().removeWatches(path, this, WatcherType.Data, true);
+      } catch (Exception e) {
+        log.info("", e.getMessage());
+      }
+    }
+  }
+
+  private static abstract class QueueWatcher implements Watcher, Closeable {
+
+    protected final CoreContainer cc;
+    protected final ZkController zkController;
+    protected final String path;
+    protected final Overseer overseer;
+    protected volatile boolean closed;
+
+    public QueueWatcher(CoreContainer cc, String path) throws KeeperException {
+      this.cc = cc;
+      this.zkController = cc.getZkController();
+      this.overseer = zkController.getOverseer();
+      this.path = path;
+      List<String> items = setWatch();
+      log.info("Overseer found entries on start {}", items);
+      processQueueItems(items);
+    }
+
+    private List<String> setWatch() {
+      try {
+
+        log.info("set watch on Overseer work queue {}", path);
+
+        List<String> children = zkController.getZkClient().getChildren(path, this, true);
+        Collections.sort(children);
+        return children;
+      } catch (KeeperException.SessionExpiredException e) {
+        log.warn("ZooKeeper session expired");
+        overseer.doClose(false);
+        return null;
+      } catch (InterruptedException | AlreadyClosedException e) {
+        log.info("Already closed");
+        overseer.doClose(false);
+        return null;
+      } catch (Exception e) {
+        log.error("Unexpected error in Overseer state update loop", e);
+        overseer.doClose(false);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+    }
+
+    @Override
+    public synchronized void process(WatchedEvent event) {
+      if (Event.EventType.None.equals(event.getType())) {
+        return;
+      }
+      if (this.closed) {
+        return;
+      }
+
+      log.info("Overseer work queue has changed, processing...");
+
+      try {
+        List<String> items = setWatch();
+
+        processQueueItems(items);
+      } catch (Exception e) {
+        log.error("Exception during overseer queue queue processing", e);
+      }
+
+    }
+
+    protected abstract void processQueueItems(List<String> items) throws KeeperException;
+
+    @Override
+    public void close() {
+      this.closed = true;
+      try {
+        zkController.getZkClient().getSolrZooKeeper().removeWatches(path, this, WatcherType.Data, true);
+      } catch (Exception e) {
+        log.info("", e.getMessage());
+      }
+    }
+  }
+
+  private static class WorkQueueWatcher extends QueueWatcher {
+
+    public WorkQueueWatcher(CoreContainer cc) throws KeeperException {
+      super(cc, Overseer.OVERSEER_QUEUE);
+    }
+
+    @Override
+    protected void processQueueItems(List<String> items) {
+      log.info("Found state update queue items {}", items);
+      List<String> fullPaths = new ArrayList<>(items.size());
+      for (String item : items) {
+        fullPaths.add(path + "/" + item);
+      }
+
+      Map<String,byte[]> data = zkController.getZkClient().getData(fullPaths);
+
+      for (byte[] item : data.values()) {
+        final ZkNodeProps message = ZkNodeProps.load(item);
+        try {
+          boolean success = overseer.processQueueItem(message);
+        } catch (InterruptedException e) {
+          log.error("Overseer state update queue processing interrupted");
+          return;
+        }
+      }
+
+      try {
+        overseer.writePendingUpdates();
+      } catch (InterruptedException e) {
+        log.error("Overseer state update queue processing interrupted");
+        return;
+      }
+
+      zkController.getZkClient().delete(fullPaths, true);
+    }
+  }
+
+  private static class CollectionWorkQueueWatcher extends QueueWatcher {
+
+    private final OverseerCollectionMessageHandler collMessageHandler;
+    private final OverseerConfigSetMessageHandler configMessageHandler;
+    private final DistributedMap failureMap;
+    private final DistributedMap runningMap;
+
+    private final DistributedMap completedMap;
+
+    public CollectionWorkQueueWatcher(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) throws KeeperException {
+      super(cc, Overseer.OVERSEER_COLLECTION_QUEUE_WORK);
+      collMessageHandler = new OverseerCollectionMessageHandler(cc, myId, overseerLbClient, adminPath, stats, overseer);
+      configMessageHandler = new OverseerConfigSetMessageHandler(cc);
+      failureMap = Overseer.getFailureMap(cc.getZkController().getZkStateReader().getZkClient());
+      runningMap = Overseer.getRunningMap(cc.getZkController().getZkStateReader().getZkClient());
+      completedMap = Overseer.getCompletedMap(cc.getZkController().getZkStateReader().getZkClient());
+    }
+
+    @Override
+    public void close() {
+      super.close();
+      IOUtils.closeQuietly(collMessageHandler);
+      IOUtils.closeQuietly(configMessageHandler);
+    }
+
+    @Override
+    protected synchronized void processQueueItems(List<String> items) throws KeeperException {
+      log.info("Found collection queue items {}", items);
+      List<String> fullPaths = new ArrayList<>(items.size());
+      for (String item : items) {
+        fullPaths.add(path + "/" + item);
+      }
+
+      Map<String,byte[]> data = zkController.getZkClient().getData(fullPaths);
+
+      ParWork.getRootSharedExecutor().submit(()->{
+        try {
+          runAsync(items, fullPaths, data);
+        } catch (Exception e) {
+          log.error("failed processing collection queue items " + items);
+        }
+      });
+
+    }
+
+    private void runAsync(List<String> items, List<String> fullPaths, Map<String,byte[]> data) throws KeeperException {
+      for (Map.Entry<String,byte[]> entry : data.entrySet()) {
+        byte[] item = entry.getValue();
+        if (item == null) {
+          log.error("empty item {}", entry.getKey());
+          continue;
+        }
+
+        final ZkNodeProps message = ZkNodeProps.load(item);
+        try {
+          String operation = message.getStr(Overseer.QUEUE_OPERATION);
+          if (operation == null) {
+            log.error("Msg does not have required " + Overseer.QUEUE_OPERATION + ": {}", message);
+            continue;
+          }
+
+          final String asyncId = message.getStr(ASYNC);
+
+          OverseerSolrResponse response;
+          if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
+            response = configMessageHandler.processMessage(message, operation);
+          } else {
+            response = collMessageHandler.processMessage(message, operation);
+          }
+
+
+//          try {
+//            overseer.writePendingUpdates();
+//          } catch (InterruptedException e) {
+//            log.error("Overseer state update queue processing interrupted");
+//            return;
+//          }
+
+          log.info("response {}", response);
+
+
+          if (asyncId != null) {
+            if (response != null && (response.getResponse().get("failure") != null || response.getResponse().get("exception") != null)) {
+              if (log.isDebugEnabled()) {
+                log.debug("Updated failed map for task with id:[{}]", asyncId);
+              }
+              failureMap.put(asyncId, OverseerSolrResponseSerializer.serialize(response));
+            } else {
+              if (log.isDebugEnabled()) {
+                log.debug("Updated completed map for task with zkid:[{}]", asyncId);
+              }
+              completedMap.put(asyncId, OverseerSolrResponseSerializer.serialize(response));
+
+            }
+          } else {
+            byte[] sdata = OverseerSolrResponseSerializer.serialize(response);
+            String responsePath = Overseer.OVERSEER_COLLECTION_MAP_COMPLETED + "/" + OverseerTaskQueue.RESPONSE_PREFIX
+                + entry.getKey().substring(entry.getKey().lastIndexOf("-") + 1);
+            zkController.getZkClient().setData( responsePath, sdata, true);
+            log.debug("Completed task:[{}] {}", message, response.getResponse());
+          }
+
+
+        } catch (InterruptedException e) {
+          log.error("Overseer state update queue processing interrupted");
+          return;
+        }
+      }
+
+      for (String item : items) {
+        if (item.startsWith("qnr-")) {
+          fullPaths.remove(path + "/" + item);
+        }
+      }
+
+      zkController.getZkClient().delete(fullPaths, true);
+    }
+  }
+
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
index 8b2e1c7..ea3ef9a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionConfigSetProcessor.java
@@ -42,65 +42,24 @@ import org.apache.zookeeper.KeeperException;
  */
 public class OverseerCollectionConfigSetProcessor extends OverseerTaskProcessor {
 
-   public OverseerCollectionConfigSetProcessor(CoreContainer cc, ZkStateReader zkStateReader, String myId, LBHttp2SolrClient overseerLbClient,
-                                               String adminPath, Stats stats, Overseer overseer,
-                                               OverseerNodePrioritizer overseerNodePrioritizer) throws KeeperException {
-    this(cc,
-        zkStateReader,
-        myId,
-        adminPath,
-        stats,
-        overseerLbClient, (HttpShardHandlerFactory) cc.getShardHandlerFactory(),
-        overseer,
-        overseerNodePrioritizer,
-        overseer.getCollectionQueue(zkStateReader.getZkClient(), stats),
-        Overseer.getRunningMap(zkStateReader.getZkClient()),
-        Overseer.getCompletedMap(zkStateReader.getZkClient()),
-        Overseer.getFailureMap(zkStateReader.getZkClient())
-    );
+  public OverseerCollectionConfigSetProcessor(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) throws KeeperException {
+    this(cc, myId, overseerLbClient, adminPath, stats, overseer, overseer.getCollectionQueue(cc.getZkController().getZkStateReader().getZkClient(), stats),
+        Overseer.getRunningMap(cc.getZkController().getZkStateReader().getZkClient()), Overseer.getCompletedMap(cc.getZkController().getZkStateReader().getZkClient()),
+        Overseer.getFailureMap(cc.getZkController().getZkStateReader().getZkClient()));
   }
 
-  protected OverseerCollectionConfigSetProcessor(CoreContainer cc, ZkStateReader zkStateReader, String myId,
-                                        String adminPath,
-                                        Stats stats,
-                                        LBHttp2SolrClient overseerLbClient,
-                                        HttpShardHandlerFactory shardHandlerFactory,
-                                        Overseer overseer,
-                                        OverseerNodePrioritizer overseerNodePrioritizer,
-                                        OverseerTaskQueue workQueue,
-                                        DistributedMap runningMap,
-                                        DistributedMap completedMap,
-                                        DistributedMap failureMap) {
-    super(
-        cc,
-        myId,
-        stats,
-        getOverseerMessageHandlerSelector(zkStateReader, myId, overseerLbClient, shardHandlerFactory,
-            adminPath, stats, overseer, overseerNodePrioritizer),
-        overseerNodePrioritizer,
-        workQueue,
-        runningMap,
-        completedMap,
-        failureMap);
+  protected OverseerCollectionConfigSetProcessor(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer, OverseerTaskQueue workQueue,
+      DistributedMap runningMap, DistributedMap completedMap, DistributedMap failureMap) {
+    super(cc, myId, stats, getOverseerMessageHandlerSelector(cc, myId, overseerLbClient, adminPath, stats, overseer), workQueue, runningMap, completedMap, failureMap);
   }
 
-  private static OverseerMessageHandlerSelector getOverseerMessageHandlerSelector(
-      ZkStateReader zkStateReader,
-      String myId,
-      LBHttp2SolrClient overseerLbClient,
-      HttpShardHandlerFactory shardHandlerFactory,
-      String adminPath,
-      Stats stats,
-      Overseer overseer,
-      OverseerNodePrioritizer overseerNodePrioritizer) {
-    final OverseerCollectionMessageHandler collMessageHandler = new OverseerCollectionMessageHandler(
-        zkStateReader, myId, overseerLbClient, shardHandlerFactory, adminPath, stats, overseer, overseerNodePrioritizer);
-    final OverseerConfigSetMessageHandler configMessageHandler = new OverseerConfigSetMessageHandler(
-        zkStateReader);
+  private static OverseerMessageHandlerSelector getOverseerMessageHandlerSelector(CoreContainer cc, String myId, LBHttp2SolrClient overseerLbClient, String adminPath, Stats stats, Overseer overseer) {
+
+    final OverseerConfigSetMessageHandler configMessageHandler = new OverseerConfigSetMessageHandler(cc);
     return new OverseerMessageHandlerSelector() {
       @Override
       public void close() throws IOException {
-        IOUtils.closeQuietly(collMessageHandler);
+
         IOUtils.closeQuietly(configMessageHandler);
       }
 
@@ -110,7 +69,7 @@ public class OverseerCollectionConfigSetProcessor extends OverseerTaskProcessor
         if (operation != null && operation.startsWith(CONFIGSETS_ACTION_PREFIX)) {
           return configMessageHandler;
         }
-        return collMessageHandler;
+        throw new IllegalArgumentException("No handler for " + operation + " " + message);
       }
     };
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
index ab95e2a..c70048d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerConfigSetMessageHandler.java
@@ -39,6 +39,7 @@ import org.apache.solr.common.params.ConfigSetParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.ConfigSetProperties;
+import org.apache.solr.core.CoreContainer;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
@@ -83,8 +84,8 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public OverseerConfigSetMessageHandler(ZkStateReader zkStateReader) {
-    this.zkStateReader = zkStateReader;
+  public OverseerConfigSetMessageHandler(CoreContainer cc) {
+    this.zkStateReader = cc.getZkController().getZkStateReader();
     this.configSetWriteWip = new HashSet();
     this.configSetReadWip = new HashSet();
   }
@@ -117,9 +118,7 @@ public class OverseerConfigSetMessageHandler implements OverseerMessageHandler {
                   + operation);
       }
     } catch (Exception e) {
-      // interrupt not currently thrown here, but it could be - I
-      // usually like to use a utility everywhere for this reason
-      ParWork.propagateInterrupt(e);
+      log.error("Operation " + operation + " caused exception", e);
 
       String configSetName = message.getStr(NAME);
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
index 6913b48..71fa2d3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerElectionContext.java
@@ -21,10 +21,14 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.cloud.ConnectionManager;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.Pair;
@@ -41,15 +45,24 @@ final class OverseerElectionContext extends ShardLeaderElectionContextBase {
   private volatile boolean isClosed = false;
 
   public OverseerElectionContext(final String zkNodeName, SolrZkClient zkClient, Overseer overseer) {
-    super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new ZkNodeProps(ID, zkNodeName), zkClient);
+    super(zkNodeName, Overseer.OVERSEER_ELECT, Overseer.OVERSEER_ELECT + "/leader", new Replica(ID, getIDMap(zkNodeName), null, null), zkClient);
     this.overseer = overseer;
     this.zkClient = zkClient;
   }
 
+  private static Map<String,Object> getIDMap(String zkNodeName) {
+    Map<String,Object> idMap = new HashMap<>(1);
+    idMap.put(ID, zkNodeName);
+    return idMap;
+  }
+
   @Override
   void runLeaderProcess(ElectionContext context, boolean weAreReplacement, int pauseBeforeStartMs) throws KeeperException,
           InterruptedException, IOException {
-    if (isClosed() || overseer.isDone()) {
+    log.info("Running the leader process for Overseer");
+
+    if (overseer.isDone()) {
+      log.info("Already closed, bailing ...");
       return;
     }
 
@@ -70,37 +83,22 @@ final class OverseerElectionContext extends ShardLeaderElectionContextBase {
 //      clearQueue(Overseer.getInternalWorkQueue(zkClient, new Stats()));
 //    }
 
-    log.info("Running the leader process for Overseer");
 
     super.runLeaderProcess(context, weAreReplacement, pauseBeforeStartMs);
 
     log.info("Registered as Overseer leader, starting Overseer ...");
 
-    if (isClosed()) {
-      log.info("Bailing on becoming leader, we are closed");
-      return;
-    }
-    if (!isClosed() && !overseer.getZkController().getCoreContainer().isShutDown() && !overseer.isDone() && (overseer.getUpdaterThread() == null || !overseer.getUpdaterThread().isAlive())) {
+    if (!overseer.getZkController().getCoreContainer().isShutDown() && !overseer.getZkController().isShudownCalled()
+        && !overseer.isDone()) {
+      log.info("Starting overseer after winnning Overseer election {}", id);
       overseer.start(id, context);
+    } else {
+      log.info("Will not start Overseer because we are closed");
+      cancelElection();
     }
 
   }
 
-  private void clearQueue(ZkDistributedQueue queue)
-      throws KeeperException, InterruptedException {
-    while (true) {
-      Collection<Pair<String,byte[]>> items = queue.peekElements(1000, 0, null);
-      List<String> paths = new ArrayList<>(items.size());
-      if (items.size() == 0) {
-        break;
-      }
-      for (Pair<String, byte[]> item : items) {
-        paths.add(item.first());
-      }
-      queue.remove(paths);
-    }
-  }
-
   public Overseer getOverseer() {
     return  overseer;
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java
new file mode 100644
index 0000000..90d5079
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskExecutorTask.java
@@ -0,0 +1,157 @@
+package org.apache.solr.cloud;
+
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.NodeMutator;
+import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.ReplicaMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
+import org.apache.solr.cloud.overseer.ZkStateWriter;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
+import org.apache.solr.common.ParWork;
+import org.apache.solr.common.SolrException;
+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.Slice;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentLinkedQueue;
+
+public class OverseerTaskExecutorTask implements Runnable {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final ZkController zkController;
+  private final SolrCloudManager cloudManager;
+  private final SolrZkClient zkClient;
+  private final Overseer overseer;
+  private final ZkStateWriter zkStateWriter;
+  private final ZkNodeProps message;
+
+  public OverseerTaskExecutorTask(CoreContainer cc, ZkStateWriter zkStateWriter, ZkNodeProps message) {
+    this.zkController = cc.getZkController();
+    this.zkClient = zkController.getZkClient();
+    this.cloudManager = zkController.getSolrCloudManager();
+    this.overseer = zkController.getOverseer();
+    this.zkStateWriter = zkStateWriter;
+    this.message = message;
+  }
+
+
+  private void processQueueItem(ZkNodeProps message) throws Exception {
+    if (log.isDebugEnabled()) log.debug("Consume state update from queue {} {}", message);
+
+    // assert clusterState != null;
+
+    //  if (clusterState.getZNodeVersion() == 0 || clusterState.getZNodeVersion() > lastVersion) {
+
+    final String operation = message.getStr(Overseer.QUEUE_OPERATION);
+    if (operation == null) {
+      log.error("Message missing " + Overseer.QUEUE_OPERATION + ":" + message);
+      return;
+    }
+
+    log.info("Queue operation is {}", operation);
+
+    ClusterState cs = zkStateWriter.getClusterstate(true);
+
+    log.info("Process message {} {}", message, operation);
+    ClusterState newClusterState = processMessage(message, operation, cs);
+
+    log.info("Enqueue message {}", operation);
+    zkStateWriter.enqueueUpdate(newClusterState, true);
+
+
+    if (log.isDebugEnabled()) log.debug("State update consumed from queue {}", message);
+  }
+
+  private ClusterState processMessage(final ZkNodeProps message, final String operation, ClusterState clusterState) {
+    if (log.isDebugEnabled()) {
+      log.debug("processMessage(ZkNodeProps message={}, String operation={} clusterState={})", message, operation, clusterState);
+    }
+
+    OverseerAction overseerAction = OverseerAction.get(operation);
+    if (overseerAction == null) {
+      throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
+    }
+    switch (overseerAction) {
+      case STATE:
+        return new ReplicaMutator(cloudManager).setState(clusterState, message);
+      case LEADER:
+        return new SliceMutator(cloudManager).setShardLeader(clusterState, message);
+      case ADDROUTINGRULE:
+        return new SliceMutator(cloudManager).addRoutingRule(clusterState, message);
+      case REMOVEROUTINGRULE:
+        return new SliceMutator(cloudManager).removeRoutingRule(clusterState, message);
+      case UPDATESHARDSTATE:
+        return new SliceMutator(cloudManager).updateShardState(clusterState, message);
+      //          case QUIT:
+      //            if (myId.equals(message.get(ID))) {
+      //              log.info("Quit command received {} {}", message, LeaderElector.getNodeName(myId));
+      //              try {
+      //                overseerCollectionConfigSetProcessor.close();
+      //              } catch (IOException e) {
+      //                log.error("IOException", e);
+      //              }
+      //              close();
+      //            } else {
+      //              log.warn("Overseer received wrong QUIT message {}", message);
+      //            }
+      //            break;
+      case DOWNNODE:
+        return new NodeMutator().downNode(clusterState, message);
+      default:
+        throw new RuntimeException("unknown operation:" + operation + " contents:" + message.getProperties());
+
+    }
+  }
+
+  @Override
+  public void run() {
+    log.info("OverseerTaskExecutorTask, going to process message {}", message);
+
+    try {
+      processQueueItem(message);
+    } catch (Exception e) {
+      log.error("Failed to process message " + message, e);
+    }
+  }
+
+  public static class WriteTask implements Runnable {
+    CoreContainer coreContainer;
+    ZkStateWriter zkStateWriter;
+
+    public WriteTask(CoreContainer coreContainer, ZkStateWriter zkStateWriter) {
+      this.coreContainer = coreContainer;
+      this.zkStateWriter = zkStateWriter;
+    }
+
+    @Override
+    public void run() {
+      try {
+        zkStateWriter.writePendingUpdates();
+      } catch (NullPointerException e) {
+        if (log.isDebugEnabled()) log.debug("Won't write pending updates, zkStateWriter=null");
+      } catch (Exception e) {
+        log.error("Failed to process pending updates", e);
+      }
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
index 0b6498d..628a32e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskProcessor.java
@@ -125,15 +125,12 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
   protected final OverseerMessageHandlerSelector selector;
 
-  private final OverseerNodePrioritizer prioritizer;
-
   private final String thisNode;
   private Map<Runner,Future> taskFutures = new ConcurrentHashMap<>();
 
   public OverseerTaskProcessor(CoreContainer cc, String myId,
                                         Stats stats,
                                         OverseerMessageHandlerSelector selector,
-                                        OverseerNodePrioritizer prioritizer,
                                         OverseerTaskQueue workQueue,
                                         DistributedMap runningMap,
                                         DistributedMap completedMap,
@@ -141,7 +138,6 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
     this.myId = myId;
     this.stats = stats;
     this.selector = selector;
-    this.prioritizer = prioritizer;
     this.workQueue = workQueue;
     this.runningMap = runningMap;
     this.completedMap = completedMap;
@@ -350,9 +346,9 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
     if (closeAndDone) {
       // nocommit
-//      for (Future future : taskFutures.values()) {
-//        future.cancel(false);
-//      }
+      //      for (Future future : taskFutures.values()) {
+      //        future.cancel(false);
+      //      }
       for (Future future : taskFutures.values()) {
         try {
           future.get(1, TimeUnit.SECONDS);
@@ -378,8 +374,8 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
   public static List<String> getSortedElectionNodes(SolrZkClient zk, String path) throws KeeperException, InterruptedException {
     List<String> children = zk.getChildren(path, null, true);
-      LeaderElector.sortSeqs(children);
-      return children;
+    LeaderElector.sortSeqs(children);
+    return children;
   }
 
   public static String getLeaderNode(SolrZkClient zkClient) throws KeeperException, InterruptedException {
@@ -533,15 +529,15 @@ public class OverseerTaskProcessor implements Runnable, Closeable {
 
     private void resetTaskWithException(String id, String asyncId, String taskKey) throws KeeperException, InterruptedException {
       log.warn("Resetting task: {}, requestid: {}, taskKey: {}", id, asyncId, taskKey);
-        if (asyncId != null) {
-          if (!runningMap.remove(asyncId)) {
-            log.warn("Could not find and remove async call [{}] from the running map.", asyncId);
-          }
+      if (asyncId != null) {
+        if (!runningMap.remove(asyncId)) {
+          log.warn("Could not find and remove async call [{}] from the running map.", asyncId);
         }
+      }
 
-        synchronized (runningTasks) {
-          runningTasks.remove(id);
-        }
+      synchronized (runningTasks) {
+        runningTasks.remove(id);
+      }
 
     }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
index eabc34f..9c0f6f9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
@@ -16,31 +16,29 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.function.Predicate;
 
 import com.codahale.metrics.Timer;
-import org.apache.solr.common.ParWork;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -52,9 +50,11 @@ import org.slf4j.LoggerFactory;
 public class OverseerTaskQueue extends ZkDistributedQueue {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
-  public static final String RESPONSE_PREFIX = "qnr-" ;
+  static final String RESPONSE_PREFIX = "qnr-" ;
+  public static final byte[] BYTES = new byte[0];
 
-  private final LongAdder pendingResponses = new LongAdder();
+  private final AtomicBoolean shuttingDown = new AtomicBoolean(false);
+  private final AtomicInteger pendingResponses = new AtomicInteger(0);
 
   public OverseerTaskQueue(SolrZkClient zookeeper, String dir) {
     this(zookeeper, dir, new Stats());
@@ -65,11 +65,11 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
   }
 
   public void allowOverseerPendingTasksToComplete() {
-    while (pendingResponses.sum() > 0) {
+    shuttingDown.set(true);
+    while (pendingResponses.get() > 0) {
       try {
-        Thread.sleep(100);
+        Thread.sleep(50);
       } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
         log.error("Interrupted while waiting for overseer queue to drain before shutdown!");
       }
     }
@@ -80,47 +80,28 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
    */
   public boolean containsTaskWithRequestId(String requestIdKey, String requestId)
       throws KeeperException, InterruptedException {
-    Set<String> childNames;
-        updateLock.lockInterruptibly();
-    try {
-
-     childNames = knownChildren.keySet();
 
-
-      stats.setQueueLength(childNames.size());
-      for (String childName : childNames) {
-        if (childName != null && childName.startsWith(PREFIX)) {
-          try {
-            byte[] data = knownChildren.get(childName);
-            if (data == null) {
-              data = zookeeper.getData(dir + "/" + childName, null, null, true);
-              if (data != null) {
-                knownChildren.put(childName, data);
-              }
-            }
-            if (data != null) {
-              ZkNodeProps message = ZkNodeProps.load(data);
-              if (message.containsKey(requestIdKey)) {
-                if (log.isDebugEnabled()) {
-                  log.debug("Looking for {}, found {}", message.get(requestIdKey), requestId);
-                }
-                if(message.get(requestIdKey).equals(requestId)) return true;
+    List<String> childNames = zookeeper.getChildren(dir, null, true);
+    stats.setQueueLength(childNames.size());
+    for (String childName : childNames) {
+      if (childName != null && childName.startsWith(PREFIX)) {
+        try {
+          byte[] data = zookeeper.getData(dir + "/" + childName, null, null, true);
+          if (data != null) {
+            ZkNodeProps message = ZkNodeProps.load(data);
+            if (message.containsKey(requestIdKey)) {
+              if (log.isDebugEnabled()) {
+                log.debug("Looking for {}, found {}", message.get(requestIdKey), requestId);
               }
+              if(message.get(requestIdKey).equals(requestId)) return true;
             }
-          } catch (KeeperException.NoNodeException e) {
-            knownChildren.remove(childName);
           }
+        } catch (KeeperException.NoNodeException e) {
+          // Another client removed the node first, try next
         }
       }
-
-    } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
-      }
     }
 
-
-
     return false;
   }
 
@@ -131,30 +112,20 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
       InterruptedException {
     Timer.Context time = stats.time(dir + "_remove_event");
     try {
-      String path = dir + "/" + event.getId();
-      String responseId = RESPONSE_PREFIX
+      String path = event.getId();
+      String responsePath = dir + "/" + RESPONSE_PREFIX
           + path.substring(path.lastIndexOf("-") + 1);
-      String responsePath = dir + "/" + responseId;
 
       try {
         zookeeper.setData(responsePath, event.getBytes(), true);
       } catch (KeeperException.NoNodeException ignored) {
-        // this will often not exist or have been removed - nocommit - debug this response stuff
-        if (log.isDebugEnabled()) log.debug("Response ZK path: {} doesn't exist.", responsePath);
+        // we must handle the race case where the node no longer exists
+        log.info("Response ZK path: {} doesn't exist. Requestor may have disconnected from ZooKeeper", responsePath);
       }
       try {
         zookeeper.delete(path, -1, true);
       } catch (KeeperException.NoNodeException ignored) {
       }
-
-      updateLock.lockInterruptibly();
-      try {
-        knownChildren.remove(event.getId());
-      } finally {
-        if (updateLock.isHeldByCurrentThread()) {
-          updateLock.unlock();
-        }
-      }
     } finally {
       time.stop();
     }
@@ -165,20 +136,16 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
    */
   static final class LatchWatcher implements Watcher {
 
-    private final ReentrantLock lock;
+    private final Lock lock;
     private final Condition eventReceived;
-    private final SolrZkClient zkClient;
-    private volatile WatchedEvent event;
-    private final Event.EventType latchEventType;
-
-    private volatile boolean triggered = false;
+    private WatchedEvent event;
+    private Event.EventType latchEventType;
 
-    LatchWatcher(SolrZkClient zkClient) {
-      this(null, zkClient);
+    LatchWatcher() {
+      this(null);
     }
 
-    LatchWatcher(Event.EventType eventType, SolrZkClient zkClient) {
-      this.zkClient = zkClient;
+    LatchWatcher(Event.EventType eventType) {
       this.lock = new ReentrantLock();
       this.eventReceived = lock.newCondition();
       this.latchEventType = eventType;
@@ -192,47 +159,30 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
         return;
       }
       // If latchEventType is not null, only fire if the type matches
-      if (log.isDebugEnabled()) {
-        log.debug("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
-      }
+
+      log.info("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
+
       if (latchEventType == null || event.getType() == latchEventType) {
-        try {
-          lock.lockInterruptibly();
-        } catch (InterruptedException e) {
-          return;
-        }
+        lock.lock();
         try {
           this.event = event;
-          triggered = true;
           eventReceived.signalAll();
         } finally {
-          if (lock.isHeldByCurrentThread()) {
-            lock.unlock();
-          }
+          lock.unlock();
         }
       }
     }
 
     public void await(long timeoutMs) throws InterruptedException {
       assert timeoutMs > 0;
-      lock.lockInterruptibly();
+      lock.lock();
       try {
         if (this.event != null) {
           return;
         }
-        TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
-        while (event == null && !timeout.hasTimedOut()) {
-          try {
-            eventReceived.await(timeoutMs, TimeUnit.MILLISECONDS);
-          } catch (InterruptedException e) {
-            ParWork.propagateInterrupt(e);
-            throw e;
-          }
-        }
+        eventReceived.await(timeoutMs, TimeUnit.MILLISECONDS);
       } finally {
-        if (lock.isHeldByCurrentThread()) {
-          lock.unlock();
-        }
+        lock.unlock();
       }
     }
 
@@ -248,7 +198,17 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
    */
   private String createData(String path, byte[] data, CreateMode mode)
       throws KeeperException, InterruptedException {
-    return zookeeper.create(path, data, mode, true);
+    for (;;) {
+      try {
+        return zookeeper.create(path, data, mode, true);
+      } catch (KeeperException.NoNodeException e) {
+        try {
+          zookeeper.create(dir, BYTES, CreateMode.PERSISTENT, true);
+        } catch (KeeperException.NodeExistsException ne) {
+          // someone created it
+        }
+      }
+    }
   }
 
   /**
@@ -257,60 +217,68 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
    */
   public QueueEvent offer(byte[] data, long timeout) throws KeeperException,
       InterruptedException {
-    Timer.Context time = stats.time(dir + "_offer");
+    log.info("offer operation to the Overseeer queue {}", Utils.fromJSON(data));
+    if (shuttingDown.get()) {
+      throw new SolrException(SolrException.ErrorCode.CONFLICT,"Solr is shutting down, no more overseer tasks may be offered");
+    }
+   // Timer.Context time = stats.time(dir + "_offer");
     try {
       // Create and watch the response node before creating the request node;
       // otherwise we may miss the response.
       String watchID = createResponseNode();
 
-      LatchWatcher watcher = new LatchWatcher(zookeeper);
-      byte[] bytes = zookeeper.getData(watchID, watcher, null, true);
+      log.info("watchId for response node {}, setting a watch ... ", watchID);
 
-      // create the request node
-      createRequestNode(data, watchID);
+      LatchWatcher watcher = new LatchWatcher(Watcher.Event.EventType.NodeDataChanged);
+      Stat stat = zookeeper.exists(watchID, watcher, true);
 
-      pendingResponses.increment();
-      if (bytes == null) {
+      // create the request node
+     String path = createRequestNode(data, watchID);
+      log.info("created request node at {}", path);
+      if (stat != null) {
+        pendingResponses.incrementAndGet();
+        log.info("wait on latch {}", timeout);
         watcher.await(timeout);
-        bytes = zookeeper.getData(watchID, null, null, true);
       }
-
+      byte[] bytes = zookeeper.getData(watchID, null, null, true);
+      log.info("get data from response node {} {} {}", watchID, bytes == null ? null : bytes.length, watcher.getWatchedEvent());
       // create the event before deleting the node, otherwise we can get the deleted
       // event from the watcher.
       QueueEvent event =  new QueueEvent(watchID, bytes, watcher.getWatchedEvent());
+      log.info("delete response node... {}", watchID);
       zookeeper.delete(watchID, -1, true);
       return event;
     } finally {
-      time.stop();
-      pendingResponses.decrement();
+     // time.stop();
+      pendingResponses.decrementAndGet();
     }
   }
 
-  void createRequestNode(byte[] data, String watchID) throws KeeperException, InterruptedException {
-    createData(dir + "/" + PREFIX + watchID.substring(watchID.lastIndexOf("-") + 1),
+
+  String createRequestNode(byte[] data, String watchID) throws KeeperException, InterruptedException {
+    return createData(dir + "/" + PREFIX + watchID.substring(watchID.lastIndexOf("-") + 1),
         data, CreateMode.PERSISTENT);
   }
 
   String createResponseNode() throws KeeperException, InterruptedException {
     return createData(
-            dir + "/" + RESPONSE_PREFIX,
-            null, CreateMode.EPHEMERAL_SEQUENTIAL);
+        Overseer.OVERSEER_COLLECTION_MAP_COMPLETED + "/" + RESPONSE_PREFIX,
+        null, CreateMode.EPHEMERAL_SEQUENTIAL);
   }
 
 
   public List<QueueEvent> peekTopN(int n, Predicate<String> excludeSet, long waitMillis)
       throws KeeperException, InterruptedException {
-    if (log.isDebugEnabled()) log.debug("peekTopN {} {}", n, excludeSet);
     ArrayList<QueueEvent> topN = new ArrayList<>();
 
+    log.debug("Peeking for top {} elements. ExcludeSet: {}", n, excludeSet);
     Timer.Context time;
     if (waitMillis == Long.MAX_VALUE) time = stats.time(dir + "_peekTopN_wait_forever");
     else time = stats.time(dir + "_peekTopN_wait" + waitMillis);
 
     try {
-      for (Pair<String, byte[]> element : peekElements(n, waitMillis, excludeSet)) {
-        if (log.isDebugEnabled()) log.debug("Add to topN {}", dir + "/" + element.first());
-        topN.add(new QueueEvent(new File(element.first()).getName(),
+      for (Pair<String, byte[]> element : peekElements(n, waitMillis, child -> !excludeSet.test(dir + "/" + child))) {
+        topN.add(new QueueEvent(dir + "/" + element.first(),
             element.second(), null));
       }
       printQueueEventsListElementIds(topN);
@@ -337,45 +305,20 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
    * Gets last element of the Queue without removing it.
    */
   public String getTailId() throws KeeperException, InterruptedException {
+    log.info("getTailId");
     // TODO: could we use getChildren here?  Unsure what freshness guarantee the caller needs.
-    TreeMap<String,byte[]> orderedChildren = fetchZkChildren(null, null);
-
-    for (Map.Entry<String,byte[]>  headNode : orderedChildren.entrySet())
+    TreeSet<String> orderedChildren = fetchZkChildren(null);
+    log.info("getTailId found {} children", orderedChildren);
+    for (String headNode : orderedChildren.descendingSet())
       if (headNode != null) {
         try {
-          byte[] data;
-          updateLock.lockInterruptibly();
-          try {
-            data = knownChildren.get(headNode.getKey());
-          } finally {
-            if (updateLock.isHeldByCurrentThread()) {
-              updateLock.unlock();
-            }
-          }
-          if (data == null) {
-            data = zookeeper.getData(dir + "/" + headNode.getKey(), null, null, true);
-          }
-          QueueEvent queueEvent = new QueueEvent(headNode.getKey(), data, null);
-
-          updateLock.lockInterruptibly();
-          try {
-            knownChildren.put(headNode.getKey(), data);
-          } finally {
-            if (updateLock.isHeldByCurrentThread()) {
-              updateLock.unlock();
-            }
-          }
-
+          QueueEvent queueEvent = new QueueEvent(dir + "/" + headNode, zookeeper.getData(dir + "/" + headNode,
+              null, null, true), null);
+          log.info("return {}", queueEvent.getId());
           return queueEvent.getId();
         } catch (KeeperException.NoNodeException e) {
-          updateLock.lockInterruptibly();
-          try {
-            knownChildren.remove(headNode.getKey());
-          } finally {
-            if (updateLock.isHeldByCurrentThread()) {
-              updateLock.unlock();
-            }
-          }
+          // Another client removed the node first, try next
+          log.info("no node found {}", dir + "/" + headNode);
         }
       }
     return null;
@@ -402,12 +345,11 @@ public class OverseerTaskQueue extends ZkDistributedQueue {
       return true;
     }
 
-    private volatile WatchedEvent event = null;
-    private volatile String id;
-    private volatile  byte[] bytes;
+    private WatchedEvent event = null;
+    private String id;
+    private byte[] bytes;
 
     QueueEvent(String id, byte[] bytes, WatchedEvent event) {
-      if (log.isDebugEnabled()) log.debug("Create QueueEvent with id {} {} {}", id, bytes != null ? bytes.length : 0, event);
       this.id = id;
       this.bytes = bytes;
       this.event = event;
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
index 75cef2f..2ce6251 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveringCoreTermWatcher.java
@@ -55,11 +55,11 @@ public class RecoveringCoreTermWatcher implements ZkShardTerms.CoreTermWatcher {
       }
 
       if (solrCore.getCoreDescriptor() == null || solrCore.getCoreDescriptor().getCloudDescriptor() == null) return true;
-      String coreNodeName = solrCore.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
-      if (terms.haveHighestTermValue(coreNodeName)) return true;
-      if (terms.getTerm(coreNodeName) != null && lastTermDoRecovery.get() < terms.getTerm(coreNodeName)) {
-        log.info("Start recovery on {} because core's term is less than leader's term", coreNodeName);
-        lastTermDoRecovery.set(terms.getTerm(coreNodeName));
+      String coreName = solrCore.getCoreDescriptor().getName();
+      if (terms.haveHighestTermValue(coreName)) return true;
+      if (terms.getTerm(coreName) != null && lastTermDoRecovery.get() < terms.getTerm(coreName)) {
+        log.info("Start recovery on {} because core's term is less than leader's term", coreName);
+        lastTermDoRecovery.set(terms.getTerm(coreName));
         solrCore.getUpdateHandler().getSolrCoreState().doRecovery(solrCore.getCoreContainer(), solrCore.getCoreDescriptor());
       }
     } catch (Exception e) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
index f1ea34b..595344b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
@@ -69,6 +69,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
@@ -83,6 +84,8 @@ import java.util.concurrent.atomic.AtomicInteger;
  */
 public class RecoveryStrategy implements Runnable, Closeable {
 
+  private volatile CountDownLatch latch;
+
   public static class Builder implements NamedListInitializedPlugin {
     private NamedList args;
 
@@ -123,7 +126,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
   private volatile RecoveryListener recoveryListener;
   private final ZkController zkController;
   private final String baseUrl;
-  private volatile String coreZkNodeName;
   private final ZkStateReader zkStateReader;
   private volatile String coreName;
   private final AtomicInteger retries = new AtomicInteger(0);
@@ -148,7 +150,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
     zkController = cc.getZkController();
     zkStateReader = zkController.getZkStateReader();
     baseUrl = zkController.getBaseUrl();
-    coreZkNodeName = cd.getCloudDescriptor().getCoreNodeName();
     replicaType = cd.getCloudDescriptor().getReplicaType();
   }
 
@@ -211,17 +212,25 @@ public class RecoveryStrategy implements Runnable, Closeable {
           if (finalReplicationHandler != null) finalReplicationHandler.abortFetch();
         });
 
+        closer.collect("latch", () -> {
+          try {
+            latch.countDown();
+            latch = null;
+          } catch (NullPointerException e) {
+            // expected
+          }
+        });
+
       }
     } finally {
       core = null;
     }
-    log.warn("Stopping recovery for core=[{}] coreNodeName=[{}]", coreName, coreZkNodeName);
+    log.warn("Stopping recovery for core=[{}]", coreName);
     //ObjectReleaseTracker.release(this);
   }
 
   final private void recoveryFailed(final SolrCore core,
-      final ZkController zkController, final String baseUrl,
-      final String shardZkNodeName, final CoreDescriptor cd) throws Exception {
+      final ZkController zkController, final String baseUrl, final CoreDescriptor cd) throws Exception {
     SolrException.log(log, "Recovery failed - I give up.");
     try {
       if (zkController.getZkClient().isConnected()) {
@@ -239,16 +248,16 @@ public class RecoveryStrategy implements Runnable, Closeable {
    * 
    * @lucene.experimental
    */
-  protected String getReplicateLeaderUrl(ZkNodeProps leaderprops) {
-    return new ZkCoreNodeProps(leaderprops).getCoreUrl();
+  protected String getReplicateLeaderUrl(Replica leaderprops) {
+    return leaderprops.getCoreUrl();
   }
 
-  final private void replicate(String nodeName, SolrCore core, ZkNodeProps leaderprops)
+  final private void replicate(String nodeName, SolrCore core, Replica leaderprops)
       throws SolrServerException, IOException {
 
     final String leaderUrl = getReplicateLeaderUrl(leaderprops);
 
-    log.info("Attempting to replicate from [{}].", leaderUrl);
+    log.info("Attempting to replicate from [{}].", leaderprops);
 
     // send commit
     commitOnLeader(leaderUrl);
@@ -268,8 +277,10 @@ public class RecoveryStrategy implements Runnable, Closeable {
     // always download the tlogs from the leader when running with cdcr enabled. We need to have all the tlogs
     // to ensure leader failover doesn't cause missing docs on the target
 
-    if (isClosed()) return; // we check closed on return
     boolean success = false;
+
+    log.info("do replication fetch [{}].", solrParams);
+
     IndexFetcher.IndexFetchResult result = replicationHandler.doFetch(solrParams, false);
 
     if (result.getMessage().equals(IndexFetcher.IndexFetchResult.FAILED_BY_INTERRUPT_MESSAGE)) {
@@ -278,7 +289,10 @@ public class RecoveryStrategy implements Runnable, Closeable {
     }
 
     if (result.getSuccessful()) {
+      log.info("replication fetch reported as success");
       success= true;
+    } else {
+      log.error("replication fetch reported as failed: {} {} {}", result.getMessage(), result, result.getException());
     }
 
     if (!success) {
@@ -319,6 +333,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
   final private void commitOnLeader(String leaderUrl) throws SolrServerException,
       IOException {
+    log.info("send commit to leader");
     Http2SolrClient client = core.getCoreContainer().getUpdateShardHandler().getRecoveryOnlyClient();
     UpdateRequest ureq = new UpdateRequest();
     ureq.setBasePath(leaderUrl);
@@ -333,9 +348,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
   @Override
   final public void run() {
     try {
-      if (isClosed()) {
-        return;
-      }
+
       // set request info for logging
 
 
@@ -389,7 +402,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
                                                                                             // though
       try {
         CloudDescriptor cloudDesc = this.coreDescriptor.getCloudDescriptor();
-        ZkNodeProps leaderprops = zkStateReader.getLeaderRetry(
+        Replica leaderprops = zkStateReader.getLeaderRetry(
             cloudDesc.getCollectionName(), cloudDesc.getShardId());
         final String leaderBaseUrl = leaderprops.getStr(ZkStateReader.BASE_URL_PROP);
         final String leaderCoreName = leaderprops.getStr(ZkStateReader.CORE_NAME_PROP);
@@ -488,7 +501,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
           if (retries.incrementAndGet() >= maxRetries) {
             SolrException.log(log, "Recovery failed - max retries exceeded (" + retries + ").");
             try {
-              recoveryFailed(core, zkController, baseUrl, coreZkNodeName, this.coreDescriptor);
+              recoveryFailed(core, zkController, baseUrl, this.coreDescriptor);
             } catch (InterruptedException e) {
               ParWork.propagateInterrupt(e);
               return;
@@ -533,13 +546,14 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
   // TODO: perhaps make this grab a new core each time through the loop to handle core reloads?
   public final void doSyncOrReplicateRecovery(SolrCore core) throws Exception {
+    log.info("Do peersync or replication recovery core={} collection={}", core.getName(), core.getCoreDescriptor().getCollectionName());
     boolean successfulRecovery = false;
 
     UpdateLog ulog;
     ulog = core.getUpdateHandler().getUpdateLog();
     if (ulog == null) {
       SolrException.log(log, "No UpdateLog found - cannot recover.");
-      recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
+      recoveryFailed(core, zkController, baseUrl,
           this.coreDescriptor);
       return;
     }
@@ -659,12 +673,8 @@ public class RecoveryStrategy implements Runnable, Closeable {
           break;
         }
 
-        sendPrepRecoveryCmd(leader.getBaseUrl(), leader.getCoreName(), slice);
+        sendPrepRecoveryCmd(leader.getBaseUrl(), leader.getName(), slice);
 
-        if (isClosed()) {
-          log.info("RecoveryStrategy has been closed");
-          break;
-        }
 
         // we wait a bit so that any updates on the leader
         // that started before they saw recovering state
@@ -704,7 +714,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
             cloudDebugLog(core, "synced");
 
             log.info("Replaying updates buffered during PeerSync.");
-            replayFuture = replay(core);
+            replay(core);
 
             // sync success
             successfulRecovery = true;
@@ -714,10 +724,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
           log.info("PeerSync Recovery was not successful - trying replication.");
         }
 
-        if (isClosed()) {
-          log.info("RecoveryStrategy has been closed");
-          break;
-        }
 
         log.info("Starting Replication Recovery.");
 
@@ -725,17 +731,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
           replicate(zkController.getNodeName(), core, leader);
 
-          if (isClosed()) {
-            log.info("RecoveryStrategy has been closed");
-            break;
-          }
-
-          replayFuture = replay(core);
-
-          if (isClosed()) {
-            log.info("RecoveryStrategy has been closed");
-            break;
-          }
+          replay(core);
 
           log.info("Replication Recovery was successful.");
           successfulRecovery = true;
@@ -780,17 +776,13 @@ public class RecoveryStrategy implements Runnable, Closeable {
         // Or do a fall off retry...
         try {
 
-          if (isClosed()) {
-            log.info("RecoveryStrategy has been closed");
-            break;
-          }
 
           log.error("Recovery failed - trying again... ({})", retries);
 
           if (retries.incrementAndGet() >= maxRetries) {
             SolrException.log(log, "Recovery failed - max retries exceeded (" + retries + ").");
             try {
-              recoveryFailed(core, zkController, baseUrl, coreZkNodeName, this.coreDescriptor);
+              recoveryFailed(core, zkController, baseUrl, this.coreDescriptor);
             } catch(InterruptedException e) {
               ParWork.propagateInterrupt(e);
               return;
@@ -842,7 +834,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
     while (true) {
       CloudDescriptor cloudDesc = coreDesc.getCloudDescriptor();
       DocCollection docCollection = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName());
-      if (!isClosed() && mayPutReplicaAsDown && numTried == 1 && docCollection.getReplica(coreDesc.getCloudDescriptor().getCoreNodeName()).getState() == Replica.State.ACTIVE) {
+      if (!isClosed() && mayPutReplicaAsDown && numTried == 1 && docCollection.getReplica(coreDesc.getName()).getState() == Replica.State.ACTIVE) {
         // this operation may take a long time, by putting replica into DOWN state, client won't query this replica
         // zkController.publish(coreDesc, Replica.State.DOWN);
         // TODO: We should be in recovery and ignored by queries?
@@ -871,7 +863,7 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
   public static Runnable testing_beforeReplayBufferingUpdates;
 
-    final private Future<RecoveryInfo> replay(SolrCore core)
+    final private void replay(SolrCore core)
       throws InterruptedException, ExecutionException {
     if (testing_beforeReplayBufferingUpdates != null) {
       testing_beforeReplayBufferingUpdates.run();
@@ -882,7 +874,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
           new ModifiableSolrParams());
       core.getUpdateHandler().getUpdateLog().copyOverBufferingUpdates(new CommitUpdateCommand(req, false));
       req.close();
-      return null;
     }
     Future<RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().applyBufferedUpdates();
     if (future == null) {
@@ -911,8 +902,6 @@ public class RecoveryStrategy implements Runnable, Closeable {
 
     // solrcloud_debug
     cloudDebugLog(core, "replayed");
-
-    return future;
   }
 
   final private void cloudDebugLog(SolrCore core, String op) {
@@ -942,41 +931,71 @@ public class RecoveryStrategy implements Runnable, Closeable {
   final private void sendPrepRecoveryCmd(String leaderBaseUrl, String leaderCoreName, Slice slice)
       throws SolrServerException, IOException {
 
+    if (coreDescriptor.getCollectionName() == null) {
+      throw new IllegalStateException("Collection name cannot be null");
+    }
+
     WaitForState prepCmd = new WaitForState();
-    prepCmd.setCoreName(leaderCoreName);
+    prepCmd.setCoreName(coreName);
     prepCmd.setNodeName(zkController.getNodeName());
-    prepCmd.setCoreNodeName(coreZkNodeName);
     prepCmd.setState(Replica.State.RECOVERING);
     prepCmd.setCheckLive(true);
     prepCmd.setOnlyIfLeader(true);
+    prepCmd.setCollection(coreDescriptor.getCollectionName());
+    prepCmd.setShardId(coreDescriptor.getCloudDescriptor().getShardId());
     final Slice.State state = slice.getState();
     if (state != Slice.State.CONSTRUCTION && state != Slice.State.RECOVERY && state != Slice.State.RECOVERY_FAILED) {
       prepCmd.setOnlyIfLeaderActive(true);
     }
 
+    log.info("Sending prep recovery command to {} for core {} params={}", leaderBaseUrl, leaderCoreName, prepCmd.getParams());
+
     int conflictWaitMs = zkController.getLeaderConflictResolveWait();
     int readTimeout = conflictWaitMs + Integer.parseInt(System.getProperty("prepRecoveryReadTimeoutExtraWait", "30000"));
     // nocommit
     try (Http2SolrClient client = new Http2SolrClient.Builder(leaderBaseUrl).withHttpClient(core.getCoreContainer().getUpdateShardHandler().
-        getRecoveryOnlyClient()).idleTimeout(readTimeout).connectionTimeout(3000).markInternalRequest().build()) {
+        getRecoveryOnlyClient()).idleTimeout(readTimeout).markInternalRequest().build()) {
       prepCmd.setBasePath(leaderBaseUrl);
       log.info("Sending prep recovery command to [{}]; [{}]", leaderBaseUrl, prepCmd);
-      Cancellable result = client.asyncRequest(prepCmd, null, new NamedListAsyncListener());
+      latch = new CountDownLatch(1);
+      Cancellable result = client.asyncRequest(prepCmd, null, new NamedListAsyncListener(latch));
       prevSendPreRecoveryHttpUriRequest = result;
+      try {
+        latch.await();
+      } catch (InterruptedException e) {
+        ParWork.propagateInterrupt(e);
+      } finally {
+        prevSendPreRecoveryHttpUriRequest = null;
+        latch = null;
+      }
 
-      client.waitForOutstandingRequests();
     }
   }
 
   private static class NamedListAsyncListener implements AsyncListener<NamedList<Object>> {
+
+    private final CountDownLatch latch;
+
+    public NamedListAsyncListener(CountDownLatch latch) {
+      this.latch = latch;
+    }
+
     @Override
     public void onSuccess(NamedList<Object> entries) {
+      try {
+        latch.countDown();
+      } catch (NullPointerException e) {
 
+      }
     }
 
     @Override
     public void onFailure(Throwable throwable) {
+      try {
+        latch.countDown();
+      } catch (NullPointerException e) {
 
+      }
     }
   }
 }
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 ae7b6c0..1be3027 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -21,6 +21,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
+import net.sf.saxon.trans.Err;
 import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.AlreadyClosedException;
@@ -68,7 +69,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
   public ShardLeaderElectionContext(LeaderElector leaderElector,
                                     final String shardId, final String collection,
-                                    final String coreNodeName, ZkNodeProps props, ZkController zkController, CoreContainer cc) {
+                                    final String coreNodeName, Replica props, ZkController zkController, CoreContainer cc) {
     super(coreNodeName, ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection
                     + "/leader_elect/" + shardId,  ZkStateReader.getShardLeadersPath(
             collection, shardId), props,
@@ -122,16 +123,16 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
   @Override
   void runLeaderProcess(ElectionContext context, boolean weAreReplacement, int pauseBeforeStart) throws KeeperException,
           InterruptedException, IOException {
-    if (isClosed()) {
-      return;
-    }
 
     String coreName = leaderProps.getStr(ZkStateReader.CORE_NAME_PROP);
+
+    log.info("Run leader process for shard election {}", coreName);
+
     ActionThrottle lt;
     try (SolrCore core = cc.getCore(coreName)) {
       if (core == null) {
-        // shutdown or removed
-        return;
+        log.error("No SolrCore found, cannot become leader {}", coreName);
+        throw new SolrException(ErrorCode.SERVER_ERROR, "No SolrCore found, cannot become leader " + coreName);
       }
       MDCLoggingContext.setCore(core);
       lt = core.getUpdateHandler().getSolrCoreState().getLeaderThrottle();
@@ -143,13 +144,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
       if (log.isDebugEnabled()) log.debug("Running the leader process for shard={} and weAreReplacement={} and leaderVoteWait={}", shardId, weAreReplacement, leaderVoteWait);
 
-      if (isClosed()) {
-        // Solr is shutting down or the ZooKeeper session expired while waiting for replicas. If the later,
-        // we cannot be sure we are still the leader, so we should bail out. The OnReconnect handler will
-        // re-register the cores and handle a new leadership election.
-        return;
-      }
-
       Replica.Type replicaType;
       String coreNodeName;
       boolean setTermToMax = false;
@@ -157,11 +151,10 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
       CoreDescriptor cd = core.getCoreDescriptor();
       CloudDescriptor cloudCd = cd.getCloudDescriptor();
       replicaType = cloudCd.getReplicaType();
-      coreNodeName = cloudCd.getCoreNodeName();
       // should I be leader?
       ZkShardTerms zkShardTerms = zkController.getShardTerms(collection, shardId);
-      if (zkShardTerms.registered(coreNodeName) && !zkShardTerms.canBecomeLeader(coreNodeName)) {
-        if (!waitForEligibleBecomeLeaderAfterTimeout(zkShardTerms, coreNodeName, leaderVoteWait)) {
+      if (zkShardTerms.registered(coreName) && !zkShardTerms.canBecomeLeader(coreName)) {
+        if (!waitForEligibleBecomeLeaderAfterTimeout(zkShardTerms, coreName, leaderVoteWait)) {
           rejoinLeaderElection(core);
           return;
         } else {
@@ -170,22 +163,14 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
       }
 
-      if (isClosed()) {
-        return;
-      }
 
       log.info("I may be the new leader - try and sync");
 
-      if (isClosed()) {
-        return;
-      }
       // nocommit
       // we are going to attempt to be the leader
       // first cancel any current recovery
       core.getUpdateHandler().getSolrCoreState().cancelRecovery();
-      if (isClosed()) {
-        return;
-      }
+
       PeerSync.PeerSyncResult result = null;
       boolean success = false;
       try {
@@ -198,6 +183,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
       UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
 
       if (!success) {
+
+        log.warn("Our sync attempt failed ulog={}", ulog);
         boolean hasRecentUpdates = false;
         if (ulog != null) {
           // TODO: we could optimize this if necessary
@@ -206,19 +193,22 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
           }
         }
 
+        log.warn("Checking for recent versions in the update log", hasRecentUpdates);
         if (!hasRecentUpdates) {
           // we failed sync, but we have no versions - we can't sync in that case
           // - we were active
           // before, so become leader anyway if no one else has any versions either
           if (result.getOtherHasVersions().orElse(false)) {
             log.info("We failed sync, but we have no versions - we can't sync in that case. But others have some versions, so we should not become leader");
-            success = false;
+              rejoinLeaderElection(core);
+              return;
           } else {
             log.info("We failed sync, but we have no versions - we can't sync in that case - we were active before, so become leader anyway");
             success = true;
           }
         }
       }
+      log.info("Our sync attempt succeeded");
 
       // solrcloud_debug
       if (log.isDebugEnabled()) {
@@ -236,92 +226,69 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
       }
       if (!success) {
-        if (isClosed()) {
-          log.info("Bailing on leader election, we are closed");
-          return;
-        }
         log.info("Sync with potential leader failed, rejoining election ...");
         rejoinLeaderElection(core);
         return;
       }
 
-      if (!isClosed()) {
-        try {
-          if (replicaType == Replica.Type.TLOG) {
-            // stop replicate from old leader
-            zkController.stopReplicationFromLeader(coreName);
-            if (weAreReplacement) {
-
-              Future<UpdateLog.RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().recoverFromCurrentLog();
-              if (future != null) {
-                log.info("Replaying tlog before become new leader");
-                future.get();
-              } else {
-                log.info("New leader does not have old tlog to replay");
-              }
-
+      try {
+        if (replicaType == Replica.Type.TLOG) {
+          // stop replicate from old leader
+          zkController.stopReplicationFromLeader(coreName);
+          if (weAreReplacement) {
+
+            Future<UpdateLog.RecoveryInfo> future = core.getUpdateHandler().getUpdateLog().recoverFromCurrentLog();
+            if (future != null) {
+              log.info("Replaying tlog before become new leader");
+              future.get();
+            } else {
+              log.info("New leader does not have old tlog to replay");
             }
+
           }
-          // in case of leaderVoteWait timeout, a replica with lower term can win the election
-          if (setTermToMax) {
-            log.error("WARNING: Potential data loss -- Replica {} became leader after timeout (leaderVoteWait) " + "without being up-to-date with the previous leader", coreNodeName);
-            zkController.getShardTerms(collection, shardId).setTermEqualsToLeader(coreNodeName);
-          }
+        }
+        // in case of leaderVoteWait timeout, a replica with lower term can win the election
+        if (setTermToMax) {
+          log.error("WARNING: Potential data loss -- Replica {} became leader after timeout (leaderVoteWait) " + "without being up-to-date with the previous leader", coreName);
+          zkController.getShardTerms(collection, shardId).setTermEqualsToLeader(coreName);
+        }
 
-          super.runLeaderProcess(context, weAreReplacement, 0);
+        super.runLeaderProcess(context, weAreReplacement, 0);
 
-          assert shardId != null;
+        assert shardId != null;
 
-          core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
-          publishActive(core);
-          ZkNodeProps zkNodes = ZkNodeProps
-              .fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(), ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.BASE_URL_PROP,
-                  leaderProps.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP), ZkStateReader.CORE_NAME_PROP,
-                  leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.CORE_NODE_NAME_PROP, leaderProps.get(ZkStateReader.CORE_NODE_NAME_PROP), ZkStateReader.STATE_PROP,
-                  Replica.State.ACTIVE.toString());
-          assert zkController != null;
-          assert zkController.getOverseer() != null;
-          zkController.getOverseer().offerStateUpdate(Utils.toJSON(zkNodes));
+        core.getCoreDescriptor().getCloudDescriptor().setLeader(true);
+        publishActive(core);
+        ZkNodeProps zkNodes = ZkNodeProps
+            .fromKeyVals(Overseer.QUEUE_OPERATION, OverseerAction.LEADER.toLower(), ZkStateReader.SHARD_ID_PROP, shardId, ZkStateReader.COLLECTION_PROP, collection, ZkStateReader.BASE_URL_PROP,
+                leaderProps.get(ZkStateReader.BASE_URL_PROP), ZkStateReader.NODE_NAME_PROP, leaderProps.get(ZkStateReader.NODE_NAME_PROP), ZkStateReader.CORE_NAME_PROP,
+                leaderProps.get(ZkStateReader.CORE_NAME_PROP), ZkStateReader.STATE_PROP, Replica.State.ACTIVE.toString());
+        assert zkController != null;
+        assert zkController.getOverseer() != null;
 
-          log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
+        log.info("Publish leader state");
+        zkController.getOverseer().offerStateUpdate(Utils.toJSON(zkNodes));
 
-        } catch (AlreadyClosedException | InterruptedException e) {
-          ParWork.propagateInterrupt("Already closed or interrupted, bailing..", e);
-          return;
-        } catch (Exception e) {
-          SolrException.log(log, "There was a problem trying to register as the leader", e);
-          ParWork.propagateInterrupt(e);
-          if (isClosed()) {
-            return;
-          }
-          if (e instanceof IOException || (e instanceof KeeperException && (!(e instanceof SessionExpiredException)))) {
+        log.info("I am the new leader: " + ZkCoreNodeProps.getCoreUrl(leaderProps) + " " + shardId);
 
-            if (core == null) {
-              if (log.isDebugEnabled()) log.debug("SolrCore not found:" + coreName + " in " + cc.getLoadedCoreNames());
-              return;
-            }
-            core.getCoreDescriptor().getCloudDescriptor().setLeader(false);
+      } catch (AlreadyClosedException | InterruptedException e) {
+        ParWork.propagateInterrupt("Already closed or interrupted, bailing..", e);
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      } catch (SessionExpiredException e) {
+        throw e;
+      } catch (Exception e) {
+        SolrException.log(log, "There was a problem trying to register as the leader", e);
 
-            // we could not publish ourselves as leader - try and rejoin election
-            try {
-              if (isClosed()) {
-                return;
-              }
-              rejoinLeaderElection(core);
-            } catch (Exception exc) {
-              ParWork.propagateInterrupt(e);
-              throw new SolrException(ErrorCode.SERVER_ERROR, e);
-            }
+        core.getCoreDescriptor().getCloudDescriptor().setLeader(false);
 
-          } else {
-            throw new SolrException(ErrorCode.SERVER_ERROR, e);
-          }
-        }
-      } else {
-        log.info("Bailing on leader election, we are closed");
-        cancelElection();
+        // we could not publish ourselves as leader - try and rejoin election
+
+        rejoinLeaderElection(core);
       }
 
+
+    } catch (AlreadyClosedException e) {
+      log.info("CoreContainer is shutting down, won't become leader");
     } finally {
       MDCLoggingContext.clear();
     }
@@ -334,7 +301,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
    */
   private boolean waitForEligibleBecomeLeaderAfterTimeout(ZkShardTerms zkShardTerms, String coreNodeName, int timeout) throws InterruptedException {
     long timeoutAt = System.nanoTime() + TimeUnit.NANOSECONDS.convert(timeout, TimeUnit.MILLISECONDS);
-    while (!isClosed()) {
+    while (true) {
       if (System.nanoTime() > timeoutAt) {
         log.warn("After waiting for {}ms, no other potential leader was found, {} try to become leader anyway (core_term:{}, highest_term:{})",
             timeout, coreNodeName, zkShardTerms.getTerm(coreNodeName), zkShardTerms.getHighestTerm());
@@ -349,7 +316,6 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
       
       Thread.sleep(500L);
     }
-    return false;
   }
 
   /**
@@ -386,20 +352,11 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
   private void rejoinLeaderElection(SolrCore core)
           throws InterruptedException, KeeperException, IOException {
     // remove our ephemeral and re join the election
-    if (isClosed()) {
-      log.debug("Not rejoining election because CoreContainer is closed");
-      return;
-    }
 
-    log.info("There may be a better leader candidate than us - going back into recovery");
+    log.info("There may be a better leader candidate than us - will cancel election, rejoin election, and kick off recovery");
 
     cancelElection();
 
-    if (isClosed()) {
-      log.debug("Not rejoining election because CoreContainer is closed");
-      return;
-    }
-
     this.isClosed = false;
     super.closed = false;
     leaderElector.joinElection(this, true);
@@ -415,9 +372,5 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
     return collection;
   }
 
-  @Override
-  public boolean isClosed() {
-    return isClosed || cc.isShutDown() || zkController.getZkClient().isConnected() == false;
-  }
 }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
index 2e69314..f1e0269 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContextBase.java
@@ -28,6 +28,7 @@ import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.util.Utils;
@@ -49,7 +50,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
   private volatile Integer leaderZkNodeParentVersion;
 
   public ShardLeaderElectionContextBase(final String coreNodeName, String electionPath, String leaderPath,
-                                        ZkNodeProps props, SolrZkClient zkClient) {
+                                        Replica props, SolrZkClient zkClient) {
     super(coreNodeName, electionPath, leaderPath, props);
     this.zkClient = zkClient;
   }
@@ -90,7 +91,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
             ops.add(Op.check(Paths.get(leaderPath).getParent().toString(), leaderZkNodeParentVersion));
             ops.add(Op.delete(leaderSeqPath, -1));
             ops.add(Op.delete(leaderPath, -1));
-            zkClient.multi(ops);
+            zkClient.multi(ops, false);
           } catch (KeeperException e) {
             if (e instanceof NoNodeException) {
               // okay
@@ -153,10 +154,11 @@ class ShardLeaderElectionContextBase extends ElectionContext {
     List<String> errors = new ArrayList<>();
 
     try {
-      if (isClosed()) {
-        log.info("Bailing on becoming leader, we are closed");
-        return;
+
+      if (leaderSeqPath == null) {
+        throw new IllegalStateException("We have won as leader, but we have no leader election node known to us leaderPath " + leaderPath);
       }
+
       log.info("Creating leader registration node {} after winning as {} parent is {}", leaderPath, leaderSeqPath, parent);
       List<Op> ops = new ArrayList<>(3);
 
@@ -170,7 +172,7 @@ class ShardLeaderElectionContextBase extends ElectionContext {
       ops.add(Op.setData(parent, null, -1));
       List<OpResult> results;
 
-      results = zkClient.multi(ops);
+      results = zkClient.multi(ops, false);
       log.info("Results from call {}", results);
       Iterator<Op> it = ops.iterator();
       for (OpResult result : results) {
diff --git a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
index d859512..4e88454 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
@@ -20,9 +20,11 @@ import java.io.Closeable;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -64,21 +66,16 @@ public class SyncStrategy implements Closeable {
     public String baseUrl;
   }
   
-  public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, ZkNodeProps leaderProps) {
+  public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, Replica leaderProps) {
     return sync(zkController, core, leaderProps, false);
   }
   
-  public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, ZkNodeProps leaderProps,
+  public PeerSync.PeerSyncResult sync(ZkController zkController, SolrCore core, Replica leaderProps,
       boolean peerSyncOnlyWithActive) {
     if (SKIP_AUTO_RECOVERY) {
       return PeerSync.PeerSyncResult.success();
     }
 
-    if (isClosed()) {
-      log.warn("Closed, skipping sync up.");
-      return PeerSync.PeerSyncResult.failure();
-    }
-
     if (log.isInfoEnabled()) {
       log.info("Sync replicas to {}", ZkCoreNodeProps.getCoreUrl(leaderProps));
     }
@@ -91,16 +88,9 @@ public class SyncStrategy implements Closeable {
     return syncReplicas(zkController, core, leaderProps, peerSyncOnlyWithActive);
   }
 
-  private boolean isClosed() {
-    return isClosed || (zkController != null && zkController.getCoreContainer().isShutDown());
-  }
-
   private PeerSync.PeerSyncResult syncReplicas(ZkController zkController, SolrCore core,
-      ZkNodeProps leaderProps, boolean peerSyncOnlyWithActive) {
-    if (isClosed()) {
-      log.info("We have been closed, won't sync with replicas");
-      return PeerSync.PeerSyncResult.failure();
-    }
+      Replica leaderProps, boolean peerSyncOnlyWithActive) {
+
     boolean success = false;
     PeerSync.PeerSyncResult result = null;
     assert core != null;
@@ -119,10 +109,6 @@ public class SyncStrategy implements Closeable {
       SolrException.log(log, "Sync Failed", e);
     }
     try {
-      if (isClosed()) {
-        log.info("We have been closed, won't attempt to sync replicas back to leader");
-        return PeerSync.PeerSyncResult.failure();
-      }
       
       if (success) {
         log.info("Sync Success - now sync replicas to me");
@@ -144,13 +130,9 @@ public class SyncStrategy implements Closeable {
   
   private PeerSync.PeerSyncResult syncWithReplicas(ZkController zkController, SolrCore core,
       ZkNodeProps props, String collection, String shardId, boolean peerSyncOnlyWithActive) throws Exception {
-    List<ZkCoreNodeProps> nodes = zkController.getZkStateReader()
-        .getReplicaProps(collection, shardId,core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName());
-    
-    if (isClosed()) {
-      log.info("We have been closed, won't sync with replicas");
-      return PeerSync.PeerSyncResult.failure();
-    }
+
+    List<Replica> nodes = zkController.getZkStateReader()
+        .getReplicaProps(collection, shardId,core.getCoreDescriptor().getName());
     
     if (nodes == null) {
       // I have no replicas
@@ -158,7 +140,7 @@ public class SyncStrategy implements Closeable {
     }
     
     List<String> syncWith = new ArrayList<>(nodes.size());
-    for (ZkCoreNodeProps node : nodes) {
+    for (Replica node : nodes) {
       syncWith.add(node.getCoreUrl());
     }
     
@@ -174,20 +156,15 @@ public class SyncStrategy implements Closeable {
   }
   
   private void syncToMe(ZkController zkController, String collection,
-                        String shardId, ZkNodeProps leaderProps, CoreDescriptor cd,
+                        String shardId, Replica leaderProps, CoreDescriptor cd,
                         int nUpdates) {
-    
-    if (isClosed()) {
-      log.info("We have been closed, won't sync replicas to me.");
-      return;
-    }
-    
+
     // sync everyone else
     // TODO: we should do this in parallel at least
-    List<ZkCoreNodeProps> nodes = zkController
+    List<Replica> nodes = zkController
         .getZkStateReader()
         .getReplicaProps(collection, shardId,
-            cd.getCloudDescriptor().getCoreNodeName());
+            cd.getName());
     if (nodes == null) {
       if (log.isInfoEnabled()) {
         log.info("{} has no replicas", ZkCoreNodeProps.getCoreUrl(leaderProps));
@@ -195,14 +172,14 @@ public class SyncStrategy implements Closeable {
       return;
     }
 
-    ZkCoreNodeProps zkLeader = new ZkCoreNodeProps(leaderProps);
-    for (ZkCoreNodeProps node : nodes) {
+
+    for (Replica node : nodes) {
       try {
         if (log.isInfoEnabled()) {
           log.info("{}: try and ask {} to sync", ZkCoreNodeProps.getCoreUrl(leaderProps), node.getCoreUrl());
         }
         
-        requestSync(node.getBaseUrl(), node.getCoreUrl(), zkLeader.getCoreUrl(), node.getCoreName(), nUpdates);
+        requestSync(node.getBaseUrl(), node.getCoreUrl(), leaderProps.getCoreUrl(), node.getName(), nUpdates);
         
       } catch (Exception e) {
         ParWork.propagateInterrupt(e);
@@ -210,23 +187,23 @@ public class SyncStrategy implements Closeable {
       }
     }
     
-    
-    for(;;) {
-      ShardResponse srsp = shardHandler.takeCompletedOrError();
-      if (srsp == null) break;
-      boolean success = handleResponse(srsp);
-      if (srsp.getException() != null) {
-        SolrException.log(log, "Sync request error: " + srsp.getException());
-      }
-      
-      if (!success) {
-        if (log.isInfoEnabled()) {
-          log.info("{}: Sync failed - replica ({}) should try to recover."
-              , ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+    if (nodes.size() > 0) {
+      for (; ; ) {
+        ShardResponse srsp = shardHandler.takeCompletedOrError();
+        if (srsp == null) break;
+        boolean success = handleResponse(srsp);
+        if (srsp.getException() != null) {
+          SolrException.log(log, "Sync request error: " + srsp.getException());
         }
-      } else {
-        if (log.isInfoEnabled()) {
-          log.info("{}: sync completed with {}", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+
+        if (!success) {
+          if (log.isInfoEnabled()) {
+            log.info("{}: Sync failed - replica ({}) should try to recover.", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+          }
+        } else {
+          if (log.isInfoEnabled()) {
+            log.info("{}: sync completed with {}", ZkCoreNodeProps.getCoreUrl(leaderProps), srsp.getShardAddress());
+          }
         }
       }
     }
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 a52567b..cdce9afa 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -111,7 +111,6 @@ import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTIONS_ZKNODE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.REJOIN_AT_HEAD_PROP;
@@ -206,7 +205,9 @@ public class ZkController implements Closeable, Runnable {
 
   private static byte[] emptyJson = "{}".getBytes(StandardCharsets.UTF_8);
 
-  private final Map<ContextKey, ElectionContext> electionContexts = new ConcurrentHashMap<>(64, 0.75f, 16) {
+  private final Map<String, LeaderElector> leaderElectors = new ConcurrentHashMap<>(16);
+
+  private final Map<ContextKey, ElectionContext> electionContexts = new ConcurrentHashMap<>(16) {
     @Override
     public ElectionContext put(ContextKey key, ElectionContext value) {
       if (ZkController.this.isClosed || cc.isShutDown()) {
@@ -216,7 +217,7 @@ public class ZkController implements Closeable, Runnable {
     }
   };
 
-  private final Map<ContextKey, ElectionContext> overseerContexts = new ConcurrentHashMap<>(3, 0.75f, 1) {
+  private final Map<ContextKey, ElectionContext> overseerContexts = new ConcurrentHashMap<>() {
     @Override
     public ElectionContext put(ContextKey key, ElectionContext value) {
       if (ZkController.this.isClosed || cc.isShutDown()) {
@@ -375,9 +376,9 @@ public class ZkController implements Closeable, Runnable {
 
   public void closeLeaderContext(CoreDescriptor cd) {
     String collection = cd.getCloudDescriptor().getCollectionName();
-    final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
+    final String coreName = cd.getName();
 
-    ContextKey contextKey = new ContextKey(collection, coreNodeName);
+    ContextKey contextKey = new ContextKey(collection, coreName);
     ElectionContext context = electionContexts.get(contextKey);
     if (context != null) {
       try {
@@ -415,9 +416,14 @@ public class ZkController implements Closeable, Runnable {
       @Override
       public synchronized void command() {
 
-        try (ParWork worker = new ParWork("disconnected", true)) {
+        try (ParWork worker = new ParWork("disconnected", true, true)) {
           worker.collect("OverseerElectionContexts", overseerContexts.values());
-          worker.collect("Overseer", ZkController.this.overseer);
+
+          worker.collect("Overseer", () -> {
+            if (ZkController.this.overseerElector != null) {
+              ZkController.this.overseerElector.getContext().close();
+            }
+          });
           worker.collect("", () -> {
             clearZkCollectionTerms();
           });
@@ -431,13 +437,18 @@ public class ZkController implements Closeable, Runnable {
         }
       }
     });
+    overseerContexts.clear();
+    electionContexts.clear();
     zkClient.setAclProvider(zkACLProvider);
     zkClient.getConnectionManager().setOnReconnect(new OnReconnect() {
 
       @Override
       public void command() throws SessionExpiredException {
         synchronized (initLock) {
-          if (cc.isShutDown() || !zkClient.isConnected()) return;
+          if (cc.isShutDown() || isClosed() || shudownCalled || !zkClient.isConnected()) {
+            log.info("skipping zk reconnect logic due to shutdown");
+            return;
+          }
           log.info("ZooKeeper session re-connected ... refreshing core states after session expiration.");
           try {
             // recreate our watchers first so that they exist even on any problems below
@@ -457,17 +468,8 @@ public class ZkController implements Closeable, Runnable {
 
             // start the overseer first as following code may need it's processing
 
-            ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient ,overseer);
-            ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
-            if (prevContext != null) {
-              prevContext.close();
-            }
-            if (overseerElector != null) {
-              ParWork.close(overseerElector.getContext());
-            }
-            overseerElector = new LeaderElector(zkClient, new ContextKey("overseer", "overseer"), overseerContexts);
-            ZkController.this.overseer = new Overseer(cc.getUpdateShardHandler(),
-                    CommonParams.CORES_HANDLER_PATH, zkStateReader, ZkController.this, cloudConfig);
+            ElectionContext context = getOverseerContext();
+
             overseerElector.setup(context);
             overseerElector.joinElection(context, true);
 
@@ -477,7 +479,7 @@ public class ZkController implements Closeable, Runnable {
 
             List<CoreDescriptor> descriptors = descriptorsSupplier.get();
             // re register all descriptors
-            try (ParWork parWork = new ParWork(this)) {
+            try (ParWork parWork = new ParWork(this, true, true)) {
               if (descriptors != null) {
                 for (CoreDescriptor descriptor : descriptors) {
                   // TODO: we need to think carefully about what happens when it
@@ -487,7 +489,7 @@ public class ZkController implements Closeable, Runnable {
                   // with connection loss
                   try {
                     // unload solrcores that have been 'failed over'
-                    throwErrorIfReplicaReplaced(descriptor);
+                    // throwErrorIfReplicaReplaced(descriptor);
 
                     parWork.collect(new RegisterCoreAsync(descriptor, true, true));
 
@@ -500,22 +502,21 @@ public class ZkController implements Closeable, Runnable {
 
             // notify any other objects that need to know when the session was re-connected
 
-            try (ParWork parWork = new ParWork(this)) {
+            try (ParWork parWork = new ParWork(this, true, true)) {
               // the OnReconnect operation can be expensive per listener, so do that async in the background
               reconnectListeners.forEach(listener -> {
                 try {
                   parWork.collect(new OnReconnectNotifyAsync(listener));
                 } catch (Exception exc) {
-                  SolrZkClient.checkInterrupted(exc);
                   // not much we can do here other than warn in the log
                   log.warn("Error when notifying OnReconnect listener {} after session re-connected.", listener, exc);
                 }
               });
             }
           } catch (InterruptedException e) {
-            log.warn("interrupted");
+            log.warn("interrupted", e);
           } catch (SessionExpiredException e) {
-            throw e;
+            log.warn("SessionExpiredException", e);
           } catch (AlreadyClosedException e) {
             log.info("Already closed");
             return;
@@ -534,7 +535,7 @@ public class ZkController implements Closeable, Runnable {
       }});
     zkClient.setDisconnectListener(() -> {
       if (isClosed()) return;
-        try (ParWork worker = new ParWork("disconnected", true)) {
+        try (ParWork worker = new ParWork("disconnected", true, true)) {
           if (zkClient.isConnected()) {
             worker.collect(ZkController.this.overseerContexts);
           } else {
@@ -554,6 +555,18 @@ public class ZkController implements Closeable, Runnable {
     init();
   }
 
+  private ElectionContext getOverseerContext() {
+    ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, overseer);
+    ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
+    if (prevContext != null) {
+      prevContext.close();
+    }
+    if (overseerElector != null) {
+      ParWork.close(overseerElector.getContext());
+    }
+    return context;
+  }
+
   /**
    * <p>Verifies if /clusterstate.json exists in Zookeepeer, and if it does and is not empty, refuses to start and outputs
    * a helpful message regarding collection migration.</p>
@@ -614,7 +627,7 @@ public class ZkController implements Closeable, Runnable {
   public void disconnect() {
     log.info("disconnect");
     this.dcCalled = true;
-    try (ParWork closer = new ParWork(this, true)) {
+    try (ParWork closer = new ParWork(this, true, true)) {
       closer.collect( "replicateFromLeaders", replicateFromLeaders);
 
       if (getZkClient().getConnectionManager().isConnected()) {
@@ -647,36 +660,41 @@ public class ZkController implements Closeable, Runnable {
    */
   public void close() {
     log.info("Closing ZkController");
-    assert closeTracker.close();
+    //assert closeTracker.close();
+
+    IOUtils.closeQuietly(overseerElector);
+
+    leaderElectors.forEach((s, leaderElector) -> IOUtils.closeQuietly(leaderElector));
+
+    if (overseer != null) {
+      overseer.closeAndDone();
+    }
+
     this.shudownCalled = true;
 
     this.isClosed = true;
 
     try (ParWork closer = new ParWork(this, true, true)) {
+      closer.collect(overseer);
       closer.collect(replicateFromLeaders);
       closer.collect(electionContexts);
       closer.collect(collectionToTerms);
       closer.collect(sysPropsCacher);
       closer.collect(cloudManager);
       closer.collect(cloudSolrClient);
+      closer.collect(overseerContexts);
     }
 
-    try {
-      if (overseer != null) {
-        overseer.closeAndDone();
-      }
-      ParWork.close(overseerContexts);
-    } finally {
-      IOUtils.closeQuietly(zkStateReader);
+    IOUtils.closeQuietly(zkStateReader);
 
-      if (closeZkClient) {
-        IOUtils.closeQuietly(zkClient);
-      }
+    if (closeZkClient) {
+      IOUtils.closeQuietly(zkClient);
+    }
 
-      SolrLifcycleListener.removeShutdown(this);
+    SolrLifcycleListener.removeShutdown(this);
+
+    assert ObjectReleaseTracker.release(this);
 
-      assert ObjectReleaseTracker.release(this);
-    }
   }
 
   /**
@@ -694,7 +712,7 @@ public class ZkController implements Closeable, Runnable {
     if (shard == null) return;
 
     // if this replica is not a leader, it will be put in recovery state by the leader
-    if (shard.getReplica(cd.getCloudDescriptor().getCoreNodeName()) != shard.getLeader()) return;
+    if (shard.getReplica(cd.getName()) != shard.getLeader()) return;
 
     int numActiveReplicas = shard.getReplicas(
         rep -> rep.getState() == Replica.State.ACTIVE
@@ -704,7 +722,7 @@ public class ZkController implements Closeable, Runnable {
 
     // at least the leader still be able to search, we should give up leadership if other replicas can take over
     if (numActiveReplicas >= 2) {
-      String key = cd.getCollectionName() + ":" + cd.getCloudDescriptor().getCoreNodeName();
+      String key = cd.getCollectionName() + ":" + cd.getName();
       //TODO better handling the case when delete replica was failed
       if (replicasMetTragicEvent.putIfAbsent(key, tragicException) == null) {
         log.warn("Leader {} met tragic exception, give up its leadership", key, tragicException);
@@ -714,7 +732,7 @@ public class ZkController implements Closeable, Runnable {
           props.put(Overseer.QUEUE_OPERATION, "deletereplica");
           props.put(COLLECTION_PROP, cd.getCollectionName());
           props.put(SHARD_ID_PROP, shard.getName());
-          props.put(REPLICA_PROP, cd.getCloudDescriptor().getCoreNodeName());
+          props.put(REPLICA_PROP, cd.getName());
           getOverseerCollectionQueue().offer(Utils.toJSON(new ZkNodeProps(props)));
 
           props.clear();
@@ -763,6 +781,7 @@ public class ZkController implements Closeable, Runnable {
       cloudSolrClient = new CloudHttp2SolrClient.Builder(zkStateReader)
           .withHttpClient(cc.getUpdateShardHandler().getTheSharedHttpClient())
           .build();
+      cloudSolrClient.connect();
       cloudManager = new SolrClientCloudManager(
           new ZkDistributedQueueFactory(zkClient),
           cloudSolrClient,
@@ -859,6 +878,10 @@ public class ZkController implements Closeable, Runnable {
     return isClosed || getCoreContainer().isShutDown();
   }
 
+  boolean isShudownCalled() {
+    return shudownCalled;
+  }
+
   /**
    * Create the zknodes necessary for a cluster to operate
    *
@@ -884,7 +907,6 @@ public class ZkController implements Closeable, Runnable {
     paths.put(Overseer.OVERSEER_ELECT + LeaderElector.ELECTION_NODE, null);
 
     paths.put(Overseer.OVERSEER_QUEUE, null);
-    paths.put(Overseer.OVERSEER_QUEUE_WORK, null);
     paths.put(Overseer.OVERSEER_COLLECTION_QUEUE_WORK, null);
     paths.put(Overseer.OVERSEER_COLLECTION_MAP_RUNNING, null);
     paths.put(Overseer.OVERSEER_COLLECTION_MAP_COMPLETED, null);
@@ -1137,18 +1159,15 @@ public class ZkController implements Closeable, Runnable {
         this.overseerConfigSetQueue = overseer.getConfigSetQueue(zkClient);
         this.sysPropsCacher = new NodesSysPropsCacher(getSolrCloudManager().getNodeStateProvider(),
                 getNodeName(), zkStateReader);
-
+        overseerElector = new LeaderElector(this, new ContextKey("overseer", "overseer"), overseerContexts);
         try (ParWork worker = new ParWork(this, false, true)) {
           // start the overseer first as following code may need it's processing
           worker.collect("startOverseer", () -> {
-            LeaderElector overseerElector = new LeaderElector(zkClient, new ContextKey("overseer", "overseer"), overseerContexts);
-            ElectionContext context = new OverseerElectionContext(getNodeName(), zkClient, overseer);
-            ElectionContext prevContext = overseerContexts.put(new ContextKey("overseer", "overseer"), context);
-            if (prevContext != null) {
-              prevContext.close();
-            }
+            ElectionContext context = getOverseerContext();
+            log.info("Overseer setting up context {}", context.leaderProps);
             overseerElector.setup(context);
             try {
+              log.info("Overseer joining election {}", context.leaderProps);
               overseerElector.joinElection(context, false);
             } catch (KeeperException e) {
               throw new SolrException(ErrorCode.SERVER_ERROR, e);
@@ -1160,19 +1179,19 @@ public class ZkController implements Closeable, Runnable {
             }
           });
 
-          worker.collect("publishDownState", () -> {
-            try {
-              Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null);
-              if (stat != null && stat.getNumChildren() > 0) {
-                publishDownStates();
-              }
-            } catch (InterruptedException e) {
-              ParWork.propagateInterrupt(e);
-              throw new SolrException(ErrorCode.SERVER_ERROR, e);
-            } catch (KeeperException e) {
-              throw new SolrException(ErrorCode.SERVER_ERROR, e);
-            }
-          });
+//          worker.collect("publishDownState", () -> {
+//            try {
+//              Stat stat = zkClient.exists(ZkStateReader.LIVE_NODES_ZKNODE, null);
+//              if (stat != null && stat.getNumChildren() > 0) {
+//                publishDownStates();
+//              }
+//            } catch (InterruptedException e) {
+//              ParWork.propagateInterrupt(e);
+//              throw new SolrException(ErrorCode.SERVER_ERROR, e);
+//            } catch (KeeperException e) {
+//              throw new SolrException(ErrorCode.SERVER_ERROR, e);
+//            }
+//          });
         }
         // Do this last to signal we're up.
         createEphemeralLiveNode();
@@ -1193,64 +1212,8 @@ public class ZkController implements Closeable, Runnable {
   private synchronized void shutdown() {
     if (this.shudownCalled) return;
     this.shudownCalled = true;
-    Thread updaterThead = overseer.getUpdaterThread();
-    log.info("Cluster shutdown initiated");
-    if (updaterThead != null && updaterThead.isAlive()) {
-      log.info("We are the Overseer, wait for others to shutdown");
-
-      CountDownLatch latch = new CountDownLatch(1);
-      List<String> children = null;
-      try {
-        children = zkClient.getChildren("/solr" + ZkStateReader.LIVE_NODES_ZKNODE, new Watcher() {
-          @Override
-          public void process(WatchedEvent event) {
-            if (Event.EventType.None.equals(event.getType())) {
-              return;
-            }
-            if (event.getType() == Event.EventType.NodeChildrenChanged) {
-              Thread updaterThead = overseer.getUpdaterThread();
-              if (updaterThead == null || !updaterThead.isAlive()) {
-                log.info("We were the Overseer, but it seems not anymore, shutting down");
-                latch.countDown();
-                return;
-              }
-
-              try {
-                List<String> children = zkClient.getChildren("/solr" + ZkStateReader.LIVE_NODES_ZKNODE, this, false);
-                if (children.size() == 1) {
-                  latch.countDown();
-                }
-              } catch (KeeperException e) {
-                log.error("Exception on proper shutdown", e);
-                return;
-              } catch (InterruptedException e) {
-                ParWork.propagateInterrupt(e);
-                return;
-              }
-            }
-          }
-        }, false);
-      } catch (KeeperException e) {
-        log.error("Time out waiting to see solr live nodes go down " + children.size());
-        return;
-      } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
-        return;
-      }
 
-      if (children.size() > 1) {
-        boolean success = false;
-        try {
-          success = latch.await(10, TimeUnit.SECONDS);
-        } catch (InterruptedException e) {
-          ParWork.propagateInterrupt(e);
-          return;
-        }
-        if (!success) {
-          log.error("Time out waiting to see solr live nodes go down " + children.size());
-        }
-      }
-    }
+    log.info("Cluster shutdown initiated");
 
     URL url = null;
     try {
@@ -1361,33 +1324,17 @@ public class ZkController implements Closeable, Runnable {
     return zkClient.exists(path);
   }
 
-  public void registerUnloadWatcher(String collection, String shardId, String coreNodeName, String name) {
-    zkStateReader.registerDocCollectionWatcher(collection,
-        new UnloadCoreOnDeletedWatcher(coreNodeName, shardId, name));
-  }
-
-  /**
-   * Register shard with ZooKeeper.
-   *
-   * @return the shardId for the SolrCore
-   */
-  public String register(String coreName, final CoreDescriptor desc, boolean skipRecovery) throws Exception {
-    try (SolrCore core = cc.getCore(coreName)) {
-      return register(core, desc, false, false, skipRecovery);
-    }
-  }
-
-  public String register(SolrCore core, final CoreDescriptor desc, boolean skipRecovery) throws Exception {
-    return register(core, desc, false, false, skipRecovery);
+  public void registerUnloadWatcher(String collection, String shardId, String name) {
+    // nocommit - this thing is currently bad
+//    zkStateReader.registerDocCollectionWatcher(collection,
+//        new UnloadCoreOnDeletedWatcher(shardId, name));
   }
 
-
   public String register(String coreName, final CoreDescriptor desc, boolean recoverReloadedCores,
       boolean afterExpiration, boolean skipRecovery) throws Exception {
     try (SolrCore core = cc.getCore(coreName)) {
      return register(core, desc, recoverReloadedCores, afterExpiration, skipRecovery);
     }
-
   }
 
   /**
@@ -1395,56 +1342,63 @@ public class ZkController implements Closeable, Runnable {
    *
    * @return the shardId for the SolrCore
    */
-  public String register(SolrCore core, final CoreDescriptor desc, boolean recoverReloadedCores,
+  private String register(SolrCore core, final CoreDescriptor desc, boolean recoverReloadedCores,
                          boolean afterExpiration, boolean skipRecovery) throws Exception {
     MDCLoggingContext.setCoreDescriptor(cc, desc);
     try {
-      if (isClosed()) {
-        throw new AlreadyClosedException();
-      }
       // pre register has published our down state
       String coreName = core.getName();
       final String baseUrl = getBaseUrl();
       final CloudDescriptor cloudDesc = desc.getCloudDescriptor();
       final String collection = cloudDesc.getCollectionName();
       final String shardId = cloudDesc.getShardId();
-      final String coreZkNodeName = cloudDesc.getCoreNodeName();
-      assert coreZkNodeName != null : "we should have a coreNodeName by now";
-      log.info("Register SolrCore, baseUrl={} collection={}, shard={} coreNodeName={}", baseUrl, collection, shardId, coreZkNodeName);
-
-
-      // check replica's existence in clusterstate first
+      log.info("Register SolrCore, core={} baseUrl={} collection={}, shard={} skipRecovery={}", coreName, baseUrl, collection, shardId, skipRecovery);
+      AtomicReference<DocCollection> coll = new AtomicReference<>();
       try {
-        zkStateReader.waitForState(collection, 30000,
-            TimeUnit.MILLISECONDS, (collectionState) -> getReplicaOrNull(collectionState, shardId, coreZkNodeName) != null);
+        zkStateReader.waitForState(collection, 3, TimeUnit.SECONDS, (l, c) -> { // nocommit timeout
+//          if (isClosed()) {
+//            throw new AlreadyClosedException();
+//          }
+
+          coll.set(c);
+          if (c == null) {
+            return false;
+          }
+          if (c.getReplica(coreName) != null) {
+            return true;
+          }
+          return false;
+        });
       } catch (TimeoutException e) {
-        throw new SolrException(ErrorCode.SERVER_ERROR, "Error registering SolrCore, timeout waiting for replica present in clusterstate");
+        throw new TimeoutException("Timeout waiting to see core " + coreName  + " \n" + coll.get());
       }
-      Replica replica = getReplicaOrNull(zkStateReader.getClusterState().getCollectionOrNull(collection), shardId, coreZkNodeName);
+
+      Replica replica = getReplicaOrNull(zkStateReader.getClusterState().getCollectionOrNull(collection), shardId, coreName);
       if (replica == null) {
         throw new SolrException(ErrorCode.SERVER_ERROR, "Error registering SolrCore, replica is removed from clusterstate");
       }
 
-
-      if (replica.getType() != Type.PULL) {
-        getCollectionTerms(collection).register(cloudDesc.getShardId(), coreZkNodeName);
-      }
+      // nocommit - we need to know what type we are without getting the clusterstate
+      ///  if (replica.getType() != Type.PULL) {
+      log.info("Register terms for replica {}", coreName);
+      getCollectionTerms(collection).register(cloudDesc.getShardId(), coreName);
+      //  }
 
       ZkShardTerms shardTerms = getShardTerms(collection, cloudDesc.getShardId());
 
-      log.info("Register replica - core:{} address:{} collection:{} shard:{}",
-          coreName, baseUrl, collection, shardId);
-
+      log.info("Register replica - core:{} address:{} collection:{} shard:{}", coreName, baseUrl, collection, shardId);
+      //
       try {
         // If we're a preferred leader, insert ourselves at the head of the queue
         boolean joinAtHead = replica.getBool(SliceMutator.PREFERRED_LEADER_PROP, false);
         if (replica.getType() != Type.PULL) {
+          // nocommit review
           joinElection(desc, afterExpiration, joinAtHead);
         } else if (replica.getType() == Type.PULL) {
           if (joinAtHead) {
-            log.warn("Replica {} was designated as preferred leader but it's type is {}, It won't join election", coreZkNodeName, Type.PULL);
+            log.warn("Replica {} was designated as preferred leader but it's type is {}, It won't join election", coreName, Type.PULL);
           }
-          log.debug("Replica {} skipping election because it's type is {}", coreZkNodeName, Type.PULL);
+          log.debug("Replica {} skipping election because it's type is {}", coreName, Type.PULL);
           startReplicationFromLeader(coreName, false);
         }
       } catch (InterruptedException e) {
@@ -1453,24 +1407,18 @@ public class ZkController implements Closeable, Runnable {
       } catch (KeeperException | IOException e) {
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
       }
+      joinElection(desc, afterExpiration, false);
 
-
-      // don't wait if we have closed
-      if (cc.isShutDown()) {
-        throw new AlreadyClosedException();
-      }
-
-//      getZkStateReader().waitForState(collection, 30, TimeUnit.SECONDS, (n,c) -> c != null && c.getLeader(shardId) != null && c.getLeader(shardId).getState().equals(
-//          Replica.State.ACTIVE));
-
-      //  there should be no stale leader state at this point, dont hit zk directly
       log.info("Wait to see leader for {}, {}", collection, shardId);
-      String leaderUrl = zkStateReader.getLeaderUrl(collection, shardId, 60000);
+      Replica leader = zkStateReader.getLeaderRetry(collection, shardId, 10000);
 
       String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
-      log.info("We are {} and leader is {}", ourUrl, leaderUrl);
-      boolean isLeader = leaderUrl.equals(ourUrl);
-      assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!";
+      boolean isLeader = leader.getName() .equals(coreName);
+
+      log.info("We are {} and leader is {} isLeader={}", ourUrl, leader.getCoreUrl(), isLeader);
+
+      log.info("Check if we should recover isLeader={}", isLeader);
+      //assert !(isLeader && replica.getType() == Type.PULL) : "Pull replica became leader!";
 
       try {
 
@@ -1495,7 +1443,7 @@ public class ZkController implements Closeable, Runnable {
           if (slice.getState() != Slice.State.CONSTRUCTION || !isLeader) {
             Future<UpdateLog.RecoveryInfo> recoveryFuture = core.getUpdateHandler().getUpdateLog().recoverFromLog();
             if (recoveryFuture != null) {
-              log.info("Replaying tlog for {} during startup... NOTE: This can take a while.", ourUrl);
+              log.info("Replaying tlog for {} during startup... NOTE: This can take a while.", core);
               recoveryFuture.get(); // NOTE: this could potentially block for
               // minutes or more!
               // TODO: public as recovering in the mean time?
@@ -1507,16 +1455,12 @@ public class ZkController implements Closeable, Runnable {
             }
           }
         }
-        boolean didRecovery
-            = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreZkNodeName, shardId, core, cc, afterExpiration);
+        boolean didRecovery = checkRecovery(recoverReloadedCores, isLeader, skipRecovery, collection, coreName, shardId, core, cc, afterExpiration);
 
         if (!didRecovery) {
           if (isTlogReplicaAndNotLeader) {
             startReplicationFromLeader(coreName, true);
           }
-          if (!isLeader) {
-            publish(desc, Replica.State.ACTIVE, true, false);
-          }
         }
 
         if (replica.getType() != Type.PULL) {
@@ -1529,29 +1473,24 @@ public class ZkController implements Closeable, Runnable {
         getZkStateReader().registerCore(cloudDesc.getCollectionName());
         // the watcher is added to a set so multiple calls of this method will left only one watcher
         // nocommit
-        registerUnloadWatcher(cloudDesc.getCollectionName(), cloudDesc.getShardId(), cloudDesc.getCoreNodeName(), desc.getName());
+        registerUnloadWatcher(cloudDesc.getCollectionName(), cloudDesc.getShardId(), desc.getName());
 
       } catch (Exception e) {
         SolrZkClient.checkInterrupted(e);
         unregister(coreName, desc, false);
         throw e;
       }
-
+      log.info("SolrCore Registered, core{} baseUrl={} collection={}, shard={}", coreName, baseUrl, collection, shardId);
       return shardId;
     } finally {
       MDCLoggingContext.clear();
     }
   }
 
-  private Replica getReplicaOrNull(DocCollection docCollection, String shard, String coreNodeName) {
+  private Replica getReplicaOrNull(DocCollection docCollection, String shard, String coreName) {
     if (docCollection == null) return null;
 
-    Slice slice = docCollection.getSlice(shard);
-    if (slice == null) return null;
-
-    Replica replica = slice.getReplica(coreNodeName);
-    if (replica == null) return null;
-    if (!getNodeName().equals(replica.getNodeName())) return null;
+    Replica replica = docCollection.getReplica(coreName);
 
     return replica;
   }
@@ -1630,7 +1569,7 @@ public class ZkController implements Closeable, Runnable {
    * Get leader props directly from zk nodes.
    * @throws SessionExpiredException on zk session expiration.
    */
-  public ZkCoreNodeProps getLeaderProps(final String collection,
+  public Replica getLeaderProps(final String collection,
                                         final String slice, int timeoutms) throws InterruptedException, SessionExpiredException {
     return getLeaderProps(collection, slice, timeoutms, true);
   }
@@ -1641,7 +1580,7 @@ public class ZkController implements Closeable, Runnable {
    * @return leader props
    * @throws SessionExpiredException on zk session expiration.
    */
-  public ZkCoreNodeProps getLeaderProps(final String collection,
+  public Replica getLeaderProps(final String collection,
                                         final String slice, int timeoutms, boolean failImmediatelyOnExpiration) throws InterruptedException, SessionExpiredException {
     TimeOut timeout = new TimeOut(timeoutms, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
     Exception exp = null;
@@ -1651,7 +1590,8 @@ public class ZkController implements Closeable, Runnable {
 
         byte[] data = zkClient.getData(ZkStateReader.getShardLeadersPath(collection, slice), null, null);
         ZkCoreNodeProps leaderProps = new ZkCoreNodeProps(ZkNodeProps.load(data));
-        return leaderProps;
+        // nocommit - right key for leader name?
+        return new Replica(leaderProps.getNodeProps().getStr("name"), leaderProps.getNodeProps().getProperties(), collection, slice);
 
       } catch (Exception e) {
         SolrZkClient.checkInterrupted(e);
@@ -1663,17 +1603,13 @@ public class ZkController implements Closeable, Runnable {
   }
 
 
-  private void joinElection(CoreDescriptor cd, boolean afterExpiration, boolean joinAtHead)
+  private boolean joinElection(CoreDescriptor cd, boolean afterExpiration, boolean joinAtHead)
       throws InterruptedException, KeeperException, IOException {
-    if (this.isClosed || cc.isShutDown()) {
-      log.warn("cannot join election, closed");
-      return;
-    }
+    log.info("joinElection {}", cd.getName());
     // look for old context - if we find it, cancel it
     String collection = cd.getCloudDescriptor().getCollectionName();
-    final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
 
-    ContextKey contextKey = new ContextKey(collection, coreNodeName);
+    ContextKey contextKey = new ContextKey(collection, cd.getName());
 
     ElectionContext prevContext = electionContexts.get(contextKey);
 
@@ -1688,18 +1624,20 @@ public class ZkController implements Closeable, Runnable {
     props.put(ZkStateReader.BASE_URL_PROP, getBaseUrl());
     props.put(ZkStateReader.CORE_NAME_PROP, cd.getName());
     props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
-    props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
 
-    ZkNodeProps ourProps = new ZkNodeProps(props);
+    Replica replica = new Replica(cd.getName(), props, null, null);
+    LeaderElector leaderElector;
+    synchronized (leaderElectors) {
+      leaderElector = leaderElectors.get(replica.getName());
+      if (leaderElector == null) {
+         leaderElector = new LeaderElector(this, contextKey, electionContexts);
+         leaderElectors.put(replica.getName(), leaderElector);
+      }
+    }
 
-    LeaderElector leaderElector = new LeaderElector(zkClient, contextKey, electionContexts);
     ElectionContext context = new ShardLeaderElectionContext(leaderElector, shardId,
-        collection, coreNodeName, ourProps, this, cc);
+        collection, cd.getName(), replica, this, cc);
 
-    if (this.isClosed || cc.isShutDown()) {
-      context.close();
-      return;
-    }
     prevContext = electionContexts.put(contextKey, context);
     if (prevContext != null) {
       prevContext.close();
@@ -1707,7 +1645,7 @@ public class ZkController implements Closeable, Runnable {
 
     leaderElector.setup(context);
     log.info("Joining election ...");
-    leaderElector.joinElection(context, false, joinAtHead);
+    return leaderElector.joinElection(context, false, joinAtHead);
   }
 
 
@@ -1735,7 +1673,7 @@ public class ZkController implements Closeable, Runnable {
         core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
         return true;
       }
-
+      log.info("get shard terms {} {} {}", core.getName(), collection, shardId);
       ZkShardTerms zkShardTerms = getShardTerms(collection, shardId);
       if (zkShardTerms.registered(coreZkNodeName) && !zkShardTerms.canBecomeLeader(coreZkNodeName)) {
         if (log.isInfoEnabled()) {
@@ -1789,8 +1727,6 @@ public class ZkController implements Closeable, Runnable {
 
       String shardId = cd.getCloudDescriptor().getShardId();
 
-      String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-
       Map<String,Object> props = new HashMap<>();
       props.put(Overseer.QUEUE_OPERATION, "state");
       props.put(ZkStateReader.STATE_PROP, state.toString());
@@ -1801,13 +1737,10 @@ public class ZkController implements Closeable, Runnable {
       props.put(ZkStateReader.SHARD_ID_PROP, cd.getCloudDescriptor().getShardId());
       props.put(ZkStateReader.COLLECTION_PROP, collection);
       props.put(ZkStateReader.REPLICA_TYPE, cd.getCloudDescriptor().getReplicaType().toString());
-      props.put(ZkStateReader.FORCE_SET_STATE_PROP, "false");
+
       if (numShards != null) {
         props.put(ZkStateReader.NUM_SHARDS_PROP, numShards.toString());
       }
-      if (coreNodeName != null) {
-        props.put(ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
-      }
       try (SolrCore core = cc.getCore(cd.getName())) {
         if (core != null && state == Replica.State.ACTIVE) {
           ensureRegisteredSearcher(core);
@@ -1833,10 +1766,10 @@ public class ZkController implements Closeable, Runnable {
       if (state == Replica.State.RECOVERING && cd.getCloudDescriptor().getReplicaType() != Type.PULL) {
         // state is used by client, state of replica can change from RECOVERING to DOWN without needed to finish recovery
         // by calling this we will know that a replica actually finished recovery or not
-        getShardTerms(collection, shardId).startRecovering(coreNodeName);
+        getShardTerms(collection, shardId).startRecovering(cd.getName());
       }
       if (state == Replica.State.ACTIVE && cd.getCloudDescriptor().getReplicaType() != Type.PULL) {
-        getShardTerms(collection, shardId).doneRecovering(coreNodeName);
+        getShardTerms(collection, shardId).doneRecovering(cd.getName());
       }
 
       ZkNodeProps m = new ZkNodeProps(props);
@@ -1879,8 +1812,8 @@ public class ZkController implements Closeable, Runnable {
   }
 
   public void unregister(String coreName, CoreDescriptor cd, boolean removeCoreFromZk) throws Exception {
+    log.info("Unregister core from zookeeper {}", coreName);
     if (!zkClient.isConnected()) return;
-    final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
     final String collection = cd.getCloudDescriptor().getCollectionName();
 
     ZkCollectionTerms ct = collectionToTerms.get(collection);
@@ -1890,7 +1823,7 @@ public class ZkController implements Closeable, Runnable {
 
     zkStateReader.unregisterCore(collection);
 
-    replicasMetTragicEvent.remove(collection+":"+coreNodeName);
+    replicasMetTragicEvent.remove(collection + ":" + coreName);
 
     if (Strings.isNullOrEmpty(collection)) {
       log.error("No collection was specified.");
@@ -1898,252 +1831,21 @@ public class ZkController implements Closeable, Runnable {
       return;
     }
     final DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collection);
-    Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreNodeName);
+    Replica replica = (docCollection == null) ? null : docCollection.getReplica(coreName);
 
     if (replica == null || replica.getType() != Type.PULL) {
-      ElectionContext context = electionContexts.remove(new ContextKey(collection, coreNodeName));
+      ElectionContext context = electionContexts.remove(new ContextKey(collection, coreName));
 
       if (context != null) {
         context.close();
       }
     }
-    CloudDescriptor cloudDescriptor = cd.getCloudDescriptor();
-    if (removeCoreFromZk) {
-      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
-          OverseerAction.DELETECORE.toLower(), ZkStateReader.CORE_NAME_PROP, coreName,
-          ZkStateReader.NODE_NAME_PROP, getNodeName(),
-          ZkStateReader.COLLECTION_PROP, cloudDescriptor.getCollectionName(),
-          ZkStateReader.BASE_URL_PROP, getBaseUrl(),
-          ZkStateReader.CORE_NODE_NAME_PROP, coreNodeName);
-      overseerJobQueue.offer(Utils.toJSON(m));
-
-//      zkStateReader.waitForState(cloudDescriptor.getCollectionName(), 10, TimeUnit.SECONDS, (l,c) -> {
-//        if (c == null) return true;
-//        Slice slice = c.getSlice(cloudDescriptor.getShardId());
-//        if (slice == null) return true;
-//        Replica r = slice.getReplica(cloudDescriptor.getCoreNodeName());
-//        if (r == null) return true;
-//        return false;
-//      });
-    }
-  }
-
-  public void createCollection(String collection) throws Exception {
-    ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION,
-        CollectionParams.CollectionAction.CREATE.toLower(), ZkStateReader.NODE_NAME_PROP, getNodeName(),
-        ZkStateReader.COLLECTION_PROP, collection);
-    overseerJobQueue.offer(Utils.toJSON(m));
   }
 
   public ZkStateReader getZkStateReader() {
     return zkStateReader;
   }
 
-  private void doGetShardIdAndNodeNameProcess(CoreDescriptor cd) {
-    final String coreNodeName = cd.getCloudDescriptor().getCoreNodeName();
-
-    if (coreNodeName != null) {
-      waitForShardId(cd);
-    } else {
-      // if no explicit coreNodeName, we want to match by base url and core name
-      waitForCoreNodeName(cd);
-      waitForShardId(cd);
-    }
-  }
-
-  private void waitForCoreNodeName(CoreDescriptor cd) {
-    if (log.isDebugEnabled()) log.debug("look for our core node name");
-
-    AtomicReference<String> errorMessage = new AtomicReference<>();
-    try {
-      zkStateReader.waitForState(cd.getCollectionName(), 10, TimeUnit.SECONDS, (n, c) -> { // TODO: central timeout control
-        if (c == null)
-          return false;
-        final Map<String,Slice> slicesMap = c.getSlicesMap();
-        if (slicesMap == null) {
-          return false;
-        }
-        for (Slice slice : slicesMap.values()) {
-          for (Replica replica : slice.getReplicas()) {
-
-            String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
-            String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-
-            String msgNodeName = getNodeName();
-            String msgCore = cd.getName();
-
-            if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
-              cd.getCloudDescriptor()
-                      .setCoreNodeName(replica.getName());
-              return true;
-            }
-          }
-        }
-        return false;
-      });
-    } catch (TimeoutException | InterruptedException e) {
-      String error = errorMessage.get();
-      if (error == null)
-        error = "";
-      throw new NotInClusterStateException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName() + " " + error);
-    }
-  }
-
-  private void waitForShardId(CoreDescriptor cd) {
-    if (log.isDebugEnabled()) {
-      log.debug("waitForShardId(CoreDescriptor cd={}) - start", cd);
-    }
-
-    AtomicReference<String> returnId = new AtomicReference<>();
-    try {
-      try {
-        zkStateReader.waitForState(cd.getCollectionName(), 5, TimeUnit.SECONDS, (n, c) -> { // nocommit
-          if (c == null) return false;
-          String shardId = c.getShardId(cd.getCloudDescriptor().getCoreNodeName());
-          if (shardId != null) {
-            returnId.set(shardId);
-            return true;
-          }
-          return false;
-        });
-      } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
-        throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
-      }
-    } catch (TimeoutException e1) {
-      log.error("waitForShardId(CoreDescriptor=" + cd + ")", e1);
-
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
-    }
-
-    final String shardId = returnId.get();
-    if (shardId != null) {
-      cd.getCloudDescriptor().setShardId(shardId);
-
-      if (log.isDebugEnabled()) {
-        log.debug("waitForShardId(CoreDescriptor) - end coreNodeName=" + cd.getCloudDescriptor().getCoreNodeName() + " shardId=" + shardId);
-      }
-      return;
-    }
-
-    throw new SolrException(ErrorCode.SERVER_ERROR, "Could not get shard id for core: " + cd.getName());
-  }
-
-  public String getCoreNodeName(CoreDescriptor descriptor) {
-    String coreNodeName = descriptor.getCloudDescriptor().getCoreNodeName();
-    if (coreNodeName == null && !genericCoreNodeNames) {
-      // it's the default
-      return getNodeName() + "_" + descriptor.getName();
-    }
-
-    return coreNodeName;
-  }
-
-  public void preRegister(CoreDescriptor cd) {
-    log.info("PreRegister SolrCore, collection={}, shard={} coreNodeName={}", cd.getCloudDescriptor().getCollectionName(), cd.getCloudDescriptor().getShardId());
-
-    CloudDescriptor cloudDesc = cd.getCloudDescriptor();
-
-    String coreNodeName = getCoreNodeName(cd);
-
-    // before becoming available, make sure we are not live and active
-    // this also gets us our assigned shard id if it was not specified
-    try {
-      checkStateInZk(cd);
-
-      // make sure the node name is set on the descriptor
-      if (cloudDesc.getCoreNodeName() == null) {
-        cloudDesc.setCoreNodeName(coreNodeName);
-      }
-      log.info("PreRegister found coreNodename of {}", coreNodeName);
-
-      // publishState == false on startup
-      if (isPublishAsDownOnStartup(cloudDesc)) {
-        publish(cd, Replica.State.DOWN, false, true);
-      }
-      String collectionName = cd.getCloudDescriptor().getCollectionName();
-      DocCollection collection = zkStateReader.getClusterState().getCollectionOrNull(collectionName);
-      if (log.isDebugEnabled()) {
-        log.debug(collection == null ?
-                "Collection {} not visible yet, but flagging it so a watch is registered when it becomes visible" :
-                "Registering watch for collection {}",
-            collectionName);
-      }
-    } catch (KeeperException e) {
-      log.error("", e);
-      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
-    } catch (NotInClusterStateException e) {
-      // make the stack trace less verbose
-      throw e;
-    } catch (Exception e) {
-      ParWork.propagateInterrupt(e);
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "", e);
-    }
-
-    doGetShardIdAndNodeNameProcess(cd);
-
-  }
-
-  /**
-   * On startup, the node already published all of its replicas as DOWN,
-   * we can skip publish the replica as down
-   * @return Should publish the replica as down on startup
-   */
-  private boolean isPublishAsDownOnStartup(CloudDescriptor cloudDesc) {
-      Replica replica = zkStateReader.getClusterState().getCollection(cloudDesc.getCollectionName())
-          .getSlice(cloudDesc.getShardId())
-          .getReplica(cloudDesc.getCoreNodeName());
-      return !replica.getNodeName().equals(getNodeName());
-  }
-
-  private void checkStateInZk(CoreDescriptor cd) throws InterruptedException, NotInClusterStateException {
-    CloudDescriptor cloudDesc = cd.getCloudDescriptor();
-    String nodeName = cloudDesc.getCoreNodeName();
-    if (nodeName == null) {
-      nodeName = cloudDesc.getCoreNodeName();
-      // verify that the repair worked.
-      if (nodeName == null) {
-        throw new SolrException(ErrorCode.SERVER_ERROR, "No coreNodeName for " + cd);
-      }
-    }
-    final String coreNodeName = nodeName;
-
-    if (cloudDesc.getShardId() == null) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "No shard id for " + cd);
-    }
-    StringBuilder sb = new StringBuilder(256);
-    AtomicReference<String> errorMessage = new AtomicReference<>();
-    AtomicReference<DocCollection> collectionState = new AtomicReference<>();
-    try {
-      zkStateReader.waitForState(cd.getCollectionName(), WAIT_FOR_STATE, TimeUnit.SECONDS, (l, c) -> {
-        collectionState.set(c);
-        if (c == null) return false;
-        Slice slice = c.getSlice(cloudDesc.getShardId());
-        if (slice == null) {
-          errorMessage.set("Invalid shard: " + cloudDesc.getShardId());
-          return false;
-        }
-        Replica replica = slice.getReplica(coreNodeName);
-        if (replica == null) {
-          sb.setLength(0);
-          slice.getReplicas().stream().forEach(replica1 -> sb.append(replica1.getName() + " "));
-          errorMessage.set("coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId() + ", ignore the exception if the replica was deleted. Found: " + sb.toString());
-          return false;
-        }
-        return true;
-      });
-    } catch (TimeoutException e) {
-      String error = errorMessage.get();
-      if (error == null) error = "coreNodeName " + coreNodeName + " does not exist in shard " + cloudDesc.getShardId() + ", ignore the exception if the replica was deleted";
-
-      throw new NotInClusterStateException(ErrorCode.SERVER_ERROR, error + "\n" + collectionState.get());
-    }
-
-  }
-
   public static void linkConfSet(SolrZkClient zkClient, String collection, String confSetName) throws KeeperException, InterruptedException {
     String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection;
     log.debug("Load collection config from:{}", path);
@@ -2299,6 +2001,14 @@ public class ZkController implements Closeable, Runnable {
     }
   }
 
+  public static String generateNodeName(final String url) {
+    try {
+      return URLEncoder.encode(trimLeadingAndTrailingSlashes(url), "UTF-8");
+    } catch (UnsupportedEncodingException e) {
+      throw new Error("JVM Does not seem to support UTF-8", e);
+    }
+  }
+
   /**
    * Utility method for trimming and leading and/or trailing slashes from
    * its input.  May return the empty string.  May return null if and only
@@ -2317,11 +2027,19 @@ public class ZkController implements Closeable, Runnable {
     return out;
   }
 
-  public void rejoinOverseerElection(String electionNode, boolean joinAtHead) {
-    if (overseerElector == null) {
+  public void rejoinOverseerElection(boolean joinAtHead) {
+    boolean closeAndDone = false;
+    try {
+      closeAndDone = overseer.isCloseAndDone();
+    } catch (NullPointerException e) {
+      // okay
+    }
+
+    if (overseerElector == null || isClosed() || shudownCalled || closeAndDone) {
       return;
     }
     try {
+      String electionNode = overseerElector.getContext().electionPath;
       if (electionNode != null) {
         // Check whether we came to this node by mistake
         if ( overseerElector.getContext() != null && overseerElector.getContext().leaderSeqPath == null 
@@ -2360,7 +2078,6 @@ public class ZkController implements Closeable, Runnable {
 
     String collectionName = params.get(COLLECTION_PROP);
     String shardId = params.get(SHARD_ID_PROP);
-    String coreNodeName = params.get(CORE_NODE_NAME_PROP);
     String coreName = params.get(CORE_NAME_PROP);
     String electionNode = params.get(ELECTION_NODE_PROP);
     String baseUrl = params.get(BASE_URL_PROP);
@@ -2370,16 +2087,21 @@ public class ZkController implements Closeable, Runnable {
 
       log.info("Rejoin the shard leader election.");
 
-      ContextKey contextKey = new ContextKey(collectionName, coreNodeName);
+      ContextKey contextKey = new ContextKey(collectionName, coreName);
 
       ElectionContext prevContext = electionContexts.get(contextKey);
       if (prevContext != null) prevContext.close();
 
-      ZkNodeProps zkProps = new ZkNodeProps(BASE_URL_PROP, baseUrl, CORE_NAME_PROP, coreName, NODE_NAME_PROP, getNodeName(), CORE_NODE_NAME_PROP, coreNodeName);
+      Map<String, Object> props = new HashMap<>();
+      props.put(ZkStateReader.BASE_URL_PROP, baseUrl);
+      props.put(ZkStateReader.CORE_NAME_PROP, coreName);
+      props.put(ZkStateReader.NODE_NAME_PROP, getNodeName());
+
+      Replica replica = new Replica(coreName, props, null, null);
 
       LeaderElector elect = ((ShardLeaderElectionContext) prevContext).getLeaderElector();
       ShardLeaderElectionContext context = new ShardLeaderElectionContext(elect, shardId, collectionName,
-          coreNodeName, zkProps, this, getCoreContainer());
+          coreName, replica, this, getCoreContainer());
 
       context.leaderSeqPath = context.electionPath + LeaderElector.ELECTION_NODE + "/" + electionNode;
       elect.setup(context);
@@ -2705,12 +2427,10 @@ public class ZkController implements Closeable, Runnable {
 
   /** @lucene.internal */
   public class UnloadCoreOnDeletedWatcher implements DocCollectionWatcher {
-    String coreNodeName;
     String shard;
     String coreName;
 
-    public UnloadCoreOnDeletedWatcher(String coreNodeName, String shard, String coreName) {
-      this.coreNodeName = coreNodeName;
+    public UnloadCoreOnDeletedWatcher(String shard, String coreName) {
       this.shard = shard;
       this.coreName = coreName;
     }
@@ -2724,7 +2444,7 @@ public class ZkController implements Closeable, Runnable {
 
       if (getCoreContainer().getCoreDescriptor(coreName) == null) return true;
 
-      boolean replicaRemoved = getReplicaOrNull(collectionState, shard, coreNodeName) == null;
+      boolean replicaRemoved = getReplicaOrNull(collectionState, shard, coreName) == null;
       if (replicaRemoved) {
         try {
           log.info("Replica {} removed from clusterstate, remove it.", coreName);
@@ -2747,7 +2467,7 @@ public class ZkController implements Closeable, Runnable {
       if (this == o) return true;
       if (o == null || getClass() != o.getClass()) return false;
       UnloadCoreOnDeletedWatcher that = (UnloadCoreOnDeletedWatcher) o;
-      return Objects.equals(coreNodeName, that.coreNodeName) &&
+      return
           Objects.equals(shard, that.shard) &&
           Objects.equals(coreName, that.coreName);
     }
@@ -2755,7 +2475,7 @@ public class ZkController implements Closeable, Runnable {
     @Override
     public int hashCode() {
 
-      return Objects.hash(coreNodeName, shard, coreName);
+      return Objects.hash(shard, coreName);
     }
   }
 
@@ -2775,7 +2495,7 @@ public class ZkController implements Closeable, Runnable {
 
       for (Slice slice : slices) {
         Collection<Replica> replicas = slice.getReplicas();
-        Replica r = slice.getReplica(dcore.getCloudDescriptor().getCoreNodeName());
+        Replica r = slice.getReplica(dcore.getName());
         if (r != null) {
           return true;
         }
@@ -2802,7 +2522,7 @@ public class ZkController implements Closeable, Runnable {
         ZkStateReader.NODE_NAME_PROP, nodeName);
     try {
       overseer.getStateUpdateQueue().offer(Utils.toJSON(m));
-    } catch (AlreadyClosedException e) {
+    } catch (AlreadyClosedException | InterruptedException e) {
       ParWork.propagateInterrupt("Not publishing node as DOWN because a resource required to do so is already closed.", null, true);
       return;
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
index a413df8..c53b2f4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkDistributedQueue.java
@@ -16,7 +16,6 @@
  */
 package org.apache.solr.cloud;
 
-import java.io.File;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -27,7 +26,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Set;
-import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -37,15 +35,18 @@ import java.util.function.Predicate;
 
 import com.codahale.metrics.Timer;
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import org.apache.solr.client.solrj.cloud.DistributedQueue;
-import org.apache.solr.common.AlreadyClosedException;
-import org.apache.solr.common.ParWork;
+import org.apache.solr.cloud.overseer.NodeMutator;
+import org.apache.solr.cloud.overseer.ReplicaMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
+import org.apache.solr.common.cloud.ConnectionManager.IsClosed;
 import org.apache.solr.common.util.Pair;
-import org.apache.solr.common.util.TimeOut;
-import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.Op;
@@ -82,6 +83,7 @@ public class ZkDistributedQueue implements DistributedQueue {
    * We only bother setting a child watcher when the queue has no children in ZK.
    */
   private static final Object _IMPLEMENTATION_NOTES = null;
+  public static final byte[] DATA = new byte[0];
 
   final String dir;
 
@@ -92,22 +94,26 @@ public class ZkDistributedQueue implements DistributedQueue {
   /**
    * A lock that guards all of the mutable state that follows.
    */
-  protected final ReentrantLock updateLock = new ReentrantLock();
+  private final ReentrantLock updateLock = new ReentrantLock();
 
   /**
    * Contains the last set of children fetched from ZK. Elements are removed from the head of
    * this in-memory set as they are consumed from the queue.  Due to the distributed nature
    * of the queue, elements may appear in this set whose underlying nodes have been consumed in ZK.
-   * Therefore, methods like peek have to double-check actual node existence, and methods
-   * like poll must resolve any races by attempting to delete the underlying node.
+   * Therefore, methods like {@link #peek()} have to double-check actual node existence, and methods
+   * like {@link #poll()} must resolve any races by attempting to delete the underlying node.
    */
-  protected volatile TreeMap<String,byte[]> knownChildren;
+  private TreeSet<String> knownChildren = new TreeSet<>();
 
   /**
    * Used to wait on ZK changes to the child list; you must hold {@link #updateLock} before waiting on this condition.
    */
   private final Condition changed = updateLock.newCondition();
 
+  private boolean isDirty = true;
+
+  private int watcherCount = 0;
+
   private final int maxQueueSize;
 
   /**
@@ -133,22 +139,6 @@ public class ZkDistributedQueue implements DistributedQueue {
     this.zookeeper = zookeeper;
     this.stats = stats;
     this.maxQueueSize = maxQueueSize;
-
-    try {
-      try {
-        updateLock.lockInterruptibly();
-        fetchZkChildren(null, null);
-      } catch (KeeperException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      }
-    } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
-      }
-    }
   }
 
   /**
@@ -158,10 +148,10 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return data at the first element of the queue, or null.
    */
   @Override
-  public byte[] peek(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+  public byte[] peek() throws KeeperException, InterruptedException {
     Timer.Context time = stats.time(dir + "_peek");
     try {
-      return firstElement(acceptFilter);
+      return firstElement();
     } finally {
       time.stop();
     }
@@ -175,8 +165,8 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return data at the first element of the queue, or null.
    */
   @Override
-  public byte[] peek(Predicate<String> acceptFilter, boolean block) throws KeeperException, InterruptedException {
-    return block ? peek(acceptFilter, Long.MAX_VALUE) : peek(acceptFilter);
+  public byte[] peek(boolean block) throws KeeperException, InterruptedException {
+    return block ? peek(Long.MAX_VALUE) : peek();
   }
 
   /**
@@ -187,40 +177,29 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return data at the first element of the queue, or null.
    */
   @Override
-  public byte[] peek(Predicate<String> acceptFilter, long wait) throws KeeperException, InterruptedException {
-    byte[] result = null;
-
+  public byte[] peek(long wait) throws KeeperException, InterruptedException {
+    Preconditions.checkArgument(wait > 0);
     Timer.Context time;
     if (wait == Long.MAX_VALUE) {
       time = stats.time(dir + "_peek_wait_forever");
     } else {
       time = stats.time(dir + "_peek_wait" + wait);
     }
-
-    long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
-
-    result = firstElement(acceptFilter);
-    if (result != null) {
-      return result;
-    }
-
-    ChildWatcher watcher = new ChildWatcher(acceptFilter);
-    TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, null);
-
-    if (foundChildren.size() > 0) {
-      result = firstElement(acceptFilter);
-      return result;
-    }
-
-    TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
-
-    waitForChildren(null, foundChildren, timeout, watcher);
-    if (foundChildren.size() == 0) {
+    updateLock.lockInterruptibly();
+    try {
+      long waitNanos = TimeUnit.MILLISECONDS.toNanos(wait);
+      while (waitNanos > 0) {
+        byte[] result = firstElement();
+        if (result != null) {
+          return result;
+        }
+        waitNanos = changed.awaitNanos(waitNanos);
+      }
       return null;
+    } finally {
+      updateLock.unlock();
+      time.stop();
     }
-    result = firstElement(acceptFilter);
-    return result;
-
   }
 
   /**
@@ -230,10 +209,10 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return Head of the queue or null.
    */
   @Override
-  public byte[] poll(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+  public byte[] poll() throws KeeperException, InterruptedException {
     Timer.Context time = stats.time(dir + "_poll");
     try {
-      return removeFirst(acceptFilter);
+      return removeFirst();
     } finally {
       time.stop();
     }
@@ -245,10 +224,10 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return The former head of the queue
    */
   @Override
-  public byte[] remove(Predicate<String> acceptFilter) throws NoSuchElementException, KeeperException{
+  public byte[] remove() throws NoSuchElementException, KeeperException, InterruptedException {
     Timer.Context time = stats.time(dir + "_remove");
     try {
-      byte[] result = removeFirst(acceptFilter);
+      byte[] result = removeFirst();
       if (result == null) {
         throw new NoSuchElementException();
       }
@@ -258,32 +237,48 @@ public class ZkDistributedQueue implements DistributedQueue {
     }
   }
 
-  // TODO: use async
+//  public void remove(Collection<String> paths) throws KeeperException, InterruptedException {
+//
+//    if (log.isDebugEnabled()) log.debug("Remove paths from queue dir={} paths={}", dir, paths);
+//
+//    if (paths.isEmpty()) {
+//      if (log.isDebugEnabled()) log.debug("paths is empty, return");
+//      return;
+//    }
+//
+//    List<String> fullPaths = new ArrayList<>(paths.size());
+//    for (String node : paths) {
+//      fullPaths.add(dir + "/" + node);
+//    }
+//
+//
+//    if (log.isDebugEnabled()) log.debug("delete nodes {}", fullPaths);
+//    zookeeper.delete(fullPaths, false);
+//    if (log.isDebugEnabled()) log.debug("after delete nodes");
+//
+//    int cacheSizeBefore = knownChildren.size();
+//    knownChildren.removeAll(paths);
+//    if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+//      stats.setQueueLength(knownChildren.size());
+//    } else {
+//      // There are elements get deleted but not present in the cache,
+//      // the cache seems not valid anymore
+//      knownChildren.clear();
+//      isDirty = true;
+//    }
+//  }
+
   public void remove(Collection<String> paths) throws KeeperException, InterruptedException {
-    if (log.isDebugEnabled()) log.debug("Remove paths from queue {} {}", dir, paths);
     if (paths.isEmpty()) return;
-
-    updateLock.lockInterruptibly();
-    try {
-      for (String path : paths) {
-        knownChildren.remove(path);
-      }
-    } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
-      }
-    }
-
     List<Op> ops = new ArrayList<>();
     for (String path : paths) {
       ops.add(Op.delete(dir + "/" + path, -1));
     }
     for (int from = 0; from < ops.size(); from += 1000) {
       int to = Math.min(from + 1000, ops.size());
-      List<Op> opList = ops.subList(from, to);
-      if (opList.size() > 0) {
+      if (from < to) {
         try {
-          zookeeper.multi(ops.subList(from, to));
+          zookeeper.multi(ops.subList(from, to), true);
         } catch (KeeperException.NoNodeException e) {
           // don't know which nodes are not exist, so try to delete one by one node
           for (int j = from; j < to; j++) {
@@ -298,6 +293,17 @@ public class ZkDistributedQueue implements DistributedQueue {
         }
       }
     }
+
+    int cacheSizeBefore = knownChildren.size();
+    knownChildren.removeAll(paths);
+    if (cacheSizeBefore - paths.size() == knownChildren.size() && knownChildren.size() != 0) {
+      stats.setQueueLength(knownChildren.size());
+    } else {
+      // There are elements get deleted but not present in the cache,
+      // the cache seems not valid anymore
+      knownChildren.clear();
+      isDirty = true;
+    }
   }
 
   /**
@@ -306,39 +312,30 @@ public class ZkDistributedQueue implements DistributedQueue {
    * @return The former head of the queue
    */
   @Override
-  public byte[] take(Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
+  public byte[] take() throws KeeperException, InterruptedException {
     // Same as for element. Should refactor this.
     Timer.Context timer = stats.time(dir + "_take");
     updateLock.lockInterruptibly();
     try {
-      long waitNanos = TimeUnit.MILLISECONDS.toNanos(60000);
-
-      byte[] result = removeFirst(acceptFilter);
-      if (result != null) {
-        return result;
-      }
-
-      ChildWatcher watcher = new ChildWatcher(acceptFilter);
-      TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, acceptFilter);
-
-      TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
-
-      waitForChildren( s -> s.startsWith(PREFIX) || acceptFilter.test(s), foundChildren, timeout, watcher);
-      if (foundChildren.size() == 0) {
-        return null;
-      }
-
-      result = removeFirst(acceptFilter);
-      if (result != null) {
-        return result;
+      while (true) {
+        byte[] result = removeFirst();
+        if (result != null) {
+          return result;
+        }
+        changed.await();
       }
     } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
-      }
+      updateLock.unlock();
       timer.stop();
     }
-    return null;
+  }
+
+  private static Set<String> OPERATIONS = new HashSet<>();
+  static {
+    OPERATIONS.add("state");
+    OPERATIONS.add("leader");
+    OPERATIONS.add("downnode");
+    OPERATIONS.add("updateshardstate");
   }
 
   /**
@@ -346,40 +343,50 @@ public class ZkDistributedQueue implements DistributedQueue {
    * will be immediately visible when this method returns.
    */
   @Override
-  public void offer(byte[] data) throws KeeperException {
+  public void offer(byte[] data) throws KeeperException, InterruptedException {
+    // TODO change to accept json
+    Map json = (Map) Utils.fromJSON(data);
+
+    final String operation = (String) json.get(Overseer.QUEUE_OPERATION);
+//    if (!OPERATIONS.contains(operation)) {
+//      throw new IllegalArgumentException("unknown operation:" + operation + " contents:" + json);
+//    }
     Timer.Context time = stats.time(dir + "_offer");
-    if (log.isDebugEnabled()) log.debug("Over item to queue {}", dir);
+
     try {
-      try {
-        if (maxQueueSize > 0) {
-          if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) {
-            // If a max queue size is set, check it before creating a new queue item.
-            Stat stat = zookeeper.exists(dir, null, true);
-            if (stat == null) {
-              // jump to the code below, which tries to create dir if it doesn't exist
-              throw new KeeperException.NoNodeException();
-            }
-            int remainingCapacity = maxQueueSize - stat.getNumChildren();
-            if (remainingCapacity <= 0) {
-              throw new IllegalStateException("queue is full");
+      while (true) {
+        try {
+          if (maxQueueSize > 0) {
+            if (offerPermits.get() <= 0 || offerPermits.getAndDecrement() <= 0) { // nocommit review
+              // If a max queue size is set, check it before creating a new queue item.
+              Stat stat = zookeeper.exists(dir, null, true);
+              if (stat == null) {
+                // jump to the code below, which tries to create dir if it doesn't exist
+                throw new KeeperException.NoNodeException();
+              }
+              int remainingCapacity = maxQueueSize - stat.getNumChildren();
+              if (remainingCapacity <= 0) {
+                throw new IllegalStateException("queue is full");
+              }
+
+              // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
+              offerPermits.set(remainingCapacity / 100);
             }
+          }
 
-            // Allow this client to push up to 1% of the remaining queue capacity without rechecking.
-            offerPermits.set(remainingCapacity / 100);
+          // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
+          // This will get set again when the watcher actually fires, but that's ok.
+          zookeeper.create(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL, true);
+          isDirty = true;
+          return;
+        } catch (KeeperException.NoNodeException e) {
+          try {
+            zookeeper.create(dir, DATA, CreateMode.PERSISTENT, true);
+          } catch (KeeperException.NodeExistsException ne) {
+            // someone created it
           }
         }
-
-        // Explicitly set isDirty here so that synchronous same-thread calls behave as expected.
-        // This will get set again when the watcher actually fires, but that's ok.
-        zookeeper.create(dir + "/" + PREFIX, data, CreateMode.PERSISTENT_SEQUENTIAL, true);
-        return;
-      } catch (KeeperException.NoNodeException e) {
-        // someone created it
-      } catch (InterruptedException e) {
-        ParWork.propagateInterrupt(e);
-        throw new AlreadyClosedException(e);
       }
-
     } finally {
       time.stop();
     }
@@ -416,132 +423,72 @@ public class ZkDistributedQueue implements DistributedQueue {
 
   /**
    * Returns the name if the first known child node, or {@code null} if the queue is empty.
-   * @return
+   * This is the only place {@link #knownChildren} is ever updated!
+   * The caller must double check that the actual node still exists, since the in-memory
+   * list is inherently stale.
    */
-  private Map.Entry<String,byte[]> firstChild(boolean remove, Predicate<String> acceptFilter) {
+  private String firstChild(boolean remove, boolean refetchIfDirty) throws KeeperException, InterruptedException {
+    updateLock.lockInterruptibly();
     try {
-      updateLock.lockInterruptibly();
-      try {
-        // We always return from cache first, the cache will be cleared if the node is not exist
-        if (!knownChildren.isEmpty()) {
-          for (Map.Entry<String,byte[]> entry : knownChildren.entrySet()) {
-            if (acceptFilter != null && acceptFilter.test(entry.getKey())) {
-              continue;
-            }
-            if (remove) {
-              knownChildren.remove(entry.getKey());
-              try {
-                zookeeper.delete(dir + "/" + entry.getKey(), -1);
-              } catch (KeeperException.NoNodeException e) {
-                if (log.isDebugEnabled()) log.debug("No node found for {}", entry.getKey());
-              }
-              return entry;
-            }
-          }
-          return null;
-        }
-      } finally {
-        if (updateLock.isHeldByCurrentThread()) {
-          updateLock.unlock();
-        }
+     if (log.isDebugEnabled()) log.debug("firstChild isDirty={}, refetchIfDirty={}, remove={}", isDirty, refetchIfDirty, remove);
+      // We always return from cache first, the cache will be cleared if the node is not exist
+      if (!knownChildren.isEmpty() && !(isDirty && refetchIfDirty)) {
+        if (log.isDebugEnabled()) log.debug("returning first child");
+        return remove ? knownChildren.pollFirst() : knownChildren.first();
       }
 
-      fetchZkChildren(null, acceptFilter);
+      if (!isDirty && knownChildren.isEmpty()) {
+        if (log.isDebugEnabled()) log.debug("isDirty={} knownChildren is empty, returning null", isDirty);
+        return null;
+      }
 
-      return null;
-    } catch (InterruptedException | KeeperException e) {
-      ParWork.propagateInterrupt(e);
-      throw new AlreadyClosedException(e);
+      // Dirty, try to fetch an updated list of children from ZK.
+      // Only set a new watcher if there isn't already a watcher.
+
+      if (log.isDebugEnabled()) log.debug("isDirty={} fetch known children", isDirty);
+      ChildWatcher newWatcher = (watcherCount == 0) ? new ChildWatcher() : null;
+      knownChildren = fetchZkChildren(newWatcher);
+      if (newWatcher != null) {
+        watcherCount++; // watcher was successfully set
+      }
+      isDirty = false;
+      if (knownChildren.isEmpty()) {
+        if (log.isDebugEnabled()) log.debug("known children is empty return null");
+        return null;
+      }
+      if (log.isDebugEnabled()) log.debug("signal all and get first, remove={}", remove);
+      changed.signalAll();
+      return remove ? knownChildren.pollFirst() : knownChildren.first();
+    } finally {
+      updateLock.unlock();
     }
   }
 
   /**
    * Return the current set of children from ZK; does not change internal state.
    */
-  TreeMap<String,byte[]> fetchZkChildren(Watcher watcher, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
-    if (log.isDebugEnabled()) log.debug("fetchZkChildren");
-    TreeMap<String,byte[]> orderedChildren = new TreeMap<>();
-    updateLock.lockInterruptibly();
-    try {
-      if (knownChildren != null && !knownChildren.isEmpty()) {
-        Set<Map.Entry<String,byte[]>> entrySet = knownChildren.entrySet();
-        for (Map.Entry<String,byte[]> entry : entrySet) {
-          String childName = entry.getKey();
+  TreeSet<String> fetchZkChildren(Watcher watcher) throws InterruptedException, KeeperException {
+    while (true) {
+      try {
+        TreeSet<String> orderedChildren = new TreeSet<>();
+        if (log.isDebugEnabled()) log.debug("fetchZkChildren {}", dir);
+        List<String> childNames = zookeeper.getChildren(dir, watcher, true);
+        if (log.isDebugEnabled()) log.debug("got {} nodes", childNames);
+        stats.setQueueLength(childNames.size());
+        for (String childName : childNames) {
           // Check format
-          if (!childName.startsWith(PREFIX)) {
-
-            // responses can be written to same queue with different naming scheme
-            if (log.isDebugEnabled()) log.debug("Filtering child out by prefix name=: {}, prefix={}", childName, PREFIX);
+          if (!childName.regionMatches(0, PREFIX, 0, PREFIX.length())) {
+            log.debug("Found child node with improper name: {}", childName);
             continue;
           }
-          if (acceptFilter != null && acceptFilter.test(childName)) {
-            if (log.isDebugEnabled()) log.debug("Found child that matched exclude filter: {}", dir + "/" + childName);
-            continue;
-          }
-          if (log.isDebugEnabled()) log.debug("found: {}", childName);
-          orderedChildren.put(childName, entry.getValue());
-        }
-      }
-    } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
-      }
-    }
-
-    if (!orderedChildren.isEmpty()) {
-      if (log.isDebugEnabled()) log.debug("found children from fetch {}", orderedChildren.size());
-      return orderedChildren;
-    }
-    if (log.isDebugEnabled()) log.debug("found no children to fetch");
-    TreeMap<String,byte[]> remoteKnownChildren = new TreeMap<>();
-    try {
-      List<String> childNames = zookeeper.getChildren(dir, watcher, true);
-      stats.setQueueLength(childNames.size());
-      for (String childName : childNames) {
-        if (log.isDebugEnabled()) log.debug("Examine child: {} out of children={} acceptFilter={}", childName, childNames.size(), acceptFilter);
-        // Check format
-        if (!childName.startsWith(PREFIX)) {
-          // responses can be written to same queue with different naming scheme
-          if (log.isDebugEnabled()) log.debug("Excluding child by prefix: {}", childName);
-          continue;
-        }
-        remoteKnownChildren.put(childName, null);
-        if (acceptFilter != null && acceptFilter.test(childName)) {
-          if (log.isDebugEnabled()) log.debug("Found child that matched exclude filter: {}", childName);
-          continue;
-        }
-        if (log.isDebugEnabled()) log.debug("Add child to fetched children: {}", childName);
-        orderedChildren.put(childName, null);
-      }
-      if (log.isDebugEnabled()) log.debug("found {} remote children", remoteKnownChildren.size());
-      updateKnownChildren(remoteKnownChildren);
-      if (log.isDebugEnabled()) log.debug("returning {} matched children", orderedChildren.size());
-      return orderedChildren;
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new AlreadyClosedException(e);
-    }
-  }
-
-  private void updateKnownChildren(TreeMap<String,byte[]> children) {
-    if (log.isDebugEnabled()) log.debug("Update known children size={}", children.size());
-    TreeMap<String,byte[]> newKnownChildren = new TreeMap<>();
-    updateLock.lock();
-    try {
-      Set<Map.Entry<String,byte[]>> entrySet = children.entrySet();
-      for (Map.Entry<String,byte[]> entry : entrySet) {
-        String childName = entry.getKey();
-        byte[] data = entry.getValue();
-        if (data == null) {
-          data = knownChildren == null ? null : knownChildren.get(childName);
+          orderedChildren.add(childName);
         }
-        newKnownChildren.put(childName, data);
+        if (log.isDebugEnabled()) log.debug("returning {}", orderedChildren);
+        return orderedChildren;
+      } catch (KeeperException.NoNodeException e) {
+        zookeeper.makePath(dir, false, true);
+        // go back to the loop and try again
       }
-      knownChildren = newKnownChildren;
-    } catch (Exception e) {
-      log.error("", e);
-    } finally {
-      updateLock.unlock();
     }
   }
 
@@ -553,139 +500,67 @@ public class ZkDistributedQueue implements DistributedQueue {
    */
   @Override
   public Collection<Pair<String, byte[]>> peekElements(int max, long waitMillis, Predicate<String> acceptFilter) throws KeeperException, InterruptedException {
-    if (log.isDebugEnabled()) log.debug("peekElements {} {}", max, acceptFilter);
-    List<Pair<String,byte[]>> result = null;
-    ChildWatcher watcher = new ChildWatcher(acceptFilter);
-    TreeMap<String,byte[]> foundChildren = fetchZkChildren(watcher, acceptFilter);
+    List<String> foundChildren = new ArrayList<>();
     long waitNanos = TimeUnit.MILLISECONDS.toNanos(waitMillis);
-    TimeOut timeout = new TimeOut(waitNanos, TimeUnit.NANOSECONDS, TimeSource.NANO_TIME);
-    try {
-      if (foundChildren.size() == 0) {
-        if (log.isDebugEnabled()) log.debug("found no children, watch for them  excludeFilter={}", acceptFilter);
-        waitForChildren(acceptFilter, foundChildren, timeout, watcher);
-      }
+    if (log.isDebugEnabled()) log.debug("peekElements {} {}", max, waitMillis);
+    boolean first = true;
+    while (true) {
+      // Trigger a refresh, but only force it if this is not the first iteration.
+      if (log.isDebugEnabled()) log.debug("call first child {}", !first);
+      firstChild(false, first);
 
-      // Technically we could restart the method if we fasil to actually obtain any valid children
-      // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
-      // much more sophisticated waitNanos tracking.
-      if (log.isDebugEnabled()) log.debug("found children to process {}", foundChildren.size());
-      result = Collections.synchronizedList(new ArrayList<>(foundChildren.size()));
-      Set<String> dataPaths = new HashSet<>();
-      for (Map.Entry<String,byte[]> child : foundChildren.entrySet()) {
-        if (result.size() >= max) {
+      updateLock.lockInterruptibly();
+      try {
+        for (String child : knownChildren) {
+          if (acceptFilter.test(child)) {
+            foundChildren.add(child);
+          }
+        }
+        if (!foundChildren.isEmpty()) {
           break;
         }
-
-        byte[] data = child.getValue();
-        if (data == null) {
-          updateLock.lockInterruptibly();
-          try {
-            data = knownChildren.get(child.getKey());
-          } finally {
-            if (updateLock.isHeldByCurrentThread()) {
-              updateLock.unlock();
-            }
-          }
+        if (waitNanos <= 0) {
+          break;
         }
 
-        if (data == null) {
-          // nocommit - lets not reget what we have in knownChildren, also, use asyncp
-          dataPaths.add(dir + "/" + child.getKey());
-          if (log.isDebugEnabled()) log.debug("get data for child={}", child.getKey());
-        } else {
-          if (log.isDebugEnabled()) log.debug("found data locally already {}", child.getKey());
-          result.add(new Pair<>(child.getKey(), data));
+        // If this is our first time through, force a refresh before waiting.
+        if (first) {
+          first = false;
+          continue;
         }
-      }
 
-      if (log.isDebugEnabled()) log.debug("fetch data for paths {}", dataPaths);
-      Map<String,byte[]> dataMap = zookeeper.getData(dataPaths);
-      updateLock.lockInterruptibly();
-      List<Pair<String,byte[]>> finalResult = result;
-      try {
-        dataMap.forEach((k, bytes) -> {
-          finalResult.add(new Pair<>(k, bytes));
-          if (bytes != null) {
-            knownChildren.put(new File(k).getName(), bytes);
-          }
-        });
+        waitNanos = changed.awaitNanos(waitNanos);
       } finally {
-        if (updateLock.isHeldByCurrentThread()) {
-          updateLock.unlock();
-        }
+        updateLock.unlock();
       }
-      if (log.isDebugEnabled()) log.debug("peek elements returning {} nodes", finalResult.size());
-      return new ArrayList<>(finalResult);
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new AlreadyClosedException(e);
-    }
-  }
-
-  private void waitForChildren(Predicate<String> acceptFilter, TreeMap<String,byte[]> foundChildren, TimeOut timeout, ChildWatcher watcher) throws InterruptedException, KeeperException {
-    if (log.isDebugEnabled()) log.debug("wait for children ... {}ms", timeout.getInterval(TimeUnit.MILLISECONDS));
 
-    updateLock.lockInterruptibly();
-    try {
-      for (Map.Entry<String,byte[]> child : knownChildren.entrySet()) {
-        if (!child.getKey().startsWith(PREFIX) && (acceptFilter == null || !acceptFilter.test(child.getKey()))) {
-          foundChildren.put(child.getKey(), child.getValue());
-        }
-      }
-    } finally {
-      if (updateLock.isHeldByCurrentThread()) {
-        updateLock.unlock();
+      if (!foundChildren.isEmpty()) {
+        break;
       }
     }
-    if (!foundChildren.isEmpty()) {
-      if (log.isDebugEnabled()) log.debug("Found new children ... {}", foundChildren.size());
-      return;
-    }
-    if (timeout.hasTimedOut()) {
-      if (log.isDebugEnabled()) log.debug("0 wait time and no children found, return");
-      return;
-    }
-    TreeMap<String,byte[]> fc = null;
-    while (fc == null || fc.isEmpty()) {
-      fc = fetchZkChildren(watcher, acceptFilter);
-      if (!fc.isEmpty()) {
-        foundChildren.putAll(fc);
-        return;
+
+    // Technically we could restart the method if we fail to actually obtain any valid children
+    // from ZK, but this is a super rare case, and the latency of the ZK fetches would require
+    // much more sophisticated waitNanos tracking.
+    List<Pair<String, byte[]>> result = new ArrayList<>();
+    for (String child : foundChildren) {
+      if (result.size() >= max) {
+        break;
       }
-      updateLock.lockInterruptibly();
       try {
+        byte[] data = zookeeper.getData(dir + "/" + child, null, null, true);
+        result.add(new Pair<>(child, data));
+      } catch (KeeperException.NoNodeException e) {
+        // Another client deleted the node first, remove the in-memory and continue.
+        updateLock.lockInterruptibly();
         try {
-          changed.await(Math.min(timeout.getInterval(TimeUnit.MILLISECONDS), 5000), TimeUnit.MILLISECONDS);
-        } catch (InterruptedException e) {
-          ParWork.propagateInterrupt(e);
-        }
-        if (zookeeper.isClosed() || !zookeeper.isConnected()) {
-          throw new AlreadyClosedException();
-        }
-        if (timeout.hasTimedOut()) {
-          return;
-        }
-        for (Map.Entry<String,byte[]> child : knownChildren.entrySet()) {
-          if (acceptFilter == null || !acceptFilter.test(child.getKey())) {
-            foundChildren.put(child.getKey(), child.getValue());
-          }
-        }
-        if (!foundChildren.isEmpty()) {
-          try {
-            if (log.isDebugEnabled()) log.debug("Remove watches for {}");
-            zookeeper.getSolrZooKeeper().removeAllWatches(dir, Watcher.WatcherType.Children, false);
-          } catch (Exception e) {
-            log.info(e.getMessage());
-          }
-          return;
-        }
-
-      } finally {
-        if (updateLock.isHeldByCurrentThread()) {
+          knownChildren.remove(child);
+        } finally {
           updateLock.unlock();
         }
       }
     }
+    return result;
   }
 
   /**
@@ -693,137 +568,96 @@ public class ZkDistributedQueue implements DistributedQueue {
    *
    * @return the data at the head of the queue.
    */
-  private byte[] firstElement(Predicate<String> acceptFilter) throws KeeperException {
-    try {
-
-      Map.Entry<String,byte[]> firstChild = firstChild(false, acceptFilter);
+  private byte[] firstElement() throws KeeperException, InterruptedException {
+    while (true) {
+      if (log.isDebugEnabled()) log.debug("calling firstElement");
+      String firstChild = firstChild(false, false);
       if (firstChild == null) {
+        if (log.isDebugEnabled()) log.debug("return null");
         return null;
       }
-      byte[] data;
-      updateLock.lockInterruptibly();
-      try {
-        data = knownChildren.get(firstChild.getKey());
-        if (data != null) {
-          return data;
-        }
-      } finally {
-        if (updateLock.isHeldByCurrentThread()) {
-          updateLock.unlock();
-        }
-      }
-
       try {
-        data = zookeeper.getData(dir + "/" + firstChild.getKey(), null, null, true);
-        if (data != null) {
-          updateLock.lockInterruptibly();
-          try {
-            knownChildren.put(firstChild.getKey(), data);
-          } finally {
-            if (updateLock.isHeldByCurrentThread()) {
-              updateLock.unlock();
-            }
-          }
-        }
-
-        return data;
+        if (log.isDebugEnabled()) log.debug("get data {}", dir + "/" + firstChild);
+        return zookeeper.getData(dir + "/" + firstChild, null, null, true);
       } catch (KeeperException.NoNodeException e) {
+        if (log.isDebugEnabled()) log.debug("no node found {}, clear known children and set dirty", dir + "/" + firstChild);
+        // Another client deleted the node first, remove the in-memory and retry.
         updateLock.lockInterruptibly();
         try {
-          knownChildren.remove(firstChild.getKey());
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
         } finally {
-          if (updateLock.isHeldByCurrentThread()) {
-            updateLock.unlock();
-          }
+          updateLock.unlock();
         }
-        return null;
       }
-
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new AlreadyClosedException(e);
     }
   }
 
-  private byte[] removeFirst(Predicate<String> acceptFilter) throws KeeperException {
-    try {
-      Map.Entry<String,byte[]> firstChild = firstChild(true, acceptFilter);
+  private byte[] removeFirst() throws KeeperException, InterruptedException {
+    while (true) {
+      String firstChild = firstChild(true, false);
       if (firstChild == null) {
         return null;
       }
-
-      byte[] data;
-      updateLock.lockInterruptibly();
       try {
-        data = knownChildren.remove(firstChild.getKey());
-      } finally {
-        if (updateLock.isHeldByCurrentThread()) {
-          updateLock.unlock();
-        }
-      }
-
-      try {
-        String path = dir + "/" + firstChild.getKey();
-        if (data == null) {
-          data = zookeeper.getData(path, null, null, true);
-        }
+        String path = dir + "/" + firstChild;
+        byte[] result = zookeeper.getData(path, null, null, true);
         zookeeper.delete(path, -1, true);
+        stats.setQueueLength(knownChildren.size());
+        return result;
+      } catch (KeeperException.NoNodeException e) {
+        // Another client deleted the node first, remove the in-memory and retry.
         updateLock.lockInterruptibly();
         try {
-          knownChildren.remove(firstChild.getKey());
-
+          // Efficient only for single-consumer
+          knownChildren.clear();
+          isDirty = true;
         } finally {
-          if (updateLock.isHeldByCurrentThread()) {
-            updateLock.unlock();
-          }
+          updateLock.unlock();
         }
-        return data;
-      } catch (KeeperException.NoNodeException e) {
-        return null;
       }
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new AlreadyClosedException(e);
     }
   }
 
-  @VisibleForTesting long watcherCount() throws InterruptedException {
-    return 0;
+  @VisibleForTesting int watcherCount() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
+      return watcherCount;
+    } finally {
+      updateLock.unlock();
+    }
   }
 
-  @VisibleForTesting class ChildWatcher implements Watcher {
-
-    private final Predicate<String> acceptFilter;
+  @VisibleForTesting boolean isDirty() throws InterruptedException {
+    updateLock.lockInterruptibly();
+    try {
 
-    public ChildWatcher(Predicate<String> acceptFilter) {
-      this.acceptFilter = acceptFilter;
+      if (log.isDebugEnabled()) log.debug("isDirty {}", isDirty);
+      return isDirty;
+    } finally {
+      updateLock.unlock();
     }
+  }
+
+  @VisibleForTesting class ChildWatcher implements Watcher {
 
     @Override
     public void process(WatchedEvent event) {
+
+      if (log.isDebugEnabled()) log.debug("ChildWatcher fired {} {}", event.getType(), event.getPath());
       // session events are not change events, and do not remove the watcher; except for Expired
-      if (Event.EventType.None.equals(event.getType())) {
+      if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
         return;
       }
-      if (log.isDebugEnabled()) log.debug("DistributedQueue changed {} {}", event.getPath(), event.getType());
-
-      // nocommit - all the nodes are watching this currently instead of just the Overseer
-      if (event.getType() == Event.EventType.NodeChildrenChanged) {
-        updateLock.lock();
-        try {
-          TreeMap<String,byte[]> found = fetchZkChildren(null, acceptFilter);
-          if (!found.isEmpty()) {
-            changed.signalAll();
-          } else {
-            fetchZkChildren(this, acceptFilter);
-          }
-        } catch (KeeperException | InterruptedException e) {
-          log.error("", e);
-        } finally {
-          updateLock.unlock();
-        }
-      } else {
-
+      updateLock.lock();
+      try {
+        isDirty = true;
+        watcherCount--;
+        // optimistically signal any waiters that the queue may not be empty now, so they can wake up and retry
+        changed.signalAll();
+      } finally {
+        updateLock.unlock();
       }
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
index 83be9ef..6dd410a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkShardTerms.java
@@ -186,7 +186,7 @@ public class ZkShardTerms implements AutoCloseable{
     listeners.removeIf(coreTermWatcher -> !coreTermWatcher.onTermChanged(terms.get()));
     numListeners = listeners.size();
 
-    return removeTerm(cd.getCloudDescriptor().getCoreNodeName()) || numListeners == 0;
+    return removeTerm(cd.getName()) || numListeners == 0;
   }
 
   // package private for testing, only used by tests
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
index df9967e..2a1da0d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AddReplicaCmd.java
@@ -21,21 +21,24 @@ import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.EnumMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.stream.Collectors;
 
-import org.apache.commons.lang3.StringUtils;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.response.Cluster;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
+import org.apache.solr.cloud.overseer.ZkWriteCommand;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -60,6 +63,7 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.SKIP_CREATE_REPLICA_IN_CLUSTER_STATE;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
@@ -82,27 +86,45 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
   public static final String SKIP_NODE_ASSIGNMENT = "skipNodeAssignment";
 
   private final OverseerCollectionMessageHandler ocmh;
+  private final boolean onlyUpdateState;
+  private boolean createdShardHandler;
 
   public AddReplicaCmd(OverseerCollectionMessageHandler ocmh) {
+    this.onlyUpdateState = false;
+    this.ocmh = ocmh;
+  }
+
+  public AddReplicaCmd(OverseerCollectionMessageHandler ocmh, boolean onlyUpdateState) {
+    this.onlyUpdateState = onlyUpdateState;
     this.ocmh = ocmh;
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
-    Response response = addReplica(state, message, results, null);
-    if (response == null) return null;
-    return response.asyncFinalRunner;
+  public Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+    ShardHandler shardHandler = null;
+    ShardRequestTracker shardRequestTracker = null;
+
+    if (!onlyUpdateState) {
+      final String asyncId = message.getStr(ASYNC);
+      shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
+      shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+      createdShardHandler = true;
+    }
+
+    Response response = addReplica(state, message, shardHandler, shardRequestTracker, results);
+    return response;
   }
 
   @SuppressWarnings({"unchecked"})
-  Response addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+  Response addReplica(ClusterState clusterState, ZkNodeProps message, ShardHandler shardHandler,
+      ShardRequestTracker shardRequestTracker, @SuppressWarnings({"rawtypes"})NamedList results)
       throws IOException, InterruptedException, KeeperException {
 
     log.info("addReplica() : {}", Utils.toJSONString(message));
 
     String extCollectionName = message.getStr(COLLECTION_PROP);
     boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
-    String shard = message.getStr(SHARD_ID_PROP);
 
     final String collectionName;
     if (followAliases) {
@@ -111,19 +133,10 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       collectionName = extCollectionName;
     }
 
-    DocCollection coll = clusterState.getCollection(collectionName);
-    if (coll == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collectionName + " does not exist");
-    }
-    if (coll.getSlice(shard) == null) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-          "Collection: " + collectionName + " shard: " + shard + " does not exist");
-    }
-
     // nocommit
     boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
     boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
-    final String asyncId = message.getStr(ASYNC);
+
 
     String node = message.getStr(CoreAdminParams.NODE);
     String createNodeSetStr = message.getStr(ZkStateReader.CREATE_NODE_SET);
@@ -149,83 +162,87 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       if (node != null) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create " + totalReplicas + " replicas if 'node' parameter is specified");
       }
-      if (message.getStr(CoreAdminParams.CORE_NODE_NAME) != null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Cannot create " + totalReplicas + " replicas if 'coreNodeName' parameter is specified");
-      }
     }
 
-    List<CreateReplica> createReplicas;
-    try {
-      ocmh.zkStateReader.waitForState(collectionName, 15, TimeUnit.SECONDS, (liveNodes, collectionState) -> {
-        if (collectionState == null) {
-          return false;
-        }
-        if (collectionState.getSlices() == null) {
-          return false;
-        }
-        return true;
-      });
-    } catch (TimeoutException e) {
-      log.warn("Could not find collection with populated shards in clusterstate \n{}", clusterState);
-    }
-    clusterState = ocmh.zkStateReader.getClusterState();
+    List<CreateReplica> createReplicas = new ArrayList<>();
+
+    DocCollection collection = clusterState.getCollection(collectionName);
+    List<ReplicaPosition> positions = buildReplicaPositions(ocmh.cloudManager, clusterState, collection, message, replicaTypesVsCount);
+    for (ReplicaPosition replicaPosition : positions) {
+      clusterState = new CollectionMutator(ocmh.cloudManager).modifyCollection(clusterState, message);
+      collection = clusterState.getCollection(collectionName);
+      CreateReplica cr = assignReplicaDetails(collection, message, replicaPosition);
+
+      message = message.plus(ZkStateReader.BASE_URL_PROP, ocmh.zkStateReader.getBaseUrlForNodeName(replicaPosition.node));
 
-    ClusterState finalClusterState = clusterState;
-    createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collectionName, message, replicaTypesVsCount)
-        .stream()
-        .map(replicaPosition -> assignReplicaDetails(ocmh.cloudManager, finalClusterState, message, replicaPosition))
-        .collect(Collectors.toList());
+      message = message.plus(NODE_NAME_PROP, replicaPosition.node);
+      message = message.plus(ZkStateReader.REPLICA_TYPE, cr.replicaType.name());
 
-    if (createReplicas.size() != totalReplicas) {
-      throw new IllegalStateException("Did not get enough positions to cover new replicas");
+
+      clusterState = new SliceMutator(ocmh.cloudManager).addReplica(clusterState, message);
+      createReplicas.add(cr);
+
+     // message.getProperties().put("node_name", cr.node)
     }
 
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
-    ZkStateReader zkStateReader = ocmh.zkStateReader;
+//    createReplicas = buildReplicaPositions(ocmh.cloudManager, clusterState, collection, message, replicaTypesVsCount)
+//        .stream()
+//        .map(replicaPosition -> assignReplicaDetails(collection, message, replicaPosition))
+//        .collect(Collectors.toList());
+
 
-    final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
     for (CreateReplica createReplica : createReplicas) {
-      assert createReplica.coreName != null;
-      ModifiableSolrParams params = getReplicaParams(clusterState, message, results, collectionName, coll, skipCreateReplicaInClusterState, shardHandler, createReplica);
-      shardRequestTracker.sendShardRequest(createReplica.node, params, shardHandler);
-    }
 
-    try {
-      shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
-    } catch (Exception e) {
-      ParWork.propagateInterrupt(e);
-      return null;
+      ModifiableSolrParams params = getReplicaParams(collection, message, results, skipCreateReplicaInClusterState, shardHandler, createReplica);
+
+      log.info("create replica {} params={}", createReplica, params);
+      if (!onlyUpdateState) {
+        shardRequestTracker.sendShardRequest(createReplica.node, params, shardHandler);
+      }
     }
 
-    if (onComplete != null) onComplete.run();
+    Response response = new Response();
 
-    if (onComplete != null) onComplete.run();
+    if (!onlyUpdateState) {
+      response.responseProps = createReplicas.stream().map(
+          createReplica -> new ZkNodeProps(ZkStateReader.COLLECTION_PROP, createReplica.collectionName, ZkStateReader.SHARD_ID_PROP, createReplica.sliceName, ZkStateReader.CORE_NAME_PROP,
+              createReplica.coreName, ZkStateReader.NODE_NAME_PROP, createReplica.node)).collect(Collectors.toList());
+      response.results = results;
+
+        ZkNodeProps finalMessage = message;
+        response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+          @Override
+          public Response call() {
+            if (!onlyUpdateState && createdShardHandler) {
+              try {
+                log.info("Processs responses");
+                shardRequestTracker.processResponses(results, shardHandler, true, "ADDREPLICA failed to create replica");
+              } catch (Exception e) {
+                ParWork.propagateInterrupt(e);
+                throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+              }
+            }
+
+            String asyncId = finalMessage.getStr(ASYNC);
+            for (CreateReplica createReplica : createReplicas) {
+              waitForActiveReplica(createReplica.sliceName, collectionName, asyncId, ocmh.zkStateReader, createReplicas);
+            }
+            AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+            return response;
+          }
+        };
 
-    Response response = new Response();
-    response.responseProps =  createReplicas.stream()
-        .map(createReplica -> new ZkNodeProps(
-            ZkStateReader.COLLECTION_PROP, createReplica.collectionName,
-            ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
-            ZkStateReader.CORE_NAME_PROP, createReplica.coreName,
-            ZkStateReader.NODE_NAME_PROP, createReplica.node
-        ))
-        .collect(Collectors.toList());
-    if (results.get("failure") == null && results.get("exception") == null) {
-      response.asyncFinalRunner = new Runnable() {
-        @Override
-        public void run() {
-          waitForActiveReplica(shard, collectionName, asyncId, zkStateReader, createReplicas);
-        }
-      };
     }
 
+    response.clusterState = clusterState;
+
     return response;
   }
 
   private void waitForActiveReplica(String shard, String collectionName, String asyncId, ZkStateReader zkStateReader, List<CreateReplica> createReplicas) {
-    Set<String> coreNodeNames = ConcurrentHashMap.newKeySet(createReplicas.size());
+    Set<String> coreNames = new HashSet<>(createReplicas.size());
     for (CreateReplica replica : createReplicas) {
-      coreNodeNames.add(ocmh.waitForCoreNodeName(zkStateReader, collectionName, replica.node, replica.coreName));
+      coreNames.add(replica.coreName);
     }
     try {
       zkStateReader.waitForState(collectionName, 60, TimeUnit.SECONDS, (liveNodes, collectionState) -> {
@@ -240,14 +257,14 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
 
         List<Replica> replicas = collectionState.getReplicas();
         int found = 0;
-        for (String name : coreNodeNames) {
+        for (String name : coreNames) {
           for (Replica replica : replicas) {
-            if (replica.getName().equals(name) && replica.getState().equals(Replica.State.ACTIVE) && liveNodes.contains(replica.getNodeName())) {
+            if (replica.getName().equals(name) && replica.getState().equals(Replica.State.ACTIVE)) {
               found++;
             }
           }
         }
-        if (found == coreNodeNames.size()) {
+        if (found == coreNames.size()) {
           return true;
         }
 
@@ -261,50 +278,43 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     }
   }
 
-  private ModifiableSolrParams getReplicaParams(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, String collectionName, DocCollection coll, boolean skipCreateReplicaInClusterState, ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
-    if (coll.getStr(WITH_COLLECTION) != null) {
-      String withCollectionName = coll.getStr(WITH_COLLECTION);
-      DocCollection withCollection = clusterState.getCollection(withCollectionName);
-      if (withCollection.getActiveSlices().size() > 1)  {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The `withCollection` must have only one shard, found: " + withCollection.getActiveSlices().size());
-      }
-      String withCollectionShard = withCollection.getActiveSlices().iterator().next().getName();
-
-      List<Replica> replicas = withCollection.getReplicas(createReplica.node);
-      if (replicas == null || replicas.isEmpty()) {
-        // create a replica of withCollection on the identified node before proceeding further
-        ZkNodeProps props = new ZkNodeProps(
-            Overseer.QUEUE_OPERATION, ADDREPLICA.toString(),
-            ZkStateReader.COLLECTION_PROP, withCollectionName,
-            ZkStateReader.SHARD_ID_PROP, withCollectionShard,
-            "node", createReplica.node,
-            // since we already computed node assignments (which include assigning a node for this withCollection replica) we want to skip the assignment step
-            SKIP_NODE_ASSIGNMENT, "true",
-            CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
-        addReplica(clusterState, props, results, null);
-      }
-    }
+  private ModifiableSolrParams getReplicaParams(   DocCollection collection, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results,
+      boolean skipCreateReplicaInClusterState,
+      ShardHandler shardHandler, CreateReplica createReplica) throws IOException, InterruptedException, KeeperException {
+
+    // nocommit
+//    if (collection.getStr(WITH_COLLECTION) != null) {
+//      String withCollectionName = collection.getStr(WITH_COLLECTION);
+//      DocCollection withCollection = clusterState.getCollection(withCollectionName);
+//      if (withCollection.getActiveSlices().size() > 1)  {
+//        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "The `withCollection` must have only one shard, found: " + withCollection.getActiveSlices().size());
+//      }
+//      String withCollectionShard = withCollection.getActiveSlices().iterator().next().getName();
+//
+//      List<Replica> replicas = withCollection.getReplicas(createReplica.node);
+//      if (replicas == null || replicas.isEmpty()) {
+//        // create a replica of withCollection on the identified node before proceeding further
+//        ZkNodeProps props = new ZkNodeProps(
+//            Overseer.QUEUE_OPERATION, ADDREPLICA.toString(),
+//            ZkStateReader.COLLECTION_PROP, withCollectionName,
+//            ZkStateReader.SHARD_ID_PROP, withCollectionShard,
+//            CORE_NAME_PROP, createReplica.coreName,
+//            "node", createReplica.node,
+//            // since we already computed node assignments (which include assigning a node for this withCollection replica) we want to skip the assignment step
+//            SKIP_NODE_ASSIGNMENT, "true",
+//            CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
+//        addReplica(clusterState, props, results);
+//      }
+//    }
 
     ModifiableSolrParams params = new ModifiableSolrParams();
     ZkStateReader zkStateReader = ocmh.zkStateReader;
-
+    String collectionName = collection.getName();
     ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP, createReplica.sliceName,
         ZkStateReader.CORE_NAME_PROP, createReplica.coreName, ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP,
         zkStateReader.getBaseUrlForNodeName(createReplica.node), ZkStateReader.NODE_NAME_PROP, createReplica.node, ZkStateReader.REPLICA_TYPE, createReplica.replicaType.name());
-    if (createReplica.coreNodeName != null) {
-      props = props.plus(ZkStateReader.CORE_NODE_NAME_PROP, createReplica.coreNodeName);
-    }
-    if (!skipCreateReplicaInClusterState) {
-      try {
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
-      } catch (Exception e) {
-        ParWork.propagateInterrupt(e);
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Exception updating Overseer state queue", e);
-      }
-    }
+
     String coreUrl = ZkCoreNodeProps.getCoreUrl(props.getStr(ZkStateReader.BASE_URL_PROP), createReplica.coreName);
-    ;
-    params.set(CoreAdminParams.CORE_NODE_NAME, ocmh.waitToSeeReplicasInState(collectionName, Collections.singletonList(coreUrl), false).get(coreUrl).getName());
 
     String configName = zkStateReader.readConfigName(collectionName);
     String routeKey = message.getStr(ShardParams._ROUTE_);
@@ -317,10 +327,12 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     params.set(COLL_CONF, configName);
     params.set(CoreAdminParams.COLLECTION, collectionName);
     params.set(CoreAdminParams.REPLICA_TYPE, createReplica.replicaType.name());
+
+    log.info("Creating SolrCore with name={}", createReplica.coreName);
     if (createReplica.sliceName != null) {
       params.set(CoreAdminParams.SHARD, createReplica.sliceName);
     } else if (routeKey != null) {
-      Collection<Slice> slices = coll.getRouter().getSearchSlicesSingle(routeKey, null, coll);
+      Collection<Slice> slices = collection.getRouter().getSearchSlicesSingle(routeKey, null, collection);
       if (slices.isEmpty()) {
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No active shard serving _route_=" + routeKey + " found");
       } else {
@@ -338,71 +350,41 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     if (instanceDir != null) {
       params.set(CoreAdminParams.INSTANCE_DIR, instanceDir);
     }
-    if (createReplica.coreNodeName != null) {
-      params.set(CoreAdminParams.CORE_NODE_NAME, createReplica.coreNodeName);
-    }
+
     ocmh.addPropertyParams(message, params);
 
     return params;
   }
 
-  public static CreateReplica assignReplicaDetails(SolrCloudManager cloudManager, ClusterState clusterState,
+  public static CreateReplica assignReplicaDetails(DocCollection coll,
                                                  ZkNodeProps message, ReplicaPosition replicaPosition) {
+
+    log.info("assignReplicaDetails {} {} {}", message, replicaPosition, coll);
+
     boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
 
     String collection = message.getStr(COLLECTION_PROP);
     String node = replicaPosition.node;
     String shard = message.getStr(SHARD_ID_PROP);
-    String coreName = message.getStr(CoreAdminParams.NAME);
-    String coreNodeName = message.getStr(CoreAdminParams.CORE_NODE_NAME);
+    String coreName = message.getStr(CoreAdminParams.CORE);
     Replica.Type replicaType = replicaPosition.type;
 
-    if (StringUtils.isBlank(coreName)) {
-      coreName = message.getStr(CoreAdminParams.PROPERTY_PREFIX + CoreAdminParams.NAME);
-    }
+    log.info("Node Identified {} for creating new replica (core={}) of shard {} for collection {} currentReplicaCount {}", node, coreName, shard, collection, coll.getReplicas().size());
 
-    log.info("Node Identified {} for creating new replica of shard {} for collection {}", node, shard, collection);
-    if (!clusterState.liveNodesContain(node)) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
-    }
-    DocCollection coll = clusterState.getCollection(collection);
     if (coreName == null) {
-      coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
-    } else if (!skipCreateReplicaInClusterState) {
-      //Validate that the core name is unique in that collection
-      while (true) {
-        try {
-          clusterState = cloudManager.getClusterStateProvider().getClusterState();
-        } catch (IOException e) {
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Node: " + node + " is not live");
-        }
-        coll = clusterState.getCollection(collection);
-        for (Slice slice : coll.getSlices()) {
-          for (Replica replica : slice.getReplicas()) {
-            String replicaCoreName = replica.getStr(CORE_NAME_PROP);
-            if (coreName.equals(replicaCoreName)) {
-              log.info("Another replica with the same core name already exists" +
-                      " for this collection");
-              coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), coll, shard, replicaType);
-              continue;
-            }
-          }
-        }
-        break;
-      }
+      coreName = Assign.buildSolrCoreName(coll, coll.getName(), shard, replicaType);
     }
-    log.info("Returning CreateReplica command.");
+    log.info("Returning CreateReplica command coreName={}", coreName);
 
-    return new CreateReplica(collection, shard, node, replicaType, coreName, coreNodeName);
+    return new CreateReplica(collection, shard, node, replicaType, coreName);
   }
 
-  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
-                                                            String collectionName, ZkNodeProps message,
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState, DocCollection collection,
+                                                            ZkNodeProps message,
                                                             EnumMap<Replica.Type, Integer> replicaTypeVsCount) throws IOException, InterruptedException {
     boolean skipCreateReplicaInClusterState = message.getBool(SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, false);
     boolean skipNodeAssignment = message.getBool(SKIP_NODE_ASSIGNMENT, false);
     String sliceName = message.getStr(SHARD_ID_PROP);
-    DocCollection collection = clusterState.getCollection(collectionName);
 
     int numNrtReplicas = replicaTypeVsCount.get(Replica.Type.NRT);
     int numPullReplicas = replicaTypeVsCount.get(Replica.Type.PULL);
@@ -421,7 +403,7 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     List<ReplicaPosition> positions = null;
     if (!skipCreateReplicaInClusterState && !skipNodeAssignment) {
 
-      positions = Assign.getNodesForNewReplicas(clusterState, collection.getName(), sliceName, numNrtReplicas,
+      positions = Assign.getNodesForNewReplicas(clusterState, collection, sliceName, numNrtReplicas,
                     numTlogReplicas, numPullReplicas, createNodeSetStr, cloudManager);
     }
 
@@ -457,21 +439,23 @@ public class AddReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     public final String node;
     public final Replica.Type replicaType;
     public String coreName;
-    public String coreNodeName;
 
-    CreateReplica(String collectionName, String sliceName, String node, Replica.Type replicaType, String coreName, String coreNodeName) {
+    CreateReplica(String collectionName, String sliceName, String node, Replica.Type replicaType, String coreName) {
       this.collectionName = collectionName;
       this.sliceName = sliceName;
       this.node = node;
       this.replicaType = replicaType;
       this.coreName = coreName;
-      this.coreNodeName = coreNodeName;
     }
   }
 
   public static class Response {
     List<ZkNodeProps> responseProps;
-    Runnable asyncFinalRunner;
+    OverseerCollectionMessageHandler.Finalize asyncFinalRunner;
+
+    NamedList results;
+
+    ClusterState clusterState;
   }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
index ade7396..349ba3d 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/AliasCmd.java
@@ -92,7 +92,7 @@ abstract class AliasCmd implements OverseerCollectionMessageHandler.Cmd {
 
 
     int numShards = BaseCloudSolrClient.getShardNames(zkProps).size();
-    BaseCloudSolrClient.waitForActiveCollection(ocmh.zkStateReader, createCollName, 60, TimeUnit.SECONDS, numShards, numShards * BaseCloudSolrClient.getTotalReplicas(zkProps));
+    ocmh.zkStateReader.waitForActiveCollection(createCollName, 60, TimeUnit.SECONDS, numShards, numShards * BaseCloudSolrClient.getTotalReplicas(zkProps));
     CollectionProperties collectionProperties = new CollectionProperties(ocmh.zkStateReader);
     collectionProperties.setCollectionProperty(createCollName,ROUTED_ALIAS_NAME_CORE_PROP,aliasName);
 
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 dc60668..4aa567f 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
@@ -57,14 +57,6 @@ public class Assign {
 
   private static AtomicInteger REPLICA_CNT = new AtomicInteger(0);
 
-  public static String assignCoreNodeName(DistribStateManager stateManager, DocCollection collection) {
-    // for backward compatibility;
-    int defaultValue = defaultCounterValue(collection, "");
-    String coreNodeName = "core_node" + defaultValue;
-
-    return coreNodeName;
-  }
-
   /**
    * Assign a new unique id up to slices count - then add replicas evenly.
    *
@@ -81,13 +73,13 @@ public class Assign {
     // TODO: now that we create shards ahead of time, is this code needed?  Esp since hash ranges aren't assigned when creating via this method?
 
     if (sliceMap == null) {
-      return "shard1";
+      return "s";
     }
 
     List<String> shardIdNames = new ArrayList<>(sliceMap.keySet());
 
     if (shardIdNames.size() < numShards) {
-      return "shard" + (shardIdNames.size() + 1);
+      return "s" + (shardIdNames.size() + 1);
     }
 
     // TODO: don't need to sort to find shard with fewest replicas!
@@ -109,42 +101,33 @@ public class Assign {
     return returnShardId;
   }
 
-  public static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
+  private static String buildSolrCoreName(String collectionName, String shard, Replica.Type type, int replicaNum) {
     // TODO: Adding the suffix is great for debugging, but may be an issue if at some point we want to support a way to change replica type
-    return String.format(Locale.ROOT, "%s_%s_replica_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
+    return String.format(Locale.ROOT, "%s_%s_r_%s%s", collectionName, shard, type.name().substring(0,1).toLowerCase(Locale.ROOT), replicaNum);
   }
 
-  public static int defaultCounterValue(DocCollection collection, String shard) {
-    int defaultValue;
-    if (collection.getSlice(shard) != null && collection.getSlice(shard).getReplicas().isEmpty()) {
-      return REPLICA_CNT.incrementAndGet();
-    } else {
-      defaultValue = collection.getReplicas().size() + REPLICA_CNT.incrementAndGet() * 2;
-    }
+  public static int defaultCounterValue(DocCollection coll, String shard) {
 
-    return defaultValue;
-  }
+    if (coll == null) {
+      throw new NullPointerException("DocCollection cannot be null");
+    }
 
-  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type, boolean newCollection) {
+    if (coll.getSlice(shard) == null) {
+      return 1;
+    }
 
-    int defaultValue = defaultCounterValue(collection, shard);
-    String coreName = buildSolrCoreName(collection.getName(), shard, type, defaultValue);
+    if (coll.getSlice(shard).getReplicas() == null) {
+      return 1;
+    }
 
-    return coreName;
+    return coll.getSlice(shard).getReplicas().size() + 1;
   }
 
-  public static String buildSolrCoreName(DistribStateManager stateManager, DocCollection collection, String shard, Replica.Type type) {
-    return buildSolrCoreName(stateManager, collection, shard, type, false);
-  }
+  public static String buildSolrCoreName(DocCollection coll, String collectionName, String shard, Replica.Type type) {
+    int defaultValue = defaultCounterValue(coll, shard);
+    String coreName = buildSolrCoreName(collectionName, shard, type, defaultValue);
 
-  private static boolean existCoreName(String coreName, Slice slice) {
-    if (slice == null) return false;
-    for (Replica replica : slice.getReplicas()) {
-      if (coreName.equals(replica.getStr(CORE_NAME_PROP))) {
-        return true;
-      }
-    }
-    return false;
+    return coreName;
   }
 
   public static List<String> getLiveOrLiveAndCreateNodeSetList(final Collection<String> liveNodes, final ZkNodeProps message, final Random random) {
@@ -188,13 +171,13 @@ public class Assign {
   // Gets a list of candidate nodes to put the required replica(s) on. Throws errors if not enough replicas
   // could be created on live nodes given maxShardsPerNode, Replication factor (if from createShard) etc.
   @SuppressWarnings({"unchecked"})
-  public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, String collectionName,
+  public static List<ReplicaPosition> getNodesForNewReplicas(ClusterState clusterState, DocCollection collection,
                                                           String shard, int nrtReplicas, int tlogReplicas, int pullReplicas,
                                                           Object createNodeSet, SolrCloudManager cloudManager) throws IOException, InterruptedException, AssignmentException {
     log.debug("getNodesForNewReplicas() shard: {} , nrtReplicas : {} , tlogReplicas: {} , pullReplicas: {} , createNodeSet {}"
         , shard, nrtReplicas, tlogReplicas, pullReplicas, createNodeSet);
-    DocCollection coll = clusterState.getCollection(collectionName);
-    int maxShardsPerNode = coll.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : coll.getMaxShardsPerNode();
+
+    //int maxShardsPerNode = collection.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : collection.getMaxShardsPerNode();
     List<String> createNodeList = null;
 
     if (createNodeSet instanceof List) {
@@ -203,23 +186,9 @@ public class Assign {
       // deduplicate
       createNodeList = createNodeSet == null ? null : new ArrayList<>(new LinkedHashSet<>(StrUtils.splitSmart((String) createNodeSet, ",", true)));
     }
+    String collectionName = collection.getName();
+    HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, cloudManager.getClusterStateProvider().getClusterState().getLiveNodes(), createNodeList);
 
-    HashMap<String, ReplicaCount> nodeNameVsShardCount = getNodeNameVsShardCount(collectionName, clusterState, createNodeList);
-
-    if (createNodeList == null) { // We only care if we haven't been told to put new replicas on specific nodes.
-      long availableSlots = 0;
-      for (Map.Entry<String, ReplicaCount> ent : nodeNameVsShardCount.entrySet()) {
-        //ADDREPLICA can put more than maxShardsPerNode on an instance, so this test is necessary.
-        if (maxShardsPerNode > ent.getValue().thisCollectionNodes) {
-          availableSlots += (maxShardsPerNode - ent.getValue().thisCollectionNodes);
-        }
-      }
-      if (availableSlots < nrtReplicas + tlogReplicas + pullReplicas) {
-        throw new AssignmentException(
-            String.format(Locale.ROOT, "Cannot create %d new replicas for collection %s given the current number of eligible live nodes %d and a maxShardsPerNode of %d",
-                nrtReplicas, collectionName, nodeNameVsShardCount.size(), maxShardsPerNode));
-      }
-    }
 
     AssignRequest assignRequest = new AssignRequestBuilder()
         .forCollection(collectionName)
@@ -230,16 +199,15 @@ public class Assign {
         .onNodes(createNodeList)
         .build();
     AssignStrategyFactory assignStrategyFactory = new AssignStrategyFactory(cloudManager);
-    AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, coll);
+    AssignStrategy assignStrategy = assignStrategyFactory.create();
     return assignStrategy.assign(cloudManager, assignRequest);
   }
 
-  static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName,
-                                                                       ClusterState clusterState, List<String> createNodeList) {
-    Set<String> nodes = clusterState.getLiveNodes();
+  static HashMap<String, ReplicaCount> getNodeNameVsShardCount(String collectionName, ClusterState clusterState,
+                                                                       Set<String> liveNodes, List<String> createNodeList) {
 
-    List<String> nodeList = new ArrayList<>(nodes.size());
-    nodeList.addAll(nodes);
+    List<String> nodeList = new ArrayList<>(liveNodes.size());
+    nodeList.addAll(liveNodes);
     if (createNodeList != null) nodeList.retainAll(createNodeList);
 
     HashMap<String, ReplicaCount> nodeNameVsShardCount = new HashMap<>(nodeList.size());
@@ -255,8 +223,6 @@ public class Assign {
       return nodeNameVsShardCount;
     }
 
-    DocCollection coll = clusterState.getCollection(collectionName);
-    int maxShardsPerNode = coll.getMaxShardsPerNode() == -1 ? Integer.MAX_VALUE : coll.getMaxShardsPerNode();
     Map<String, DocCollection> collections = clusterState.getCollectionsMap();
     for (Map.Entry<String, DocCollection> entry : collections.entrySet()) {
       if (entry.getValue() == null) continue;
@@ -379,7 +345,8 @@ public class Assign {
       ClusterState clusterState = solrCloudManager.getClusterStateProvider().getClusterState();
       List<String> nodeList = assignRequest.nodes;
 
-      HashMap<String, Assign.ReplicaCount> nodeNameVsShardCount = Assign.getNodeNameVsShardCount(assignRequest.collectionName, clusterState, assignRequest.nodes);
+      HashMap<String, Assign.ReplicaCount> nodeNameVsShardCount = Assign.getNodeNameVsShardCount(assignRequest.collectionName,
+          clusterState, solrCloudManager.getClusterStateProvider().getLiveNodes(),assignRequest.nodes);
       if (nodeList == null || nodeList.isEmpty()) {
         ArrayList<Assign.ReplicaCount> sortedNodeList = new ArrayList<>(nodeNameVsShardCount.values());
         sortedNodeList.sort(Comparator.comparingInt(Assign.ReplicaCount::weight));
@@ -409,7 +376,7 @@ public class Assign {
       this.solrCloudManager = solrCloudManager;
     }
 
-    public AssignStrategy create(ClusterState clusterState, DocCollection collection) throws IOException, InterruptedException {
+    public AssignStrategy create() throws IOException, InterruptedException {
 
       return new LegacyAssignStrategy();
 
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 eef735d..666bd71 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
@@ -29,6 +29,7 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Optional;
 import java.util.Properties;
+import java.util.stream.Collectors;
 
 import org.apache.lucene.util.Version;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@@ -41,6 +42,7 @@ import org.apache.solr.common.cloud.Replica.State;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -66,7 +68,7 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extCollectionName = message.getStr(COLLECTION_PROP);
     boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
     String collectionName;
@@ -135,7 +137,14 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
     backupMgr.downloadCollectionProperties(location, backupName, collectionName);
 
     log.info("Completed backing up ZK data for backupName={}", backupName);
-    return null;
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    response.results = results;
+
+    // nocommit - we don't change this for this cmd, we should be able to indicate that to caller
+    response.clusterState = clusterState;
+
+    return response;
   }
 
   private Replica selectReplicaWithSnapshot(CollectionSnapshotMetaData snapshotMeta, Slice slice) {
@@ -215,20 +224,18 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
         }
       }
 
-      String coreName = replica.getStr(CORE_NAME_PROP);
-
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.BACKUPCORE.toString());
       params.set(NAME, slice.getName());
       params.set(CoreAdminParams.BACKUP_REPOSITORY, repoName);
       params.set(CoreAdminParams.BACKUP_LOCATION, backupPath.toASCIIString()); // note: index dir will be here then the "snapshot." + slice name
-      params.set(CORE_NAME_PROP, coreName);
+      params.set(CORE_NAME_PROP, replica.getName());
       if (snapshotMeta.isPresent()) {
         params.set(CoreAdminParams.COMMIT_NAME, snapshotMeta.get().getName());
       }
 
       shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
-      log.debug("Sent backup request to core={} for backupName={}", coreName, backupName);
+      log.debug("Sent backup request to core={} for backupName={}", replica.getName(), 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/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
index d39f4b8..1cb9771 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -52,7 +52,7 @@ public class CreateAliasCmd extends AliasCmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
       throws Exception {
     final String aliasName = message.getStr(CommonParams.NAME);
     ZkStateReader zkStateReader = ocmh.zkStateReader;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
index 4ae8fb6..7f133bd 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateCollectionCmd.java
@@ -29,9 +29,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Properties;
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
 
 import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
 import org.apache.solr.client.solrj.cloud.DistribStateManager;
@@ -41,6 +41,8 @@ import org.apache.solr.client.solrj.impl.BaseCloudSolrClient;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -86,7 +88,6 @@ import static org.apache.solr.common.params.CollectionAdminParams.COLOCATED_WITH
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.MODIFYCOLLECTION;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
-import static org.apache.solr.common.params.CommonAdminParams.WAIT_FOR_FINAL_STATE;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.util.StrUtils.formatString;
 
@@ -99,6 +100,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
   private final SolrCloudManager cloudManager;
 
   public CreateCollectionCmd(OverseerCollectionMessageHandler ocmh, CoreContainer cc, SolrCloudManager cloudManager) {
+    log.info("create CreateCollectionCmd");
     this.ocmh = ocmh;
     this.stateManager = ocmh.cloudManager.getDistribStateManager();
     this.timeSource = ocmh.cloudManager.getTimeSource();
@@ -108,13 +110,17 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+    log.info("run call for CreateCollectionCmd {}", message);
     if (ocmh.zkStateReader.aliasesManager != null) { // not a mock ZkStateReader
-      ocmh.zkStateReader.aliasesManager.update();
+      ocmh.zkStateReader.aliasesManager.update(); // nocommit - hate this
     }
+    final String async = message.getStr(ASYNC);
+    Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
+    List<ReplicaPosition> replicaPositions = null;
     final Aliases aliases = ocmh.zkStateReader.getAliases();
     final String collectionName = message.getStr(NAME);
-    final boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
+    final boolean waitForFinalState = false;
     final String alias = message.getStr(ALIAS, collectionName);
     log.info("Create collection {}", collectionName);
     if (clusterState.hasCollection(collectionName)) {
@@ -124,6 +130,9 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "collection alias already exists: " + collectionName);
     }
 
+    final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(async, message.getStr("operation"));
+    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
     String withCollection = message.getStr(CollectionAdminParams.WITH_COLLECTION);
     String withCollectionShard = null;
     if (withCollection != null) {
@@ -155,8 +164,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
     try {
 
-      final String async = message.getStr(ASYNC);
-
       Map<String,String> collectionParams = new HashMap<>();
       Map<String,Object> collectionProps = message.getProperties();
       for (Map.Entry<String,Object> entry : collectionProps.entrySet()) {
@@ -174,55 +181,12 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
       OverseerCollectionMessageHandler.createConfNode(stateManager, configName, collectionName);
 
-      // TODO need to make this makePath calls efficient and not use zkSolrClient#makePath
-      for (String shardName : shardNames) {
-        try {
-          stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + shardName, null, CreateMode.PERSISTENT, false);
-          // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect", null, CreateMode.PERSISTENT, false);
-          stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
-          stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
-          stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
-          stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
-        } catch (AlreadyExistsException e) {
-          // okay
-        }
-      }
-
-      if (log.isDebugEnabled()) log.debug("Offer create operation to Overseer queue");
-      ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
-
-      // nocommit
-      // wait for a while until we see the collection
-
-      ocmh.zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (n, c) -> c != null);
-
-      // refresh cluster state
-      clusterState = ocmh.zkStateReader.getClusterState();
-
-      List<ReplicaPosition> replicaPositions = null;
-      //      try {
-      //        replicaPositions = buildReplicaPositions(ocmh.cloudManager, clusterState,
-      //                clusterState.getCollection(collectionName), message, shardNames, sessionWrapper);
-      //      } catch (Exception e) {
-      //        ParWork.propegateInterrupt(e);
-      //        SolrException exp = new SolrException(ErrorCode.SERVER_ERROR, "call(ClusterState=" + clusterState + ", ZkNodeProps=" + message + ", NamedList=" + results + ")", e);
-      //        try {
-      //          ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
-      //          new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
-      //          // unwrap the exception
-      //        } catch (Exception e1) {
-      //          ParWork.propegateInterrupt(e1);
-      //          exp.addSuppressed(e1);
-      //        }
-      //        throw exp;
-      //      }
-
-      DocCollection docCollection = clusterState.getCollection(collectionName);
+      DocCollection docCollection = buildDocCollection(stateManager, message, true);
+      clusterState = clusterState.copyWith(collectionName, docCollection);
       try {
-        replicaPositions = buildReplicaPositions(cloudManager, clusterState, docCollection, message, shardNames);
+        replicaPositions = buildReplicaPositions(cloudManager, message, shardNames);
       } catch (Exception e) {
-        ZkNodeProps deleteMessage = new ZkNodeProps("name", collectionName);
-        new DeleteCollectionCmd(ocmh).call(clusterState, deleteMessage, results);
+        log.error("", e);
         // unwrap the exception
         throw new SolrException(ErrorCode.BAD_REQUEST, e.getMessage(), e.getCause());
       }
@@ -232,29 +196,27 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       //        throw new SolrException(ErrorCode.SERVER_ERROR, "No positions found to place replicas " + replicaPositions);
       //      }
 
-      final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(async, message.getStr("operation"));
       if (log.isDebugEnabled()) {
-        log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} , message : {2}", collectionName, shardNames, message));
+        log.debug(formatString("Creating SolrCores for new collection {0}, shardNames {1} replicaCount {2}, message : {3}", collectionName, shardNames, replicaPositions.size(), message));
       }
-      Map<String,ShardRequest> coresToCreate = new LinkedHashMap<>();
-      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String nodeName = replicaPosition.node;
 
         if (withCollection != null) {
           // check that we have a replica of `withCollection` on this node and if not, create one
-          DocCollection collection = clusterState.getCollection(withCollection);
-          List<Replica> replicas = collection.getReplicas(nodeName);
+          DocCollection wcollection = clusterState.getCollection(withCollection);
+          List<Replica> replicas = wcollection.getReplicas(nodeName);
           if (replicas == null || replicas.isEmpty()) {
             ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toString(), ZkStateReader.COLLECTION_PROP, withCollection, ZkStateReader.SHARD_ID_PROP, withCollectionShard,
                 "node", nodeName, CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.TRUE.toString()); // set to true because we want `withCollection` to be ready after this collection is created
-            // TODO: prob want too look into this being parallel or always async
-            new AddReplicaCmd(ocmh).call(clusterState, props, results);
-            clusterState = zkStateReader.getClusterState(); // refresh
+
+            new AddReplicaCmd(ocmh, true).call(clusterState, props, results);
+            clusterState = new SliceMutator(cloudManager).addReplica(clusterState, props);
           }
         }
-
-        String coreName = Assign.buildSolrCoreName(cloudManager.getDistribStateManager(), docCollection, replicaPosition.shard, replicaPosition.type, true);
+        DocCollection coll = clusterState.getCollectionOrNull(collectionName);
+        String coreName = Assign.buildSolrCoreName(coll, collectionName, replicaPosition.shard, replicaPosition.type);
         log.info(formatString("Creating core {0} as part of shard {1} of collection {2} on {3}", coreName, replicaPosition.shard, collectionName, nodeName));
 
         String baseUrl = zkStateReader.getBaseUrlForNodeName(nodeName);
@@ -264,14 +226,17 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         log.info("Base url for replica={}", baseUrl);
 
         ZkNodeProps props = new ZkNodeProps();
-        props.getProperties().putAll(message.getProperties());
+        //props.getProperties().putAll(message.getProperties());
         ZkNodeProps addReplicaProps = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toString(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP,
             replicaPosition.shard, ZkStateReader.CORE_NAME_PROP, coreName, ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(), ZkStateReader.BASE_URL_PROP, baseUrl, ZkStateReader.NODE_NAME_PROP,
-            nodeName, ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(), ZkStateReader.NUM_SHARDS_PROP, message.getStr(ZkStateReader.NUM_SHARDS_PROP), "shards", message.getStr("shards"),
-            CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
-        props.getProperties().putAll(addReplicaProps.getProperties());
+            nodeName, "node", nodeName, ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(), ZkStateReader.NUM_SHARDS_PROP, message.getStr(ZkStateReader.NUM_SHARDS_PROP), "shards",
+            message.getStr("shards"), CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState)); props.getProperties().putAll(addReplicaProps.getProperties());
         if (log.isDebugEnabled()) log.debug("Sending state update to populate clusterstate with new replica {}", props);
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
+
+        clusterState = new AddReplicaCmd(ocmh, true).call(clusterState, props, results).clusterState;
+        log.info("CreateCollectionCmd after add replica clusterstate={}", clusterState);
+
+        //clusterState = new SliceMutator(cloudManager).addReplica(clusterState, props);
 
         // Need to create new params for each request
         ModifiableSolrParams params = new ModifiableSolrParams();
@@ -282,6 +247,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         params.set(CoreAdminParams.COLLECTION, collectionName);
         params.set(CoreAdminParams.SHARD, replicaPosition.shard);
         params.set(ZkStateReader.NUM_SHARDS_PROP, shardNames.size());
+        params.set(ZkStateReader.NODE_NAME_PROP, nodeName);
         params.set(CoreAdminParams.NEW_COLLECTION, "true");
         params.set(CoreAdminParams.REPLICA_TYPE, replicaPosition.type.name());
 
@@ -302,70 +268,19 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
 
         coresToCreate.put(coreName, sreq);
       }
-
-      // wait for all replica entries to be created
-      Map<String,Replica> replicas = new ConcurrentHashMap<>();
-
-      zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS,
-          expectedReplicas(coresToCreate.size(), replicas)); // nocommit - timeout - keep this below containing timeouts - need central timeout stuff
-      // TODO what if replicas comes back wrong?
-      if (replicas.size() > 0) {
-        for (Map.Entry<String,ShardRequest> e : coresToCreate.entrySet()) {
-          ShardRequest sreq = e.getValue();
-          Replica replica = null;
-          for (Replica rep : replicas.values()) {
-            log.info("cmp {} {} {} {}", e.getKey(), sreq.shards[0], rep.getCoreName(), rep.getBaseUrl());
-            if (rep.getCoreName().equals(e.getKey()) && rep.getBaseUrl().equals(sreq.shards[0])) {
-              sreq.params.set(CoreAdminParams.CORE_NODE_NAME, rep.getName());
-              replica = rep;
-              break;
-            }
-          }
-
-          if (sreq.params.get(CoreAdminParams.CORE_NODE_NAME) == null || replica == null) {
-             continue;
-          }
-
-          log.info("Submit request to shard for for replica={}", sreq.actualShards != null ? Arrays.asList(sreq.actualShards) : "null");
-          shardHandler.submit(sreq, sreq.shards[0], sreq.params);
-        }
-      }
-
-      shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
-      @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
-      if (failure) {
-        // Let's cleanup as we hit an exception
-        // We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
-        // element, which may be interpreted by the user as a positive ack
-        ocmh.cleanupCollection(collectionName, new NamedList<Object>());
-        log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
-        throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName + "\n" + results);
-      } else {
-        log.debug("Finished create command on all shards for collection: {}", collectionName);
-
-        // Emit a warning about production use of data driven functionality
-        boolean defaultConfigSetUsed = message.getStr(COLL_CONF) == null || message.getStr(COLL_CONF).equals(ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME);
-        if (defaultConfigSetUsed) {
-          results.add("warning",
-              "Using _default configset. Data driven schema functionality" + " is enabled by default, which is NOT RECOMMENDED for production use. To turn it off:" + " curl http://{host:port}/solr/"
-                  + collectionName + "/config -d '{\"set-user-property\": {\"update.autoCreateFields\":\"false\"}}'");
-        }
-        if (async == null) {
-          zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS, BaseCloudSolrClient.expectedShardsAndActiveReplicas(shardNames.size(), replicaPositions.size()));
-        }
+      log.info("Sending create call for {} replicas", coresToCreate.size());
+      for (Map.Entry<String,ShardRequest> e : coresToCreate.entrySet()) {
+        ShardRequest sreq = e.getValue();
+        log.info("Submit request to shard for for replica coreName={} total requests={} shards={}", e.getKey(), coresToCreate.size(),
+            sreq.actualShards != null ? Arrays.asList(sreq.actualShards) : "null");
+        // nocommit - work out parallel / async 100%
+        shardHandler.submit(sreq, sreq.shards[0], sreq.params);
       }
 
       // modify the `withCollection` and store this new collection's name with it
       if (withCollection != null) {
         ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, MODIFYCOLLECTION.toString(), ZkStateReader.COLLECTION_PROP, withCollection, CollectionAdminParams.COLOCATED_WITH, collectionName);
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
-        try {
-          zkStateReader.waitForState(withCollection, 30, TimeUnit.SECONDS, (collectionState) -> collectionName.equals(collectionState.getStr(COLOCATED_WITH)));
-        } catch (TimeoutException e) {
-          log.warn("Timed out waiting to see the {} property set on collection: {}", COLOCATED_WITH, withCollection);
-          // maybe the overseer queue is backed up, we don't want to fail the create request
-          // because of this time out, continue
-        }
+        clusterState = new CollectionMutator(cloudManager).modifyCollection(clusterState, props);
       }
 
       // create an alias pointing to the new collection, if different from the collectionName
@@ -383,16 +298,70 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       log.error("Exception creating collection", ex);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, ex);
     }
-    return null;
+
+    log.info("CreateCollectionCmd clusterstate={}", clusterState);
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    if (results.get("failure") == null && results.get("exception") == null) {
+      List<ReplicaPosition> finalReplicaPositions = replicaPositions;
+      response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+        @Override
+        public AddReplicaCmd.Response call() {
+          try {
+            shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
+          } catch (KeeperException e) {
+            log.error("", e);
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+          } catch (InterruptedException e) {
+            ParWork.propagateInterrupt(e);
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+          }
+          //  nocommit - put this in finalizer and finalizer after all calls to allow parallel and forward momentum
+
+          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+          @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
+          if (failure) {
+            //        // Let's cleanup as we hit an exception
+            //        // We shouldn't be passing 'results' here for the cleanup as the response would then contain 'success'
+            //        // element, which may be interpreted by the user as a positive ack
+            //        // nocommit review
+            try {
+              response.clusterState =  ocmh.cleanupCollection(collectionName, new NamedList<Object>()).clusterState;
+            } catch (Exception e) {
+              log.error("Exception trying to clean up collection after fail {}", collectionName);
+            }
+            log.info("Cleaned up artifacts for failed create collection for [{}]", collectionName);
+                   //throw new SolrException(ErrorCode.BAD_REQUEST, "Underlying core creation failed while creating collection: " + collectionName + "\n" + results);
+          } else {
+            if (log.isDebugEnabled()) log.debug("Finished create command on all shards for collection: {}", collectionName);
+
+            // Emit a warning about production use of data driven functionality
+            boolean defaultConfigSetUsed = message.getStr(COLL_CONF) == null || message.getStr(COLL_CONF).equals(ConfigSetsHandlerApi.DEFAULT_CONFIGSET_NAME);
+            if (defaultConfigSetUsed) {
+              results.add("warning",
+                  "Using _default configset. Data driven schema functionality" + " is enabled by default, which is NOT RECOMMENDED for production use. To turn it off:" + " curl http://{host:port}/solr/"
+                      + collectionName + "/config -d '{\"set-user-property\": {\"update.autoCreateFields\":\"false\"}}'");
+            }
+
+          }
+
+
+          zkStateReader.waitForActiveCollection(collectionName, 10, TimeUnit.SECONDS, shardNames.size(), finalReplicaPositions.size());
+          return response;
+        }
+      };
+    }
+    response.clusterState = clusterState;
+    return response;
   }
 
-  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager, ClusterState clusterState,
-                                                            DocCollection docCollection,
+  public static List<ReplicaPosition> buildReplicaPositions(SolrCloudManager cloudManager,
                                                             ZkNodeProps message,
                                                             List<String> shardNames) throws IOException, InterruptedException, Assign.AssignmentException {
     if (log.isDebugEnabled()) {
-      // nocommit
-     // log.debug("buildReplicaPositions(SolrCloudManager cloudManager={}, ClusterState clusterState={}, DocCollection docCollection={}, ZkNodeProps message={}, List<String> shardNames={}, AtomicReference<PolicyHelper.SessionWrapper> sessionWrapper={}) - start", cloudManager, clusterState, docCollection, message, shardNames, sessionWrapper);
+      log.debug("buildReplicaPositions(SolrCloudManager cloudManager={}, ZkNodeProps message={}, List<String> shardNames={}) - start",
+          cloudManager, message, shardNames);
     }
 
     final String collectionName = message.getStr(NAME);
@@ -418,7 +387,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       throw new SolrException(ErrorCode.SERVER_ERROR, e);
     }
     if (nodeList.isEmpty()) {
-      log.warn("It is unusual to create a collection ("+collectionName+") without cores. liveNodes={} message={}", clusterState.getLiveNodes(), message);
+      log.warn("It is unusual to create a collection ("+collectionName+") without cores. message={}",  message);
 
       replicaPositions = new ArrayList<>();
     } else {
@@ -433,24 +402,6 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
                 + "). It's unusual to run two replica of the same slice on the same Solr-instance.");
       }
 
-      int maxShardsAllowedToCreate = maxShardsPerNode == Integer.MAX_VALUE ?
-              Integer.MAX_VALUE :
-              maxShardsPerNode * nodeList.size();
-      int requestedShardsToCreate = numSlices * totalNumReplicas;
-      if (maxShardsAllowedToCreate < requestedShardsToCreate) {
-        String msg = "Cannot create collection " + collectionName + ". Value of "
-                + MAX_SHARDS_PER_NODE + " is " + maxShardsPerNode
-                + ", and the number of nodes currently live or live and part of your "+ZkStateReader.CREATE_NODE_SET+" is " + nodeList.size()
-                + ". This allows a maximum of " + maxShardsAllowedToCreate
-                + " to be created. Value of " + ZkStateReader.NUM_SHARDS_PROP + " is " + numSlices
-                + ", value of " + NRT_REPLICAS + " is " + numNrtReplicas
-                + ", value of " + TLOG_REPLICAS + " is " + numTlogReplicas
-                + " and value of " + PULL_REPLICAS + " is " + numPullReplicas
-                + ". This requires " + requestedShardsToCreate
-                + " shards to be created (higher than the allowed number)";
-
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, msg);
-      }
       Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
               .forCollection(collectionName)
               .forShard(shardNames)
@@ -460,7 +411,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
               .onNodes(nodeList)
               .build();
       Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(cloudManager);
-      Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, docCollection);
+      Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
       replicaPositions = assignStrategy.assign(cloudManager, assignRequest);
     }
 
@@ -482,7 +433,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
     }
   }
 
-  public static DocCollection buildDocCollection(ZkNodeProps message, boolean withDocRouter) {
+  public static DocCollection buildDocCollection(DistribStateManager stateManager, ZkNodeProps message, boolean withDocRouter) {
     log.info("buildDocCollection {}", message);
     String cName = message.getStr(NAME);
     DocRouter router = null;
@@ -549,6 +500,26 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       }
     }
 
+    // TODO need to make this makePath calls efficient and not use zkSolrClient#makePath
+    for (String shardName : slices.keySet()) {
+      try {
+        stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/" + shardName, null, CreateMode.PERSISTENT, false);
+        // stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect", null, CreateMode.PERSISTENT, false);
+        stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect/" + shardName, null, CreateMode.PERSISTENT, false);
+        stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leader_elect/" + shardName + "/election", null, CreateMode.PERSISTENT, false);
+        stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + cName + "/leaders/" + shardName, null, CreateMode.PERSISTENT, false);
+        stateManager.makePath(ZkStateReader.COLLECTIONS_ZKNODE  + "/" + cName + "/terms/" + shardName, ZkStateReader.emptyJson, CreateMode.PERSISTENT, false);
+      } catch (AlreadyExistsException e) {
+        // okay
+      } catch (InterruptedException e) {
+        ParWork.propagateInterrupt(e);
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      } catch (KeeperException e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      } catch (IOException e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, e);
+      }
+    }
     DocCollection newCollection = new DocCollection(cName,
             slices, collectionProps, router, 0);
 
@@ -710,7 +681,7 @@ public class CreateCollectionCmd implements OverseerCollectionMessageHandler.Cmd
     if (numShards == null)
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "numShards" + " is a required param");
     for (int i = 0; i < numShards; i++) {
-      final String sliceName = "shard" + (i + 1);
+      final String sliceName = "s" + (i + 1);
       shardNames.add(sliceName);
     }
   }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
index e3aa49a..904a018 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateShardCmd.java
@@ -18,9 +18,14 @@ package org.apache.solr.cloud.api.collections;
 
 
 import java.lang.invoke.MethodHandles;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.DocCollection;
@@ -30,6 +35,9 @@ import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.handler.admin.ConfigSetsHandlerApi;
+import org.apache.solr.handler.component.ShardHandler;
+import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -39,6 +47,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
+import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
 import static org.apache.solr.common.params.CollectionAdminParams.FOLLOW_ALIASES;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 
@@ -52,7 +61,11 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked", "rawtypes"})
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+    return addShard(clusterState, clusterState.getCollection(message.getStr(COLLECTION_PROP)), message, results);
+  }
+
+  AddReplicaCmd.Response addShard(ClusterState clusterState, DocCollection collection, ZkNodeProps message, NamedList results) throws Exception {
     String extCollectionName = message.getStr(COLLECTION_PROP);
     String sliceName = message.getStr(SHARD_ID_PROP);
     boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
@@ -68,7 +81,6 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
     } else {
       collectionName = extCollectionName;
     }
-    DocCollection collection = clusterState.getCollection(collectionName);
 
     int numNrtReplicas = message.getInt(NRT_REPLICAS, message.getInt(REPLICATION_FACTOR, collection.getInt(NRT_REPLICAS, collection.getInt(REPLICATION_FACTOR, 1))));
     int numPullReplicas = message.getInt(PULL_REPLICAS, collection.getInt(PULL_REPLICAS, 0));
@@ -78,15 +90,10 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, NRT_REPLICAS + " + " + TLOG_REPLICAS + " must be greater than 0");
     }
 
-    //ZkStateReader zkStateReader = ocmh.zkStateReader;
-    ocmh.overseer.offerStateUpdate(Utils.toJSON(message));
-    // wait for a while until we see the shard
-    //ocmh.waitForNewShard(collectionName, sliceName);
-    // wait for a while until we see the shard and update the local view of the cluster state
-    ocmh.waitForNewShard(collectionName, sliceName);
+    // create shard
+    clusterState = new CollectionMutator(ocmh.cloudManager).createShard(clusterState, message);
 
-    // refresh clusterstate
-    clusterState = ocmh.zkStateReader.getClusterState();
+    log.info("After create shard {}", clusterState);
 
     String async = message.getStr(ASYNC);
     ZkNodeProps addReplicasProps = new ZkNodeProps(
@@ -102,36 +109,87 @@ public class CreateShardCmd implements OverseerCollectionMessageHandler.Cmd {
     ocmh.addPropertyParams(message, propertyParams);
     addReplicasProps = addReplicasProps.plus(propertyParams);
     if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
+
+    final String asyncId = message.getStr(ASYNC);
+    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+
+    OverseerCollectionMessageHandler.ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+
     final NamedList addResult = new NamedList();
+    AddReplicaCmd.Response resp;
     try {
       //ocmh.addReplica(zkStateReader.getClusterState(), addReplicasProps, addResult, () -> {
-      ocmh.addReplica(clusterState, addReplicasProps, addResult, () -> {
-        Object addResultFailure = addResult.get("failure");
-        if (addResultFailure != null) {
-          SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
-          if (failure == null) {
-            failure = new SimpleOrderedMap();
-            results.add("failure", failure);
-          }
-          failure.addAll((NamedList) addResultFailure);
-        } else {
-          SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
-          if (success == null) {
-            success = new SimpleOrderedMap();
-            results.add("success", success);
-          }
-          success.addAll((NamedList) addResult.get("success"));
-        }
-      });
+      resp = new AddReplicaCmd(ocmh)
+          .addReplica(clusterState, addReplicasProps, shardHandler, shardRequestTracker, results); //ocmh.addReplica(clusterState, addReplicasProps, addResult).clusterState;
+      clusterState = resp.clusterState;
     } catch (Assign.AssignmentException e) {
       // clean up the slice that we created
-      ZkNodeProps deleteShard = new ZkNodeProps(COLLECTION_PROP, collectionName, SHARD_ID_PROP, sliceName, ASYNC, async);
-      new DeleteShardCmd(ocmh).call(clusterState, deleteShard, results);
+      // nocommit
+//      ZkNodeProps deleteShard = new ZkNodeProps(COLLECTION_PROP, collectionName, SHARD_ID_PROP, sliceName, ASYNC, async);
+//      new DeleteShardCmd(ocmh).call(clusterState, deleteShard, results);
       throw e;
     }
 
+//    () -> {
+//      Object addResultFailure = addResult.get("failure");
+//      if (addResultFailure != null) {
+//        SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
+//        if (failure == null) {
+//          failure = new SimpleOrderedMap();
+//          results.add("failure", failure);
+//        }
+//        failure.addAll((NamedList) addResultFailure);
+//      } else {
+//        SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
+//        if (success == null) {
+//          success = new SimpleOrderedMap();
+//          results.add("success", success);
+//        }
+//        success.addAll((NamedList) addResult.get("success"));
+//      }
+//    }
+
     log.info("Finished create command on all shards for collection: {}", collectionName);
-    return null;
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+      @Override
+      public AddReplicaCmd.Response call() {
+        try {
+          shardRequestTracker.processResponses(results, shardHandler, false, null, Collections.emptySet());
+        } catch (KeeperException e) {
+          log.error("", e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        } catch (InterruptedException e) {
+          ParWork.propagateInterrupt(e);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+        }
+        //  nocommit - put this in finalizer and finalizer after all calls to allow parallel and forward momentum
+
+        if (resp.asyncFinalRunner != null) {
+          try {
+            resp.asyncFinalRunner.call();
+          } catch (Exception e) {
+            log.error("Exception waiting for active replicas", e);
+          }
+        }
+
+        @SuppressWarnings({"rawtypes"}) boolean failure = results.get("failure") != null && ((SimpleOrderedMap) results.get("failure")).size() > 0;
+        if (failure) {
+
+        } else {
+
+        }
+
+        //ocmh.zkStateReader.waitForActiveCollection(collectionName, 10, TimeUnit.SECONDS, shardNames.size(), finalReplicaPositions.size());
+        AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+        return response;
+      }
+
+    };
+
+    response.clusterState = clusterState;
+    return response;
   }
 
 }
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 79052c0..3e85afc 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
@@ -66,7 +66,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extCollectionName =  message.getStr(COLLECTION_PROP);
     boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
 
@@ -103,7 +103,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
       for (Replica replica : slice.getReplicas()) {
         if (replica.getState() != State.ACTIVE) {
           if (log.isInfoEnabled()) {
-            log.info("Replica {} is not active. Hence not sending the createsnapshot request", replica.getCoreName());
+            log.info("Replica {} is not active. Hence not sending the createsnapshot request", replica.getName());
           }
           continue; // Since replica is not active - no point sending a request.
         }
@@ -142,7 +142,7 @@ public class CreateSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
         // to have latest state.
         String coreName = (String)resp.get(CoreAdminParams.CORE);
         Slice slice = shardByCoreName.remove(coreName);
-        boolean leader = (slice.getLeader() != null && slice.getLeader().getCoreName().equals(coreName));
+        boolean leader = (slice.getLeader() != null && slice.getLeader().getName().equals(coreName));
         resp.add(SolrSnapshotManager.SHARD_ID, slice.getName());
         resp.add(SolrSnapshotManager.LEADER, leader);
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
index e65c071..59b08f8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteAliasCmd.java
@@ -33,7 +33,7 @@ public class DeleteAliasCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String aliasName = message.getStr(NAME);
 
     ZkStateReader zkStateReader = ocmh.zkStateReader;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
index 8eca9ec..1666912 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteCollectionCmd.java
@@ -19,12 +19,14 @@
 package org.apache.solr.cloud.api.collections;
 
 import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.overseer.ClusterStateMutator;
 import org.apache.solr.common.NonExistentCoreException;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Aliases;
 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.ReplicaPosition;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
@@ -67,7 +69,7 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     log.info("delete collection called");
     Object o = message.get(MaintainRoutedAliasCmd.INVOKED_BY_ROUTED_ALIAS);
     if (o != null) {
@@ -104,14 +106,6 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       SolrZkClient zkClient = zkStateReader.getZkClient();
       SolrSnapshotManager.cleanupCollectionLevelSnapshots(zkClient, collection);
 
-      if (zkStateReader.getClusterState().getCollectionOrNull(collection) == null) {
-        if (zkStateReader.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection)) {
-          // if the collection is not in the clusterstate, but is listed in zk, do nothing, it will just
-          // be removed in the finally - we cannot continue, because the below code will error if the collection
-          // is not in the clusterstate
-          return null;
-        }
-      }
       // remove collection-level metrics history
       if (deleteHistory) {
         MetricsHistoryHandler historyHandler = ocmh.overseer.getCoreContainer().getMetricsHistoryHandler();
@@ -136,6 +130,8 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Could not find collection");
       }
 
+
+      clusterState = new ClusterStateMutator(ocmh.cloudManager) .deleteCollection(clusterState, collection);
       @SuppressWarnings({"unchecked"})
       List<Replica> failedReplicas = ocmh.collectionCmd(internalMsg, params, results, null, asyncId, okayExceptions);
 
@@ -144,18 +140,6 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
       }
 
     } finally {
-      try {
-        log.info("Send DELETE operation to Overseer collection={}", collection);
-        ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETE.toLower(), NAME, collection);
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
-        // wait for a while until we don't see the collection
-        if (zkStateReader.getClusterState().getCollectionOrNull(collection) != null) {
-          zkStateReader.waitForState(collection, 5, TimeUnit.SECONDS, (collectionState) -> collectionState == null);
-        }
-      } catch (Exception e) {
-        log.error("Exception while removing collection", e);
-      }
 
       // make sure it's gone again after cores have been removed
       try {
@@ -174,7 +158,22 @@ public class DeleteCollectionCmd implements OverseerCollectionMessageHandler.Cmd
         });
       }
     }
-    return null;
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    if (results.get("failure") == null && results.get("exception") == null) {
+
+      response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+        @Override
+        public AddReplicaCmd.Response call() {
+          // TODO: wait for delete collection?
+          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+          return response;
+        }
+      };
+    }
+    response.clusterState = clusterState;
+    return response;
   }
 
   // This method returns the single collection aliases to delete, if present, or null
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
index 73e6da8..5617b9a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/DeleteNodeCmd.java
@@ -51,7 +51,7 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     ocmh.checkRequired(message, "node");
     String node = message.getStr("node");
     List<ZkNodeProps> sourceReplicas = ReplaceNodeCmd.getReplicasOfNode(node, state);
@@ -111,15 +111,8 @@ public class DeleteNodeCmd implements OverseerCollectionMessageHandler.Cmd {
           NamedList deleteResult = new NamedList();
           try {
             if (async != null) sourceReplica = sourceReplica.plus(ASYNC, async);
-            ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult, () -> {
-              if (deleteResult.get("failure") != null) {
-                synchronized (results) {
-
-                  results.add("failure", String.format(Locale.ROOT, "Failed to delete replica for collection=%s shard=%s" +
-                          " on node=%s", coll, shard, node));
-                }
-              }
-            });
+            // nocommit - return results from deleteReplica cmd
+            ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, sourceReplica.plus("parallel", "true"), deleteResult);
           } catch (KeeperException e) {
             log.warn("Error deleting ", e);
           } catch (InterruptedException e) {
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 cd82c38..dc6c122 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
@@ -21,14 +21,16 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
 
+import org.apache.solr.client.solrj.response.Cluster;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.Cmd;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -58,54 +60,63 @@ import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 public class DeleteReplicaCmd implements Cmd {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final OverseerCollectionMessageHandler ocmh;
+  private final boolean onlyUpdateState;
 
   public DeleteReplicaCmd(OverseerCollectionMessageHandler ocmh) {
+    this.onlyUpdateState = false;
+    this.ocmh = ocmh;
+  }
+
+  public DeleteReplicaCmd(OverseerCollectionMessageHandler ocmh , boolean onlyUpdateState) {
+    this.onlyUpdateState = onlyUpdateState;
     this.ocmh = ocmh;
   }
 
   @Override
   @SuppressWarnings("unchecked")
 
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
-    AddReplicaCmd.Response response = deleteReplica(clusterState, message, results, null);
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+    AddReplicaCmd.Response response = deleteReplica(clusterState, message, results);
     if (response == null) return null;
-    return response.asyncFinalRunner;
+    return response;
   }
 
 
   @SuppressWarnings("unchecked")
-  AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+  AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws KeeperException, InterruptedException {
 
     log.info("deleteReplica() : {}", Utils.toJSONString(message));
 
     //If a count is specified the strategy needs be different
     if (message.getStr(COUNT_PROP) != null) {
-      Set<String> pickReplicasTobeDeleted = deleteReplicaBasedOnCount(clusterState, message, results, onComplete);
-      if (pickReplicasTobeDeleted != null && pickReplicasTobeDeleted.size() > 0) {
-        if (results.get("failure") == null && results.get("exception") == null) {
-          String collectionName = message.getStr(COLLECTION_PROP);
-          String shard = message.getStr(SHARD_ID_PROP);
-          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
-          pickReplicasTobeDeleted.forEach(replicaName -> {
-
-            response.asyncFinalRunner = new Runnable() {
-              @Override
-              public void run() {
-                try {
-                  waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
-                } catch (Exception e) {
-                  log.error("", e);
-                }
+      AddReplicaCmd.Response resp = deleteReplicaBasedOnCount(clusterState, message, results);
+      clusterState = resp.clusterState;
+      AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+      if (results.get("failure") == null && results.get("exception") == null) {
+        response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+          @Override
+          public AddReplicaCmd.Response call() {
+            if (resp.asyncFinalRunner != null) {
+              try {
+                resp.asyncFinalRunner.call();
+              } catch (Exception e) {
+                log.error("Exception running delete replica finalizers", e);
               }
-            };
-          });
-          return response;
-        }
-        return null;
-      } else {
-        return null;
+            }
+            //          try {
+            //            waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
+            //          } catch (Exception e) {
+            //            log.error("", e);
+            //          }
+            AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+            return response;
+          }
+        };
       }
+      response.clusterState = clusterState;
+      return response;
     }
 
     ocmh.checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP);
@@ -128,24 +139,38 @@ public class DeleteReplicaCmd implements Cmd {
               "Invalid shard name : " +  shard + " in collection : " +  collectionName);
     }
 
-    deleteCore(slice, collectionName, replicaName, message, shard, results, onComplete);
-    waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
+    AddReplicaCmd.Response resp = deleteCore(clusterState, slice, collectionName, replicaName, message, shard, results);
+    clusterState = resp.clusterState;
+
+    if (clusterState.getCollectionOrNull(collectionName).getReplica(replicaName) != null) {
+      throw new IllegalStateException("Failed to remove replica from state " + replicaName);
+    }
 
     AddReplicaCmd.Response response = new AddReplicaCmd.Response();
 
     if (results.get("failure") == null && results.get("exception") == null) {
-      response.asyncFinalRunner = new Runnable() {
+      response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
         @Override
-        public void run() {
+        public AddReplicaCmd.Response call() {
+          if (resp.asyncFinalRunner != null) {
+            try {
+              resp.asyncFinalRunner.call();
+            } catch (Exception e) {
+              log.error("", e);
+            }
+          }
+
           try {
             waitForCoreNodeGone(collectionName, shard, replicaName, 30000);
           } catch (Exception e) {
             log.error("", e);
           }
+          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+          return response;
         }
       };
     }
-
+    response.clusterState = clusterState;
     return response;
   }
 
@@ -156,10 +181,9 @@ public class DeleteReplicaCmd implements Cmd {
    * @return
    */
   @SuppressWarnings({"unchecked"})
-  Set<String> deleteReplicaBasedOnCount(ClusterState clusterState,
+  AddReplicaCmd.Response deleteReplicaBasedOnCount(ClusterState clusterState,
                                  ZkNodeProps message,
-                                 @SuppressWarnings({"rawtypes"})NamedList results,
-                                 Runnable onComplete)
+                                 @SuppressWarnings({"rawtypes"})NamedList results)
           throws KeeperException, InterruptedException {
     ocmh.checkRequired(message, COLLECTION_PROP, COUNT_PROP);
     int count = Integer.parseInt(message.getStr(COUNT_PROP));
@@ -171,15 +195,14 @@ public class DeleteReplicaCmd implements Cmd {
     if (shard != null) {
       slice = coll.getSlice(shard);
       if (slice == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Invalid shard name : " +  shard +  " in collection : " + collectionName);
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Invalid shard name : " + shard + " in collection : " + collectionName);
       }
     }
     Set<String> replicasToBeDeleted = null;
-    Map<Slice, Set<String>> shardToReplicasMapping = new HashMap<Slice, Set<String>>();
+    Map<Slice,Set<String>> shardToReplicasMapping = new HashMap<Slice,Set<String>>();
     if (slice != null) {
       replicasToBeDeleted = pickReplicasTobeDeleted(slice, shard, collectionName, count);
-      shardToReplicasMapping.put(slice,replicasToBeDeleted);
+      shardToReplicasMapping.put(slice, replicasToBeDeleted);
     } else {
 
       //If there are many replicas left, remove the rest based on count.
@@ -189,26 +212,40 @@ public class DeleteReplicaCmd implements Cmd {
         shardToReplicasMapping.put(individualSlice, replicasToBeDeleted);
       }
     }
-
-    try (ParWork worker = new ParWork(this, false, true)) {
-
-      for (Map.Entry<Slice,Set<String>> entry : shardToReplicasMapping.entrySet()) {
-        Slice shardSlice = entry.getKey();
-        String shardId = shardSlice.getName();
-        Set<String> replicas = entry.getValue();
-        // callDeleteReplica on all replicas
-        for (String replica : replicas) {
-          if (log.isDebugEnabled()) log.debug("Deleting replica {}  for shard {} based on count {}", replica, shardId, count);
-          worker.collect("deleteCore", () -> {
-            deleteCore(shardSlice, collectionName, replica, message, shard, results, onComplete); return replica;
-
-          });
+    List<OverseerCollectionMessageHandler.Finalize> finalizers = new ArrayList<>();
+    for (Map.Entry<Slice,Set<String>> entry : shardToReplicasMapping.entrySet()) {
+      Slice shardSlice = entry.getKey();
+      String shardId = shardSlice.getName();
+      Set<String> replicas = entry.getValue();
+      // callDeleteReplica on all replicas
+      for (String replica : replicas) {
+        if (log.isDebugEnabled()) log.debug("Deleting replica {}  for shard {} based on count {}", replica, shardId, count);
+        // nocommit - DONT DO THIS ONE AT TIME
+
+        AddReplicaCmd.Response resp = deleteCore(clusterState, shardSlice, collectionName, replica, message, shard, results);
+        clusterState = resp.clusterState;
+        if (resp.asyncFinalRunner != null) {
+          finalizers.add(resp.asyncFinalRunner);
         }
-        results.add("shard_id", shardId);
-        results.add("replicas_deleted", replicas);
+
       }
+      results.add("shard_id", shardId);
+      results.add("replicas_deleted", replicas);
     }
-    return replicasToBeDeleted;
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.clusterState = clusterState;
+    response.asyncFinalRunner = () -> {
+      AddReplicaCmd.Response resp = new AddReplicaCmd.Response();
+      resp.asyncFinalRunner = () -> {
+        for (OverseerCollectionMessageHandler.Finalize finalize : finalizers) {
+          finalize.call();
+        }
+        return new AddReplicaCmd.Response();
+      };
+      return resp;
+    };
+    return response;
   }
 
 
@@ -225,12 +262,13 @@ public class DeleteReplicaCmd implements Cmd {
         break;
       }
       //Try avoiding to pick up the leader to minimize activity on the cluster.
-      if (leader.getCoreName().equals(replica.getCoreName())) {
+      if (leader.getName().equals(replica.getName())) {
         continue;
       }
       replicasToBeRemoved.add(replica.getName());
       count --;
     }
+    log.info("Found replicas to be removed {}", replicasToBeRemoved);
     return replicasToBeRemoved;
   }
 
@@ -261,7 +299,7 @@ public class DeleteReplicaCmd implements Cmd {
   }
 
   @SuppressWarnings({"unchecked"})
-  void deleteCore(Slice slice, String collectionName, String replicaName,ZkNodeProps message, String shard, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete) throws KeeperException, InterruptedException {
+  AddReplicaCmd.Response deleteCore(ClusterState clusterState, Slice slice, String collectionName, String replicaName, ZkNodeProps message, String shard, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
     log.info("delete core {}", replicaName);
     Replica replica = slice.getReplica(replicaName);
     if (replica == null) {
@@ -274,61 +312,69 @@ public class DeleteReplicaCmd implements Cmd {
 
     // If users are being safe and only want to remove a shard if it is down, they can specify onlyIfDown=true
     // on the command.
-    if (Boolean.parseBoolean(message.getStr(OverseerCollectionMessageHandler.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-              "Attempted to remove replica : " + collectionName + "/"  + shard + "/" + replicaName +
-              " with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'");
-    }
-
-    ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
-    String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-    String asyncId = message.getStr(ASYNC);
-
-    ModifiableSolrParams params = new ModifiableSolrParams();
-    params.add(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.UNLOAD.toString());
-    params.add(CoreAdminParams.CORE, core);
-
-    params.set(CoreAdminParams.DELETE_INDEX, message.getBool(CoreAdminParams.DELETE_INDEX, true));
-    params.set(CoreAdminParams.DELETE_INSTANCE_DIR, message.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, true));
-    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());
-
-    try {
-      ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
-    } catch (Exception e) {
-      ParWork.propagateInterrupt(e);
-      results.add("failure", "Could not complete delete " + e.getMessage());
-    }
+//    if (Boolean.parseBoolean(message.getStr(OverseerCollectionMessageHandler.ONLY_IF_DOWN)) && replica.getState() != Replica.State.DOWN) {
+//      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+//              "Attempted to remove replica : " + collectionName + "/"  + shard + "/" + replicaName +
+//              " with onlyIfDown='true', but state is '" + replica.getStr(ZkStateReader.STATE_PROP) + "'");
+//    }
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    ZkNodeProps rep = new ZkNodeProps();
+    rep.getProperties().put("replica", replicaName);
+    rep.getProperties().put("collection", replica.getCollection());
+    rep.getProperties().put(ZkStateReader.BASE_URL_PROP, replica.getBaseUrl());
+
+    log.info("Before slice remove replica {} {}", rep, clusterState);
+    clusterState = new SliceMutator(ocmh.cloudManager).removeReplica(clusterState, rep);
+    log.info("After slice remove replica {} {}", rep, clusterState);
+
+    if (!onlyUpdateState) {
+      ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
+      String asyncId = message.getStr(ASYNC);
+
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.add(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.UNLOAD.toString());
+      params.add(CoreAdminParams.CORE, replicaName);
+
+      params.set(CoreAdminParams.DELETE_INDEX, message.getBool(CoreAdminParams.DELETE_INDEX, true));
+      params.set(CoreAdminParams.DELETE_INSTANCE_DIR, message.getBool(CoreAdminParams.DELETE_INSTANCE_DIR, true));
+      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());
+
+      //    try {
+      //      ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
+      //    } catch (Exception e) {
+      //      ParWork.propagateInterrupt(e);
+      //      results.add("failure", "Could not complete delete " + e.getMessage());
+      //    }
+
+      final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+      if (isLive) {
+        response.asyncFinalRunner = () -> {
+          shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
+          return new AddReplicaCmd.Response();
+        };
 
-    final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
-    if (isLive) {
-      shardRequestTracker.sendShardRequest(replica.getNodeName(), params, shardHandler);
-    }
+      }
 
-    try {
       try {
-        if (isLive) {
-          shardRequestTracker.processResponses(results, shardHandler, false, null);
-          // try and ensure core info is removed from cluster state
-        }
-
-      } catch (Exception e) {
-        ParWork.propagateInterrupt(e);
-        results.add("failure", "Could not complete delete " + e.getMessage());
-      } finally {
         try {
-          ocmh.deleteCoreNode(collectionName, replicaName, replica, core);
+          if (isLive) {
+            shardRequestTracker.processResponses(results, shardHandler, false, null);
+            // try and ensure core info is removed from cluster state
+          }
+
         } catch (Exception e) {
           ParWork.propagateInterrupt(e);
           results.add("failure", "Could not complete delete " + e.getMessage());
         }
-        if (onComplete != null) onComplete.run();
+      } catch (Exception ex) {
+        throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Error waiting for corenode gone", ex);
       }
-    } catch (Exception ex) {
-      throw new SolrException(SolrException.ErrorCode.UNKNOWN, "Error waiting for corenode gone", ex);
     }
+    response.clusterState = clusterState;
+    return response;
   }
 
   boolean waitForCoreNodeGone(String collectionName, String shard, String replicaName, int timeoutms) throws InterruptedException {
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 aa9bc6c..40c6744 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
@@ -23,13 +23,13 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
 
 import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.overseer.CollectionMutator;
 import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -67,7 +67,7 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extCollectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
     String sliceId = message.getStr(ZkStateReader.SHARD_ID_PROP);
 
@@ -104,11 +104,13 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
       propMap.put(sliceId, Slice.State.CONSTRUCTION.toString());
       propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
       ZkNodeProps m = new ZkNodeProps(propMap);
-      ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
+      clusterState = new SliceMutator(ocmh.cloudManager).updateShardState(clusterState, message);
     }
 
-    String asyncId = message.getStr(ASYNC);
+    slice = clusterState.getCollection(collectionName).getSlice(sliceId);
 
+    String asyncId = message.getStr(ASYNC);
+    List<OverseerCollectionMessageHandler.Finalize> finalizers = new ArrayList<>();
     try {
       List<ZkNodeProps> replicas = getReplicasForSlice(collectionName, slice);
 
@@ -117,25 +119,15 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
         if (log.isInfoEnabled()) {
           log.info("Deleting replica for collection={} shard={} on node={}", replica.getStr(COLLECTION_PROP), replica.getStr(SHARD_ID_PROP), replica.getStr(CoreAdminParams.NODE));
         }
-        @SuppressWarnings({"rawtypes"})
-        NamedList deleteResult = new NamedList();
+        @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
         try {
-          ((DeleteReplicaCmd)ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, replica, deleteResult, () -> {
-
-            if (deleteResult.get("failure") != null) {
-              synchronized (results) {
-                results.add("failure", String.format(Locale.ROOT, "Failed to delete replica for collection=%s shard=%s" +
-                    " on node=%s", replica.getStr(COLLECTION_PROP), replica.getStr(SHARD_ID_PROP), replica.getStr(NODE_NAME_PROP)));
-              }
-            }
-            @SuppressWarnings({"rawtypes"})
-            SimpleOrderedMap success = (SimpleOrderedMap) deleteResult.get("success");
-            if (success != null) {
-              synchronized (results)  {
-                results.add("success", success);
-              }
-            }
-          });
+
+          // nocommit - return results from deleteReplica cmd
+          AddReplicaCmd.Response resp = ((DeleteReplicaCmd) ocmh.commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, replica, deleteResult);
+          if (resp.asyncFinalRunner != null) {
+            finalizers.add(resp.asyncFinalRunner);
+          }
+          clusterState = resp.clusterState;
         } catch (KeeperException e) {
           log.warn("Error deleting replica: {}", r, e);
           throw e;
@@ -144,31 +136,11 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
           log.warn("Error deleting replica: {}", r, e);
           throw e;
         }
-      }
-      log.debug("Waiting for delete shard action to complete");
-
-      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP,
-          collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
-      ZkStateReader zkStateReader = ocmh.zkStateReader;
-      ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
-
-      AddReplicaCmd.Response response = new AddReplicaCmd.Response();
-
-      if (results.get("failure") == null && results.get("exception") == null) {
-        response.asyncFinalRunner = new Runnable() {
-          @Override
-          public void run() {
-            try {
-              zkStateReader.waitForState(collectionName, 5, TimeUnit.SECONDS, (c) ->  c  != null && c.getSlice(sliceId) == null);
-            } catch (InterruptedException e) {
-              log.warn("",  e);
-            } catch (TimeoutException e) {
-              log.warn("",  e);
-            }
-          }
-        };
-      }
+      } log.debug("Waiting for delete shard action to complete");
+
+      ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETESHARD.toLower(), ZkStateReader.COLLECTION_PROP, collectionName, ZkStateReader.SHARD_ID_PROP, sliceId);
+
+      clusterState = new CollectionMutator(ocmh.cloudManager).deleteShard(clusterState, m);
 
       log.info("Successfully deleted collection: {} , shard: {}", collectionName, sliceId);
     } catch (SolrException e) {
@@ -178,7 +150,16 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
           "Error executing delete operation for collection: " + collectionName + " shard: " + sliceId, e);
     }
-    return null;
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.asyncFinalRunner = () -> {
+      for (OverseerCollectionMessageHandler.Finalize finalize : finalizers) {
+        finalize.call();
+      }
+      return new AddReplicaCmd.Response();
+    };
+    response.clusterState = clusterState;
+    return response;
   }
 
   private List<ZkNodeProps> getReplicasForSlice(String collectionName, Slice slice) {
@@ -187,7 +168,6 @@ public class DeleteShardCmd implements OverseerCollectionMessageHandler.Cmd {
       ZkNodeProps props = new ZkNodeProps(
           COLLECTION_PROP, collectionName,
           SHARD_ID_PROP, slice.getName(),
-          ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
           ZkStateReader.REPLICA_PROP, replica.getName(),
           CoreAdminParams.NODE, replica.getNodeName());
       sourceReplicas.add(props);
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 08dddfc..7455ad3 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
@@ -64,7 +64,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extCollectionName =  message.getStr(COLLECTION_PROP);
     boolean followAliases = message.getBool(FOLLOW_ALIASES, false);
     String collectionName;
@@ -90,7 +90,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
     Set<String> existingCores = new HashSet<>();
     for (Slice s : ocmh.zkStateReader.getClusterState().getCollection(collectionName).getSlices()) {
       for (Replica r : s.getReplicas()) {
-        existingCores.add(r.getCoreName());
+        existingCores.add(r.getName());
       }
     }
 
@@ -112,7 +112,7 @@ public class DeleteSnapshotCmd implements OverseerCollectionMessageHandler.Cmd {
         // Note - when a snapshot is found in_progress state - it is the result of overseer
         // failure while handling the snapshot creation. Since we don't know the exact set of
         // replicas to contact at this point, we try on all replicas.
-        if (meta.get().getStatus() == SnapshotStatus.InProgress || coresWithSnapshot.contains(replica.getCoreName())) {
+        if (meta.get().getStatus() == SnapshotStatus.InProgress || coresWithSnapshot.contains(replica.getName())) {
           String coreName = replica.getStr(CORE_NAME_PROP);
 
           ModifiableSolrParams params = new ModifiableSolrParams();
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
index 3f32074..a48f2d9 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
@@ -100,7 +100,7 @@ public class MaintainRoutedAliasCmd extends AliasCmd {
   }
 
   @Override
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     //---- PARSE PRIMARY MESSAGE PARAMS
     // important that we use NAME for the alias as that is what the Overseer will get a lock on before calling us
     final String aliasName = message.getStr(NAME);
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 e5f87d9..8502762 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
@@ -45,7 +45,6 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.handler.component.ShardHandler;
-import org.apache.solr.handler.component.ShardHandlerFactory;
 import org.apache.solr.update.SolrIndexSplitter;
 import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
@@ -74,7 +73,7 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
 
 
   @Override
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extSourceCollectionName = message.getStr("collection");
     String splitKey = message.getStr("split.key");
     String extTargetCollectionName = message.getStr("target.collection");
@@ -252,9 +251,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     Slice tempSourceSlice = clusterState.getCollection(tempSourceCollectionName).getSlices().iterator().next();
     Replica tempSourceLeader = zkStateReader.getLeaderRetry(tempSourceCollectionName, tempSourceSlice.getName(), 120000);
 
-    String tempCollectionReplica1 = tempSourceLeader.getCoreName();
-    String coreNodeName = ocmh.waitForCoreNodeName(zkStateReader, tempSourceCollectionName,
-        sourceLeader.getNodeName(), tempCollectionReplica1);
+    String tempCollectionReplica1 = tempSourceLeader.getName();
+
     // 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());
@@ -262,10 +260,11 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
     cmd.setCoreName(tempCollectionReplica1);
     cmd.setNodeName(sourceLeader.getNodeName());
-    cmd.setCoreNodeName(coreNodeName);
     cmd.setState(Replica.State.ACTIVE);
     cmd.setCheckLive(true);
     cmd.setOnlyIfLeader(true);
+    cmd.setCollection(tempSourceCollectionName);
+    cmd.setShardId(tempSourceSlice.getName());
     {
       final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
       // we don't want this to happen asynchronously
@@ -280,8 +279,8 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     log.info("Asking source leader to split index");
     params = new ModifiableSolrParams();
     params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
-    params.set(CoreAdminParams.CORE, sourceLeader.getStr("core"));
-    params.add(CoreAdminParams.TARGET_CORE, tempSourceLeader.getStr("core"));
+    params.set(CoreAdminParams.CORE, sourceLeader.getName());
+    params.add(CoreAdminParams.TARGET_CORE, tempSourceLeader.getName());
     params.set(CoreAdminParams.RANGES, splitRange.toString());
     params.set("split.key", splitKey);
 
@@ -296,8 +295,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
       log.info("Creating a replica of temporary collection: {} on the target leader node: {}",
           tempSourceCollectionName, targetLeader.getNodeName());
     }
-    String tempCollectionReplica2 = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(),
-        zkStateReader.getClusterState().getCollection(tempSourceCollectionName), tempSourceSlice.getName(), Replica.Type.NRT);
+    int replicas = 0;
+    DocCollection docCollection = clusterState.getCollectionOrNull(tempSourceCollectionName);
+    if (docCollection != null) {
+      replicas = docCollection.getReplicas().size();
+    }
+
+    String tempCollectionReplica2 = Assign.buildSolrCoreName(docCollection,
+       tempSourceCollectionName, tempSourceSlice.getName(), Replica.Type.NRT);
     props = new HashMap<>();
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
     props.put(COLLECTION_PROP, tempSourceCollectionName);
@@ -314,15 +319,14 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     if (asyncId != null) {
       props.put(ASYNC, asyncId);
     }
-    ((AddReplicaCmd)ocmh.commandMap.get(ADDREPLICA)).addReplica(clusterState, new ZkNodeProps(props), results, null);
+    ((AddReplicaCmd)ocmh.commandMap.get(ADDREPLICA)).call(clusterState, new ZkNodeProps(props), results);
 
     {
       final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
       syncRequestTracker.processResponses(results, shardHandler, true, "MIGRATE failed to create replica of " +
         "temporary collection in target leader node.");
     }
-    coreNodeName = ocmh.waitForCoreNodeName(zkStateReader, tempSourceCollectionName,
-        targetLeader.getNodeName(), 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());
@@ -330,7 +334,6 @@ public class MigrateCmd implements OverseerCollectionMessageHandler.Cmd {
     cmd = new CoreAdminRequest.WaitForState();
     cmd.setCoreName(tempSourceLeader.getStr("core"));
     cmd.setNodeName(targetLeader.getNodeName());
-    cmd.setCoreNodeName(coreNodeName);
     cmd.setState(Replica.State.ACTIVE);
     cmd.setCheckLive(true);
     cmd.setOnlyIfLeader(true);
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java
new file mode 100644
index 0000000..3527da7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyCollectionCmd.java
@@ -0,0 +1,28 @@
+package org.apache.solr.cloud.api.collections;
+
+import org.apache.solr.cloud.overseer.CollectionMutator;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+
+public class ModifyCollectionCmd implements OverseerCollectionMessageHandler.Cmd {
+
+  private final OverseerCollectionMessageHandler ocmh;
+  private final TimeSource timeSource;
+
+  public ModifyCollectionCmd(OverseerCollectionMessageHandler ocmh) {
+    this.ocmh = ocmh;
+    this.timeSource = ocmh.cloudManager.getTimeSource();
+  }
+
+  @Override
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, NamedList results) throws Exception {
+
+    clusterState = new CollectionMutator(ocmh.cloudManager).modifyCollection(clusterState, message);
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.clusterState = clusterState;
+    return response;
+  }
+}
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 8182c41..20bcfd9 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
@@ -23,8 +23,8 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Locale;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
-import org.apache.solr.cloud.ActiveReplicaWatcher;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrCloseableLatch;
 import org.apache.solr.common.SolrException;
@@ -67,12 +67,11 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
-    moveReplica(ocmh.zkStateReader.getClusterState(), message, results);
-    return null;
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
+    return moveReplica(ocmh.zkStateReader.getClusterState(), message, results);
   }
 
-  private void moveReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  private AddReplicaCmd.Response moveReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
     if (log.isDebugEnabled()) {
       log.debug("moveReplica() : {}", Utils.toJSONString(message));
     }
@@ -105,14 +104,12 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       String replicaName = message.getStr(REPLICA_PROP);
       replica = coll.getReplica(replicaName);
       if (replica == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Collection: " + collection + " replica: " + replicaName + " does not exist");
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " replica: " + replicaName + " does not exist");
       }
     } else {
       String sourceNode = message.getStr(CollectionParams.SOURCE_NODE, message.getStr(CollectionParams.FROM_NODE));
       if (sourceNode == null) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'" + CollectionParams.SOURCE_NODE +
-            " or '" + CollectionParams.FROM_NODE + "' is a required param");
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "'" + CollectionParams.SOURCE_NODE + " or '" + CollectionParams.FROM_NODE + "' is a required param");
       }
       String shardId = message.getStr(SHARD_ID_PROP);
       if (shardId == null) {
@@ -121,8 +118,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
       Slice slice = coll.getSlice(shardId);
       List<Replica> sliceReplicas = new ArrayList<>(slice.getReplicas(r -> sourceNode.equals(r.getNodeName())));
       if (sliceReplicas.isEmpty()) {
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-            "Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection: " + collection + " node: " + sourceNode + " does not have any replica belonging to shard: " + shardId);
       }
       Collections.shuffle(sliceReplicas, OverseerCollectionMessageHandler.RANDOM);
       replica = sliceReplicas.iterator().next();
@@ -140,8 +136,7 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
           List<Replica> replicasOnSourceNode = coll.getReplicas(replica.getNodeName());
           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()
+                "Collection: " + collection + " is co-located with collection: " + colocatedCollectionName + " and has a single replica: " + replica.getName() + " on node: " + replica.getNodeName()
                     + " so it is not possible to move it to another node");
           }
         }
@@ -158,41 +153,58 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     assert slice != null;
     Object dataDir = replica.get("dataDir");
     boolean isSharedFS = replica.getBool(ZkStateReader.SHARED_STORAGE_PROP, false) && dataDir != null;
-
+    OverseerCollectionMessageHandler.Finalize finalizer = null;
     if (isSharedFS && inPlaceMove) {
       log.debug("-- moveHdfsReplica");
       moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice, timeout, waitForFinalState);
     } else {
       log.debug("-- moveNormalReplica (inPlaceMove={}, isSharedFS={}", inPlaceMove, isSharedFS);
-      moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout, waitForFinalState);
+      finalizer = moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout, waitForFinalState);
     }
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    OverseerCollectionMessageHandler.Finalize finalIzer = finalizer;
+    response.asyncFinalRunner = new OverseerCollectionMessageHandler.Finalize() {
+      @Override
+      public AddReplicaCmd.Response call() {
+        if (finalIzer != null) {
+          try {
+            finalIzer.call();
+          } catch (Exception e) {
+            log.error("Exception during MoveReplica", e);
+          }
+        }
+        AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+        return response;
+      }
+    };
+
+    response.clusterState = clusterState;
+
+    return response;
   }
 
   @SuppressWarnings({"unchecked"})
-  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 {
+  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())) {
       skipCreateReplicaInClusterState = "false";
-      ZkNodeProps removeReplicasProps = new ZkNodeProps(
-          COLLECTION_PROP, coll.getName(),
-          SHARD_ID_PROP, slice.getName(),
-          REPLICA_PROP, replica.getName()
-      );
+      ZkNodeProps removeReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), REPLICA_PROP, replica.getName());
       removeReplicasProps.getProperties().put(CoreAdminParams.DELETE_DATA_DIR, false);
       removeReplicasProps.getProperties().put(CoreAdminParams.DELETE_INDEX, false);
       if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
-      @SuppressWarnings({"rawtypes"})
-      NamedList deleteResult = new NamedList();
+      @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
       try {
-        ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
+        ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult);
       } catch (SolrException e) {
         // assume this failed completely so there's nothing to roll back
         deleteResult.add("failure", e.toString());
       }
       if (deleteResult.get("failure") != null) {
-        String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s",
-            coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
+        String errorString = String
+            .format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s", coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
         log.warn(errorString);
         results.add("failure", errorString);
         return;
@@ -215,124 +227,118 @@ public class MoveReplicaCmd implements OverseerCollectionMessageHandler.Cmd {
     }
 
     String ulogDir = replica.getStr(CoreAdminParams.ULOG_DIR);
-    ZkNodeProps addReplicasProps = new ZkNodeProps(
-        COLLECTION_PROP, coll.getName(),
-        SHARD_ID_PROP, slice.getName(),
-        CoreAdminParams.NODE, targetNode,
-        CoreAdminParams.CORE_NODE_NAME, replica.getName(),
-        CoreAdminParams.NAME, replica.getCoreName(),
-        WAIT_FOR_FINAL_STATE, String.valueOf(waitForFinalState),
-        SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, skipCreateReplicaInClusterState,
-        CoreAdminParams.ULOG_DIR, ulogDir.substring(0, ulogDir.lastIndexOf(UpdateLog.TLOG_NAME)),
-        CoreAdminParams.DATA_DIR, dataDir,
-        ZkStateReader.REPLICA_TYPE, replica.getType().name());
+    ZkNodeProps addReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), CoreAdminParams.NODE, targetNode, CoreAdminParams.NAME, replica.getName(),
+        WAIT_FOR_FINAL_STATE, String.valueOf(waitForFinalState), SKIP_CREATE_REPLICA_IN_CLUSTER_STATE, skipCreateReplicaInClusterState, CoreAdminParams.ULOG_DIR,
+        ulogDir.substring(0, ulogDir.lastIndexOf(UpdateLog.TLOG_NAME)), CoreAdminParams.DATA_DIR, dataDir, ZkStateReader.REPLICA_TYPE, replica.getType().name());
 
-    if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
-    @SuppressWarnings({"rawtypes"})
-    NamedList addResult = new NamedList();
+    if (async != null) addReplicasProps.getProperties().put(ASYNC, async);
+    @SuppressWarnings({"rawtypes"}) NamedList addResult = new NamedList();
     try {
-      ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, addResult, null);
+      ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, addResult);
     } catch (Exception e) {
       ParWork.propagateInterrupt(e);
       // fatal error - try rolling back
-      String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
-          " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+      String errorString = String
+          .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " 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);
+      log.warn("Error adding replica {} - trying to roll back...", addReplicasProps, e);
       addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
-      @SuppressWarnings({"rawtypes"})
-      NamedList rollback = new NamedList();
-      ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
+      @SuppressWarnings({"rawtypes"}) NamedList rollback = new NamedList();
+      ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback);
       if (rollback.get("failure") != null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
-            + ", collection may be inconsistent: " + rollback.get("failure"));
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent: " + rollback.get("failure"));
       }
       return;
     }
     if (addResult.get("failure") != null) {
-      String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
-          " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+      String errorString = String
+          .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
       log.warn(errorString);
       results.add("failure", errorString);
       log.debug("--- trying to roll back...");
       // try to roll back
       addReplicasProps = addReplicasProps.plus(CoreAdminParams.NODE, replica.getNodeName());
-      @SuppressWarnings({"rawtypes"})
-      NamedList rollback = new NamedList();
+      @SuppressWarnings({"rawtypes"}) NamedList rollback = new NamedList();
       try {
-        ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback, null);
+        ocmh.addReplica(ocmh.zkStateReader.getClusterState(), addReplicasProps, rollback);
       } catch (Exception e) {
         ParWork.propagateInterrupt(e);
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
-            + ", collection may be inconsistent!", e);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent!", e);
       }
       if (rollback.get("failure") != null) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica
-            + ", collection may be inconsistent! Failure: " + rollback.get("failure"));
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Fatal error during MOVEREPLICA of " + replica + ", collection may be inconsistent! Failure: " + rollback.get("failure"));
       }
       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);
+      String successString = String
+          .format(Locale.ROOT, "MOVEREPLICA action completed successfully, moved replica=%s at node=%s " + "to replica=%s at node=%s", replica.getName(), replica.getNodeName(), replica.getName(),
+              targetNode);
       results.add("success", successString);
     }
   }
 
   @SuppressWarnings({"unchecked"})
-  private void moveNormalReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"})NamedList results, String targetNode, String async,
-                                 DocCollection coll, Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
-    String newCoreName = Assign.buildSolrCoreName(ocmh.overseer.getSolrCloudManager().getDistribStateManager(), coll, slice.getName(), replica.getType());
-    ZkNodeProps addReplicasProps = new ZkNodeProps(
-        COLLECTION_PROP, coll.getName(),
-        SHARD_ID_PROP, slice.getName(),
-        CoreAdminParams.NODE, targetNode,
-        CoreAdminParams.NAME, newCoreName,
+  private OverseerCollectionMessageHandler.Finalize moveNormalReplica(ClusterState clusterState, @SuppressWarnings({"rawtypes"}) NamedList results, String targetNode, String async, DocCollection coll,
+      Replica replica, Slice slice, int timeout, boolean waitForFinalState) throws Exception {
+    String newCoreName = Assign.buildSolrCoreName(coll, coll.getName(), slice.getName(), replica.getType());
+    ZkNodeProps addReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), CoreAdminParams.NODE, targetNode, CoreAdminParams.NAME, newCoreName,
         ZkStateReader.REPLICA_TYPE, replica.getType().name());
 
     if (async != null) addReplicasProps.getProperties().put(ASYNC, async + "-AddReplica-" + Math.abs(System.nanoTime()));
-    @SuppressWarnings({"rawtypes"})
-    NamedList addResult = new NamedList();
+    @SuppressWarnings({"rawtypes"}) NamedList addResult = new NamedList();
     SolrCloseableLatch countDownLatch = new SolrCloseableLatch(1, ocmh);
 
-    AddReplicaCmd.Response response = ocmh.addReplicaWithResp(clusterState, addReplicasProps, addResult, null);
+    AddReplicaCmd.Response response = ocmh.addReplicaWithResp(clusterState, addReplicasProps, addResult);
+
+    ocmh.overseer.getZkStateWriter().enqueueUpdate(response.clusterState, false);
+    ocmh.overseer.writePendingUpdates();
+
 
-    if (addResult.get("failure") != null) {
-      String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
-          " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
-      log.warn(errorString);
-      results.add("failure", errorString);
-      return;
-    }
     // wait for the other replica to be active if the source replica was a leader
 
-    log.debug("Waiting for leader's replica to recover.");
-    if (replica.equals(slice.getLeader()) || waitForFinalState) {
-      if (response != null && response.asyncFinalRunner != null) {
-        response.asyncFinalRunner.run();
-      }
-    }
 
-    ZkNodeProps removeReplicasProps = new ZkNodeProps(
-        COLLECTION_PROP, coll.getName(),
-        SHARD_ID_PROP, slice.getName(),
-        REPLICA_PROP, replica.getName());
-    if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
-    @SuppressWarnings({"rawtypes"})
-    NamedList deleteResult = new NamedList();
-    try {
-      ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
-    } catch (SolrException e) {
-      deleteResult.add("failure", e.toString());
-    }
-    if (deleteResult.get("failure") != null) {
-      String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s",
-          coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
-      log.warn(errorString);
-      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);
-      results.add("success", successString);
-    }
+
+    return new OverseerCollectionMessageHandler.Finalize() {
+      @Override
+      public AddReplicaCmd.Response call() throws Exception {
+        log.debug("Waiting for leader's replica to recover.");
+
+        response.asyncFinalRunner.call();
+
+        if (addResult.get("failure") != null) {
+          String errorString = String
+              .format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" + " on node=%s, failure=%s", coll.getName(), slice.getName(), targetNode, addResult.get("failure"));
+          log.warn(errorString);
+          results.add("failure", errorString);
+
+          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+          return response;
+        } else {
+
+          AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+          ZkNodeProps removeReplicasProps = new ZkNodeProps(COLLECTION_PROP, coll.getName(), SHARD_ID_PROP, slice.getName(), REPLICA_PROP, replica.getName());
+          if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
+          @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
+          try {
+            response.clusterState = ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult).clusterState;
+          } catch (SolrException e) {
+            deleteResult.add("failure", e.toString());
+          }
+          if (deleteResult.get("failure") != null) {
+            String errorString = String.format(Locale.ROOT, "Failed to cleanup replica collection=%s shard=%s name=%s, failure=%s", coll.getName(), slice.getName(), replica.getName(), deleteResult.get("failure"));
+            log.warn(errorString);
+            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.getName(), replica.getNodeName(), newCoreName,
+                    targetNode);
+            results.add("success", successString);
+          }
+
+          return response;
+        }
+      }
+    };
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index 8c10349..d7222ea 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -31,6 +31,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicReference;
@@ -48,21 +49,18 @@ import org.apache.solr.client.solrj.impl.Http2SolrClient;
 import org.apache.solr.client.solrj.impl.LBHttp2SolrClient;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
-import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.cloud.LockTree;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.OverseerMessageHandler;
-import org.apache.solr.cloud.OverseerNodePrioritizer;
 import org.apache.solr.cloud.OverseerSolrResponse;
-import org.apache.solr.cloud.OverseerSolrResponseSerializer;
 import org.apache.solr.cloud.OverseerTaskProcessor;
 import org.apache.solr.cloud.Stats;
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.overseer.CollectionMutator;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.AlreadyClosedException;
 import org.apache.solr.common.ParWork;
-import org.apache.solr.common.PerThreadExecService;
 import org.apache.solr.common.SolrCloseable;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -72,7 +70,6 @@ import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkConfigManager;
-import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
@@ -80,13 +77,15 @@ import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
-import org.apache.solr.handler.component.HttpShardHandler;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.handler.component.HttpShardHandlerFactory;
 import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.handler.component.ShardRequest;
@@ -95,10 +94,8 @@ import org.apache.solr.logging.MDCLoggingContext;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.data.Stat;
-import org.eclipse.jetty.util.BlockingArrayQueue;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -110,7 +107,6 @@ import static org.apache.solr.client.solrj.response.RequestStatusState.SUBMITTED
 import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
-import static org.apache.solr.common.cloud.ZkStateReader.CORE_NODE_NAME_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.ELECTION_NODE_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.PROPERTY_VALUE_PROP;
@@ -178,7 +174,10 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   // This is used for handling mutual exclusion of the tasks.
 
   final private LockTree lockTree = new LockTree();
-  ExecutorService tpe = ParWork.getParExecutorService("overseerTPE", 0, 16, 0, new BlockingArrayQueue());
+
+  ExecutorService tpe = new ExecutorUtil.MDCAwareThreadPoolExecutor(5, 10, 0L, TimeUnit.MILLISECONDS,
+      new SynchronousQueue<>(),
+      new SolrNamedThreadFactory("OverseerCollectionMessageHandlerThreadFactory"));
 
   public static final Random RANDOM;
   static {
@@ -196,17 +195,15 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
   private volatile boolean isClosed;
 
-  public OverseerCollectionMessageHandler(ZkStateReader zkStateReader, String myId,
+  public OverseerCollectionMessageHandler(CoreContainer cc, String myId,
                                           LBHttp2SolrClient overseerLbClient,
-                                          HttpShardHandlerFactory shardHandlerFactory,
                                           String adminPath,
                                           Stats stats,
-                                          Overseer overseer,
-                                          OverseerNodePrioritizer overseerPrioritizer) {
+                                          Overseer overseer) {
     // TODO: can leak single instance of this oddly in AddReplicaTest
     // assert ObjectReleaseTracker.track(this);
-    this.zkStateReader = zkStateReader;
-    this.shardHandlerFactory = shardHandlerFactory;
+    this.zkStateReader = cc.getZkController().getZkStateReader();
+    this.shardHandlerFactory = (HttpShardHandlerFactory) cc.getShardHandlerFactory();
     this.overseerLbClient = overseerLbClient;
     this.adminPath = adminPath;
     this.myId = myId;
@@ -223,8 +220,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
             .put(CREATESNAPSHOT, new CreateSnapshotCmd(this))
             .put(DELETESNAPSHOT, new DeleteSnapshotCmd(this))
             .put(SPLITSHARD, new SplitShardCmd(this))
-            .put(ADDROLE, new OverseerRoleCmd(this, ADDROLE, overseerPrioritizer))
-            .put(REMOVEROLE, new OverseerRoleCmd(this, REMOVEROLE, overseerPrioritizer))
             .put(MOCK_COLL_TASK, this::mockOperation)
             .put(MOCK_SHARD_TASK, this::mockOperation)
             .put(MOCK_REPLICA_TASK, this::mockOperation)
@@ -260,7 +255,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     MDCLoggingContext.setShard(message.getStr(SHARD_ID_PROP));
     MDCLoggingContext.setReplica(message.getStr(REPLICA_PROP));
     log.debug("OverseerCollectionMessageHandler.processMessage : {} , {}", operation, message);
-
+    ClusterState clusterState = overseer.getZkStateWriter().getClusterstate(false);
     @SuppressWarnings({"rawtypes"})
     NamedList results = new NamedList();
 
@@ -268,10 +263,41 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       CollectionAction action = getCollectionAction(operation);
       Cmd command = commandMap.get(action);
       if (command != null) {
-        Runnable runFinal = command.call(overseer.getZkStateReader().getClusterState(), message, results);
-        if (runFinal != null && results.get("failure") == null && results.get("exception") == null) {
-          runFinal.run();
+        AddReplicaCmd.Response responce = command.call(clusterState, message, results);
+        if (responce == null || responce.clusterState == null) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "CMD did not return a new clusterstate:" + operation);
+        }
+
+        log.info("Command returned clusterstate={} results={}", responce.clusterState, results);
+
+        overseer.getZkStateWriter().enqueueUpdate(responce.clusterState, false);
+
+        overseer.writePendingUpdates();
+
+
+        // nocommit consider
+        if (responce != null && responce.asyncFinalRunner != null && results.get("failure") == null && results.get("exception") == null) {
+          AddReplicaCmd.Response resp = responce.asyncFinalRunner.call();
+          log.info("Finalize after Command returned clusterstate={}", resp.clusterState);
+          if (resp.clusterState != null) {
+            overseer.getZkStateWriter().enqueueUpdate(responce.clusterState, false);
+            overseer.writePendingUpdates();
+          }
         }
+
+        String collection = message.getStr("collection");
+        if (collection == null) {
+          collection = message.getStr("name");
+        }
+        if (collection != null) {
+          Integer version = overseer.getZkStateWriter().lastWrittenVersion(collection);
+          if (version != null && !action.equals(DELETE)) {
+            results.add("csver", version); // nocommit - find out which version was written by overseer and return it in response for this
+          } else {
+             //deleted
+          }
+        }
+
       } else {
         throw new SolrException(ErrorCode.BAD_REQUEST, "Unknown operation:"
                 + operation);
@@ -305,13 +331,13 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
       nl.add("rspCode", e instanceof SolrException ? ((SolrException)e).code() : -1);
       results.add("exception", nl);
     }
-    //  results.addAll(threadSafeResults);
+
     return new OverseerSolrResponse(results);
   }
 
   @SuppressForbidden(reason = "Needs currentTimeMillis for mock requests")
   @SuppressWarnings({"unchecked"})
-  private Runnable mockOperation(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws InterruptedException {
+  private AddReplicaCmd.Response mockOperation(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws InterruptedException {
     //only for test purposes
     Thread.sleep(message.getInt("sleep", 1));
     if (log.isInfoEnabled()) {
@@ -330,20 +356,25 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   @SuppressWarnings({"unchecked"})
-  private Runnable reloadCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
+  private AddReplicaCmd.Response reloadCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws KeeperException, InterruptedException {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CoreAdminParams.ACTION, CoreAdminAction.RELOAD.toString());
 
     String asyncId = message.getStr(ASYNC);
     collectionCmd(message, params, results, Replica.State.ACTIVE, asyncId);
-    return null;
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.results = results;
+    // nocommit - we don't change this for this cmd, we should be able to indicate that to caller
+    response.clusterState = clusterState;
+    return response;
   }
 
   @SuppressWarnings("unchecked")
-  private Runnable processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+  private AddReplicaCmd.Response processRebalanceLeaders(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
     checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, CORE_NAME_PROP, ELECTION_NODE_PROP,
-            CORE_NODE_NAME_PROP, BASE_URL_PROP, REJOIN_AT_HEAD_PROP);
+            BASE_URL_PROP, REJOIN_AT_HEAD_PROP);
 
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(COLLECTION_PROP, message.getStr(COLLECTION_PROP));
@@ -351,7 +382,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     params.set(REJOIN_AT_HEAD_PROP, message.getStr(REJOIN_AT_HEAD_PROP));
     params.set(CoreAdminParams.ACTION, CoreAdminAction.REJOINLEADERELECTION.toString());
     params.set(CORE_NAME_PROP, message.getStr(CORE_NAME_PROP));
-    params.set(CORE_NODE_NAME_PROP, message.getStr(CORE_NODE_NAME_PROP));
     params.set(ELECTION_NODE_PROP, message.getStr(ELECTION_NODE_PROP));
     params.set(BASE_URL_PROP, message.getStr(BASE_URL_PROP));
 
@@ -370,7 +400,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   @SuppressWarnings("unchecked")
-  private Runnable processReplicaAddPropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+  private AddReplicaCmd.Response processReplicaAddPropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
     checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, ZkStateReader.NUM_SHARDS_PROP, "shards", REPLICA_PROP, PROPERTY_PROP, PROPERTY_VALUE_PROP);
     Map<String, Object> propMap = new HashMap<>(message.getProperties().size() + 1);
@@ -381,7 +411,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     return null;
   }
 
-  private Runnable processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+  private AddReplicaCmd.Response processReplicaDeletePropertyCommand(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
     checkRequired(message, COLLECTION_PROP, SHARD_ID_PROP, REPLICA_PROP, PROPERTY_PROP);
     Map<String, Object> propMap = new HashMap<>(message.getProperties().size() + 1);
@@ -392,7 +422,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     return null;
   }
 
-  private Runnable balanceProperty(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  private AddReplicaCmd.Response balanceProperty(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     if (StringUtils.isBlank(message.getStr(COLLECTION_PROP)) || StringUtils.isBlank(message.getStr(PROPERTY_PROP))) {
       throw new SolrException(ErrorCode.BAD_REQUEST,
               "The '" + COLLECTION_PROP + "' and '" + PROPERTY_PROP +
@@ -438,10 +468,9 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   @SuppressWarnings("unchecked")
-  void deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+  AddReplicaCmd.Response deleteReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
-    ((DeleteReplicaCmd) commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, message, results, onComplete);
-
+    return ((DeleteReplicaCmd) commandMap.get(DELETEREPLICA)).deleteReplica(clusterState, message, results);
   }
 
   void deleteCoreNode(String collectionName, String replicaName, Replica replica, String core) throws Exception {
@@ -450,7 +479,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
             ZkStateReader.CORE_NAME_PROP, core,
             ZkStateReader.NODE_NAME_PROP, replica.getStr(ZkStateReader.NODE_NAME_PROP),
             ZkStateReader.COLLECTION_PROP, collectionName,
-            ZkStateReader.CORE_NODE_NAME_PROP, replicaName,
             ZkStateReader.BASE_URL_PROP, replica.getStr(ZkStateReader.BASE_URL_PROP));
     overseer.offerStateUpdate(Utils.toJSON(m));
   }
@@ -482,7 +510,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   @SuppressWarnings({"unchecked"})
   void commit(@SuppressWarnings({"rawtypes"})NamedList results, String slice, Replica parentShardLeader) {
     log.debug("Calling soft commit to make sub shard updates visible");
-    String coreUrl = new ZkCoreNodeProps(parentShardLeader).getCoreUrl();
+    String coreUrl = parentShardLeader.getCoreUrl();
     // HttpShardHandler is hard coded to send a QueryRequest hence we go direct
     // and we force open a searcher so that we have documents to show upon switching states
     UpdateResponse updateResponse = null;
@@ -505,41 +533,6 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     return ureq.process(httpClient);
   }
 
-  static String waitForCoreNodeName(ZkStateReader zkStateReader, String collectionName, String msgNodeName, String msgCore) {
-    AtomicReference<String> errorMessage = new AtomicReference<>();
-    AtomicReference<String> coreNodeName = new AtomicReference<>();
-    try {
-      zkStateReader.waitForState(collectionName, 15, TimeUnit.SECONDS, (n, c) -> {
-        if (c == null)
-          return false;
-        final Map<String,Slice> slicesMap = c.getSlicesMap();
-        for (Slice slice : slicesMap.values()) {
-          for (Replica replica : slice.getReplicas()) {
-
-            String nodeName = replica.getStr(ZkStateReader.NODE_NAME_PROP);
-            String core = replica.getStr(ZkStateReader.CORE_NAME_PROP);
-
-            if (msgNodeName.equals(nodeName) && core.equals(msgCore)) {
-              coreNodeName.set(replica.getName());
-              return true;
-            }
-          }
-        }
-        return false;
-      });
-    } catch (TimeoutException e) {
-      String error = errorMessage.get();
-      if (error == null)
-        error = "Timeout waiting for collection state.";
-      throw new ZkController.NotInClusterStateException(ErrorCode.SERVER_ERROR, error);
-    } catch (InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Interrupted", e);
-    }
-
-    return coreNodeName.get();
-  }
-
   void waitForNewShard(String collectionName, String sliceName) {
     log.debug("Waiting for slice {} of collection {} to be available", sliceName, collectionName);
     try {
@@ -594,7 +587,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
 
-  private Runnable modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
+  private AddReplicaCmd.Response modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
 
     final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);
@@ -602,55 +595,31 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     //remove the property here to avoid any errors down the pipeline due to this property appearing
     String configName = (String) message.getProperties().remove(CollectionAdminParams.COLL_CONF);
 
-    if(configName != null) {
+    if (configName != null) {
       validateConfigOrThrowSolrException(configName);
 
       createConfNode(cloudManager.getDistribStateManager(), configName, collectionName);
       reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
     }
 
-    overseer.offerStateUpdate(Utils.toJSON(message));
-
-    try {
-      zkStateReader.waitForState(collectionName, 30, TimeUnit.SECONDS, (n, c) -> {
-        if (c == null) return false;
-
-        for (Map.Entry<String,Object> updateEntry : message.getProperties().entrySet()) {
-          String updateKey = updateEntry.getKey();
-
-          if (!updateKey.equals(ZkStateReader.COLLECTION_PROP)
-                  && !updateKey.equals(Overseer.QUEUE_OPERATION)
-                  && updateEntry.getValue() != null // handled below in a separate conditional
-                  && !updateEntry.getValue().equals(c.get(updateKey))) {
-            return false;
-          }
-
-          if (updateEntry.getValue() == null && c.containsKey(updateKey)) {
-            return false;
-          }
-        }
-        return true;
-      });
-    } catch (TimeoutException | InterruptedException e) {
-      ParWork.propagateInterrupt(e);
-      log.error("modifyCollection(ClusterState=" + clusterState + ", ZkNodeProps=" + message + ", NamedList=" + results + ")", e);
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Could not modify collection " + message, e);
-    }
+    clusterState = new CollectionMutator(cloudManager).modifyCollection(clusterState, message);
 
     // if switching to/from read-only mode reload the collection
     if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
       reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
     }
-    return null;
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.clusterState = clusterState;
+    return response;
   }
 
-  void cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  AddReplicaCmd.Response cleanupCollection(String collectionName, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     log.error("Cleaning up collection [{}].", collectionName);
     Map<String, Object> props = makeMap(
             Overseer.QUEUE_OPERATION, DELETE.toLower(),
             NAME, collectionName);
-    commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
-    zkStateReader.waitForState(collectionName, 10, TimeUnit.SECONDS, (liveNodes, collectionState) -> collectionState == null);
+    AddReplicaCmd.Response response = commandMap.get(DELETE).call(zkStateReader.getClusterState(), new ZkNodeProps(props), results);
+    return response;
   }
 
   Map<String, Replica> waitToSeeReplicasInState(String collectionName, Collection<String> coreUrls, boolean requireActive) {
@@ -701,19 +670,18 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
     return result.get();
   }
 
-  List<ZkNodeProps> addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+  AddReplicaCmd.Response addReplica(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
           throws Exception {
 
-    AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
-    if (response == null) return null;
-    return response.responseProps;
+    AddReplicaCmd.Response response = commandMap.get(ADDREPLICA).call(clusterState, message, results);
+
+    return response;
   }
 
-   AddReplicaCmd.Response addReplicaWithResp(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results, Runnable onComplete)
+  AddReplicaCmd.Response  addReplicaWithResp(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
       throws Exception {
 
-    AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).addReplica(clusterState, message, results, onComplete);
-    if (response == null) return null;
+    AddReplicaCmd.Response response = ((AddReplicaCmd) commandMap.get(ADDREPLICA)).call(clusterState, message, results);
     return response;
   }
 
@@ -1004,25 +972,14 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   @Override
   public void close() throws IOException {
     this.isClosed = true;
-    try {
-      cloudManager.close();
-    } catch (NullPointerException e) {
-      // okay
-    } finally {
-      latches.forEach(countDownLatch -> countDownLatch.countDown());
-      latches.clear();
-
-      if (tpe != null) {
-        if (!tpe.isShutdown()) {
-          tpe.shutdownNow();
-          try {
-            tpe.awaitTermination(10, TimeUnit.SECONDS);
-          } catch (InterruptedException e) {
-            ParWork.propagateInterrupt(e);
-          }
-        }
+
+    if (tpe != null) {
+      if (!tpe.isShutdown()) {
+        ExecutorUtil.shutdownAndAwaitTermination(tpe);
       }
     }
+    latches.forEach(countDownLatch -> countDownLatch.countDown());
+    latches.clear();
 
     // assert ObjectReleaseTracker.release(this);
   }
@@ -1033,7 +990,11 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
   }
 
   protected interface Cmd {
-    Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception;
+    AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception;
+  }
+
+  protected interface Finalize {
+    AddReplicaCmd.Response call() throws Exception;
   }
 
   /*
@@ -1082,7 +1043,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
             // For thread safety, only simple clone the ModifiableSolrParams
             ModifiableSolrParams cloneParams = new ModifiableSolrParams();
             cloneParams.add(params);
-            cloneParams.set(CoreAdminParams.CORE, replica.getStr(ZkStateReader.CORE_NAME_PROP));
+            cloneParams.set(CoreAdminParams.CORE, replica.getName());
 
             sendShardRequest(replica.getStr(ZkStateReader.NODE_NAME_PROP), cloneParams, shardHandler);
           } else {
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
index 22792ed..d97490b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerRoleCmd.java
@@ -25,7 +25,6 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.cloud.OverseerNodePrioritizer;
-import org.apache.solr.common.ParWork;
 import org.apache.solr.common.cloud.ClusterState;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
@@ -58,7 +57,7 @@ public class OverseerRoleCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked", "rawtypes"})
-  public Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     SolrZkClient zkClient = zkStateReader.getZkClient();
     Map roles = null;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
index f0bd7e7..a240809 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerStatusCmd.java
@@ -22,6 +22,7 @@ import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
 
 import com.codahale.metrics.Timer;
 import org.apache.solr.cloud.OverseerTaskProcessor;
@@ -46,7 +47,7 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings("unchecked")
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     String leaderNode = OverseerTaskProcessor.getLeaderNode(zkStateReader.getZkClient());
     results.add("leader", leaderNode);
@@ -54,9 +55,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
     zkStateReader.getZkClient().getData("/overseer/queue",null, stat);
     results.add("overseer_queue_size", stat.getNumChildren());
     stat = new Stat();
-    zkStateReader.getZkClient().getData("/overseer/queue-work",null, stat);
-    results.add("overseer_work_queue_size", stat.getNumChildren());
-    stat = new Stat();
     zkStateReader.getZkClient().getData("/overseer/collection-queue-work",null, stat);
     results.add("overseer_collection_queue_size", stat.getNumChildren());
 
@@ -67,8 +65,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
     @SuppressWarnings({"rawtypes"})
     NamedList stateUpdateQueueStats = new NamedList();
     @SuppressWarnings({"rawtypes"})
-    NamedList workQueueStats = new NamedList();
-    @SuppressWarnings({"rawtypes"})
     NamedList collectionQueueStats = new NamedList();
     Stats stats = ocmh.stats;
     for (Map.Entry<String, Stats.Stat> entry : stats.getStats().entrySet()) {
@@ -93,8 +89,6 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
         }
       } else if (key.startsWith("/overseer/queue_"))  {
         stateUpdateQueueStats.add(key.substring(16), lst);
-      } else if (key.startsWith("/overseer/queue-work_"))  {
-        workQueueStats.add(key.substring(21), lst);
       } else if (key.startsWith("/overseer/collection-queue-work_"))  {
         collectionQueueStats.add(key.substring(32), lst);
       } else  {
@@ -111,9 +105,13 @@ public class OverseerStatusCmd implements OverseerCollectionMessageHandler.Cmd {
     results.add("overseer_operations", overseerStats);
     results.add("collection_operations", collectionStats);
     results.add("overseer_queue", stateUpdateQueueStats);
-    results.add("overseer_internal_queue", workQueueStats);
     results.add("collection_queue", collectionQueueStats);
 
-    return null;
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+
+    response.clusterState = clusterState;
+
+    return response;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index fd3516a..8d61ef2 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -167,7 +167,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
 
     log.debug("*** called: {}", message);
 
@@ -618,8 +618,8 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
     }
     // build a baseUrl of the replica
     for (Replica r : coll.getReplicas()) {
-      if (replicaName.equals(r.getCoreName())) {
-        return r.getBaseUrl() + "/" + r.getCoreName();
+      if (replicaName.equals(r.getName())) {
+        return r.getBaseUrl() + "/" + r.getName();
       }
     }
     return null;
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
index b5d61f5..bc7d886 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RenameCmd.java
@@ -44,7 +44,7 @@ public class RenameCmd implements OverseerCollectionMessageHandler.Cmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String extCollectionName = message.getStr(CoreAdminParams.NAME);
     String target = message.getStr(CollectionAdminParams.TARGET);
 
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 da99e84..20cce88 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
@@ -27,6 +27,8 @@ import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+
 import org.apache.solr.cloud.ActiveReplicaWatcher;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrCloseableLatch;
@@ -61,7 +63,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked"})
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     String source = message.getStr(CollectionParams.SOURCE_NODE, message.getStr("source"));
     String target = message.getStr(CollectionParams.TARGET_NODE, message.getStr("target"));
@@ -92,63 +94,39 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
       }
     }
     // map of collectionName_coreNodeName to watchers
-    Map<String, CollectionStateWatcher> watchers = new HashMap<>();
+    Map<String,CollectionStateWatcher> watchers = new HashMap<>();
     List<ZkNodeProps> createdReplicas = new ArrayList<>();
 
     AtomicBoolean anyOneFailed = new AtomicBoolean(false);
     SolrCloseableLatch countDownLatch = new SolrCloseableLatch(sourceReplicas.size(), ocmh);
 
     SolrCloseableLatch replicasToRecover = new SolrCloseableLatch(numLeaders, ocmh);
-    try {
-      for (ZkNodeProps sourceReplica : sourceReplicas) {
-        @SuppressWarnings({"rawtypes"})
-        NamedList nl = new NamedList();
-        String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
-        if (log.isInfoEnabled()) {
-          log.info("Going to create replica for collection={} shard={} on node={}", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
-        }
-        String targetNode = target;
-        if (targetNode == null) {
-          Replica.Type replicaType = Replica.Type.get(sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
-          int numNrtReplicas = replicaType == Replica.Type.NRT ? 1 : 0;
-          int numTlogReplicas = replicaType == Replica.Type.TLOG ? 1 : 0;
-          int numPullReplicas = replicaType == Replica.Type.PULL ? 1 : 0;
-          Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder()
-              .forCollection(sourceCollection)
-              .forShard(Collections.singletonList(sourceReplica.getStr(SHARD_ID_PROP)))
-              .assignNrtReplicas(numNrtReplicas)
-              .assignTlogReplicas(numTlogReplicas)
-              .assignPullReplicas(numPullReplicas)
-              .onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes()))
-              .build();
-          Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
-          Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, clusterState.getCollection(sourceCollection));
-          targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
-        }
-        ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
-        if (async != null) msg.getProperties().put(ASYNC, async);
-        final ZkNodeProps addedReplica = ocmh.addReplica(clusterState,
-            msg, nl, () -> {
-              countDownLatch.countDown();
-              if (nl.get("failure") != null) {
-                String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
-                    " on node=%s", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
-                log.warn(errorString);
-                // one replica creation failed. Make the best attempt to
-                // delete all the replicas created so far in the target
-                // and exit
-                synchronized (results) {
-                  results.add("failure", errorString);
-                  anyOneFailed.set(true);
-                }
-              } else {
-                if (log.isDebugEnabled()) {
-                  log.debug("Successfully created replica for collection={} shard={} on node={}",
-                      sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
-                }
-              }
-            }).get(0);
-
+    List<Runnable> runners = new ArrayList<>();
+    for (ZkNodeProps sourceReplica : sourceReplicas) {
+      @SuppressWarnings({"rawtypes"}) NamedList nl = new NamedList();
+      String sourceCollection = sourceReplica.getStr(COLLECTION_PROP);
+      if (log.isInfoEnabled()) {
+        log.info("Going to create replica for collection={} shard={} on node={}", sourceCollection, sourceReplica.getStr(SHARD_ID_PROP), target);
+      }
+      String targetNode = target;
+      if (targetNode == null) {
+        Replica.Type replicaType = Replica.Type.get(sourceReplica.getStr(ZkStateReader.REPLICA_TYPE));
+        int numNrtReplicas = replicaType == Replica.Type.NRT ? 1 : 0;
+        int numTlogReplicas = replicaType == Replica.Type.TLOG ? 1 : 0;
+        int numPullReplicas = replicaType == Replica.Type.PULL ? 1 : 0;
+        Assign.AssignRequest assignRequest = new Assign.AssignRequestBuilder().forCollection(sourceCollection).forShard(Collections.singletonList(sourceReplica.getStr(SHARD_ID_PROP)))
+            .assignNrtReplicas(numNrtReplicas).assignTlogReplicas(numTlogReplicas).assignPullReplicas(numPullReplicas)
+            .onNodes(new ArrayList<>(ocmh.cloudManager.getClusterStateProvider().getLiveNodes())).build();
+        Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
+        Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
+        targetNode = assignStrategy.assign(ocmh.cloudManager, assignRequest).get(0).node;
+      }
+      ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, targetNode);
+      if (async != null) msg.getProperties().put(ASYNC, async);
+      AddReplicaCmd.Response response = ocmh.addReplica(clusterState, msg, nl);
+      clusterState = response.clusterState;
+      Runnable runner = () -> {
+        final ZkNodeProps addedReplica = response.responseProps.get(0);
         if (addedReplica != null) {
           createdReplicas.add(addedReplica);
           if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false) || waitForFinalState) {
@@ -158,11 +136,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
             String key = collectionName + "_" + replicaName;
             CollectionStateWatcher watcher;
             if (waitForFinalState) {
-              watcher = new ActiveReplicaWatcher(collectionName, Collections.singletonList(replicaName),
-                  null, replicasToRecover);
+              watcher = new ActiveReplicaWatcher(collectionName, Collections.singletonList(replicaName), null, replicasToRecover);
             } else {
-              watcher = new LeaderRecoveryWatcher(collectionName, shardName, replicaName,
-                  addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
+              watcher = new LeaderRecoveryWatcher(collectionName, shardName, replicaName, addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
             }
             watchers.put(key, watcher);
             log.debug("--- adding {}, {}", key, watcher);
@@ -171,66 +147,90 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
             log.debug("--- not waiting for {}", addedReplica);
           }
         }
+      };
+      runners.add(runner);
+    }
+
+
+
+    AddReplicaCmd.Response response = new AddReplicaCmd.Response();
+    response.results = results;
+    response.clusterState = clusterState;
+
+    int finalNumLeaders = numLeaders;
+    response.asyncFinalRunner = () -> {
+      for (Runnable runner : runners) {
+        runner.run();
       }
 
       log.debug("Waiting for replicas to be added");
-      if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
-        log.info("Timed out waiting for replicas to be added");
-        anyOneFailed.set(true);
-      } else {
-        log.debug("Finished waiting for replicas to be added");
+      try {
+        if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
+          log.info("Timed out waiting for replicas to be added");
+          anyOneFailed.set(true);
+        } else {
+          log.debug("Finished waiting for replicas to be added");
+        }
+
+        // now wait for leader replicas to recover
+        log.debug("Waiting for {} leader replicas to recover", finalNumLeaders);
+        if (!replicasToRecover.await(timeout, TimeUnit.SECONDS)) {
+          if (log.isInfoEnabled()) {
+            log.info("Timed out waiting for {} leader replicas to recover", replicasToRecover.getCount());
+          }
+          anyOneFailed.set(true);
+        } else {
+          log.debug("Finished waiting for leader replicas to recover");
+        }
+      } catch (InterruptedException e) {
+        ParWork.propagateInterrupt(e);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
       }
-    } finally {
-    }
-    // now wait for leader replicas to recover
-    log.debug("Waiting for {} leader replicas to recover", numLeaders);
-    if (!replicasToRecover.await(timeout, TimeUnit.SECONDS)) {
-      if (log.isInfoEnabled()) {
-        log.info("Timed out waiting for {} leader replicas to recover", replicasToRecover.getCount());
+      // remove the watchers, we're done either way
+      for (Map.Entry<String,CollectionStateWatcher> e : watchers.entrySet()) {
+        zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
       }
-      anyOneFailed.set(true);
-    } else {
-      log.debug("Finished waiting for leader replicas to recover");
-    }
-    // remove the watchers, we're done either way
-    for (Map.Entry<String, CollectionStateWatcher> e : watchers.entrySet()) {
-      zkStateReader.removeCollectionStateWatcher(e.getKey(), e.getValue());
-    }
-    if (anyOneFailed.get()) {
-      log.info("Failed to create some replicas. Cleaning up all replicas on target node");
-      SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ocmh);
-      for (ZkNodeProps createdReplica : createdReplicas) {
-        @SuppressWarnings({"rawtypes"})
-        NamedList deleteResult = new NamedList();
-        try {
-          ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult, () -> {
+      if (anyOneFailed.get()) {
+        log.info("Failed to create some replicas. Cleaning up all replicas on target node");
+        SolrCloseableLatch cleanupLatch = new SolrCloseableLatch(createdReplicas.size(), ocmh);
+        for (ZkNodeProps createdReplica : createdReplicas) {
+          @SuppressWarnings({"rawtypes"}) NamedList deleteResult = new NamedList();
+          try {
+            // nocommit - return results from deleteReplica cmd, update clusterstate
+            ocmh.deleteReplica(zkStateReader.getClusterState(), createdReplica.plus("parallel", "true"), deleteResult);
+          } catch (KeeperException e) {
             cleanupLatch.countDown();
-            if (deleteResult.get("failure") != null) {
-              synchronized (results) {
-                results.add("failure", "Could not cleanup, because of : " + deleteResult.get("failure"));
-              }
-            }
-          });
-        } catch (KeeperException e) {
-          cleanupLatch.countDown();
-          log.warn("Error deleting replica ", e);
-        } catch (Exception e) {
+            log.warn("Error deleting replica ", e);
+          } catch (Exception e) {
+            ParWork.propagateInterrupt(e);
+            log.warn("Error deleting replica ", e);
+            cleanupLatch.countDown();
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+          }
+        }
+        try {
+          cleanupLatch.await(5, TimeUnit.MINUTES);
+        } catch (InterruptedException e) {
           ParWork.propagateInterrupt(e);
-          log.warn("Error deleting replica ", e);
-          cleanupLatch.countDown();
-          throw e;
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
         }
       }
-      cleanupLatch.await(5, TimeUnit.MINUTES);
-      return null;
-    }
 
+      // we have reached this far means all replicas could be recreated
+      //now cleanup the replicas in the source node
+      try {
+        DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source, null);
+      } catch (InterruptedException e) {
+        ParWork.propagateInterrupt(e);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+      results.add("success", "REPLACENODE action completed successfully from  : " + source + " to : " + target);
+      AddReplicaCmd.Response resp = new AddReplicaCmd.Response();
+      return resp;
+
+    };
 
-    // we have reached this far means all replicas could be recreated
-    //now cleanup the replicas in the source node
-    DeleteNodeCmd.cleanupReplicas(results, state, sourceReplicas, ocmh, source, null);
-    results.add("success", "REPLACENODE action completed successfully from  : " + source + " to : " + target);
-    return null;
+    return response;
   }
 
   static List<ZkNodeProps> getReplicasOfNode(String source, ClusterState state) {
@@ -242,8 +242,7 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
             ZkNodeProps props = new ZkNodeProps(
                 COLLECTION_PROP, e.getKey(),
                 SHARD_ID_PROP, slice.getName(),
-                ZkStateReader.CORE_NAME_PROP, replica.getCoreName(),
-                ZkStateReader.REPLICA_PROP, replica.getName(),
+                ZkStateReader.CORE_NAME_PROP, replica.getName(),
                 ZkStateReader.REPLICA_TYPE, replica.getType().name(),
                 ZkStateReader.LEADER_PROP, String.valueOf(replica.equals(slice.getLeader())),
                 CoreAdminParams.NODE, source);
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 e2b01dc..219f304 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
@@ -87,7 +87,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @Override
   @SuppressWarnings({"unchecked", "rawtypes"})
-  public Runnable call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
     // TODO maybe we can inherit createCollection's options/code
 
     String restoreCollectionName = message.getStr(COLLECTION_PROP);
@@ -242,8 +242,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
         .assignPullReplicas(numPullReplicas)
         .onNodes(nodeList)
         .build();
+
     Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
-    Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, restoreCollection);
+    Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
     List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
 
     CountDownLatch countDownLatch = new CountDownLatch(restoreCollection.getSlices().size());
@@ -284,25 +285,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
       }
       ocmh.addPropertyParams(message, propMap);
       final NamedList addReplicaResult = new NamedList();
-      ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult, () -> {
-        Object addResultFailure = addReplicaResult.get("failure");
-        if (addResultFailure != null) {
-          SimpleOrderedMap failure = (SimpleOrderedMap) results.get("failure");
-          if (failure == null) {
-            failure = new SimpleOrderedMap();
-            results.add("failure", failure);
-          }
-          failure.addAll((NamedList) addResultFailure);
-        } else {
-          SimpleOrderedMap success = (SimpleOrderedMap) results.get("success");
-          if (success == null) {
-            success = new SimpleOrderedMap();
-            results.add("success", success);
-          }
-          success.addAll((NamedList) addReplicaResult.get("success"));
-        }
-        countDownLatch.countDown();
-      });
+      ocmh.addReplica(clusterState, new ZkNodeProps(propMap), addReplicaResult);
     }
 
     boolean allIsDone = countDownLatch.await(1, TimeUnit.HOURS);
@@ -339,17 +322,17 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
       for (Slice s : restoreCollection.getSlices()) {
         for (Replica r : s.getReplicas()) {
           String nodeName = r.getNodeName();
-          String coreNodeName = r.getCoreName();
+          String coreName = r.getName();
           Replica.State stateRep = r.getState();
 
           if (log.isDebugEnabled()) {
-            log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreNodeName={}, state={}", nodeName, coreNodeName,
+            log.debug("Calling REQUESTAPPLYUPDATES on: nodeName={}, coreName={}, state={}", nodeName, coreName,
                 stateRep.name());
           }
 
           ModifiableSolrParams params = new ModifiableSolrParams();
           params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
-          params.set(CoreAdminParams.NAME, coreNodeName);
+          params.set(CoreAdminParams.NAME, coreName);
 
           shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
         }
@@ -425,7 +408,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
           }
           ocmh.addPropertyParams(message, propMap);
 
-          ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results, null);
+          ocmh.addReplica(zkStateReader.getClusterState(), new ZkNodeProps(propMap), results);
         }
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
index 1e82000..c609086 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
@@ -47,7 +47,7 @@ public class SetAliasPropCmd implements Cmd {
   }
 
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
     String aliasName = message.getStr(NAME);
 
     final ZkStateReader.AliasesManager aliasesManager = messageHandler.zkStateReader.aliasesManager;
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 cdc2555..24dac32 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
@@ -27,9 +27,9 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.NoSuchElementException;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -38,10 +38,13 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.solr.client.solrj.cloud.NodeStateProvider;
 import org.apache.solr.client.solrj.cloud.ReplicaInfo;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.VersionedData;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
+import org.apache.solr.cloud.overseer.CollectionMutator;
 import org.apache.solr.cloud.overseer.OverseerAction;
+import org.apache.solr.cloud.overseer.SliceMutator;
 import org.apache.solr.common.ParWork;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
@@ -67,6 +70,7 @@ import org.apache.solr.handler.component.ShardHandler;
 import org.apache.solr.update.SolrIndexSplitter;
 import org.apache.solr.util.RTimerTree;
 import org.apache.solr.util.TestInjection;
+import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -98,13 +102,12 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
   @SuppressWarnings("unchecked")
   @Override
-  public Runnable call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
-    split(state, message,(NamedList<Object>) results);
-    return null;
+  public AddReplicaCmd.Response call(ClusterState state, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results) throws Exception {
+    return split(state, message,(NamedList<Object>) results);
   }
 
   @SuppressWarnings({"rawtypes"})
-  public boolean split(ClusterState clusterState, ZkNodeProps message, NamedList<Object> results) throws Exception {
+  public AddReplicaCmd.Response split(ClusterState clusterState, ZkNodeProps message, NamedList<Object> results) throws Exception {
     final String asyncId = message.getStr(ASYNC);
 
     boolean waitForFinalState = message.getBool(CommonAdminParams.WAIT_FOR_FINAL_STATE, false);
@@ -126,7 +129,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       collectionName = extCollectionName;
     }
 
-    log.debug("Split shard invoked: {}", message);
+    log.info("Split shard invoked: {} cs={}", message, clusterState);
     ZkStateReader zkStateReader = ocmh.zkStateReader;
     AtomicReference<String> slice = new AtomicReference<>();
     slice.set(message.getStr(ZkStateReader.SHARD_ID_PROP));
@@ -134,13 +137,12 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
     RTimerTree timings = new RTimerTree();
 
     String splitKey = message.getStr("split.key");
-    DocCollection collection = clusterState.getCollection(collectionName);
 
-    Slice parentSlice = getParentSlice(zkStateReader, collectionName, slice, splitKey);
-    if (parentSlice.getState() != Slice.State.ACTIVE) {
-      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " +
-          collectionName + "/ " + parentSlice.getName() + ", state=" + parentSlice.getState());
-    }
+    Slice parentSlice = getParentSlice(clusterState, collectionName, slice, splitKey);
+//    if (parentSlice.getState() != Slice.State.ACTIVE) {
+//      throw new SolrException(SolrException.ErrorCode.INVALID_STATE, "Parent slice is not active: " +
+//          collectionName + "/ " + parentSlice.getName() + ", state=" + parentSlice.getState());
+//    }
 
     // find the leader for the shard
     Replica parentShardLeader;
@@ -197,6 +199,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
       List<Map<String, Object>> replicas = new ArrayList<>((repFactor - 1) * 2);
 
+      Map<String, Object> replicaToPosition = new HashMap<>(replicas.size());
+
       @SuppressWarnings("deprecation")
       ShardHandler shardHandler = ocmh.shardHandlerFactory.getShardHandler(ocmh.overseerLbClient);
 
@@ -208,7 +212,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
         params.set(CoreAdminParams.GET_RANGES, "true");
         params.set(CommonAdminParams.SPLIT_METHOD, splitMethod.toLower());
-        params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+        params.set(CoreAdminParams.CORE, parentShardLeader.getName());
         // Only 2 is currently supported
         // int numSubShards = message.getInt(NUM_SUB_SHARDS, DEFAULT_NUM_SUB_SHARDS);
         // params.set(NUM_SUB_SHARDS, Integer.toString(numSubShards));
@@ -239,8 +243,8 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
 
       t = timings.sub("fillRanges");
-
-      String rangesStr = fillRanges(ocmh.cloudManager, message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
+      DocCollection collection = clusterState.getCollection(collectionName);
+      String rangesStr = fillRanges(message, collection, parentSlice, subRanges, subSlices, subShardNames, firstNrtReplica);
       t.stop();
 
       boolean oldShardsDeleted = false;
@@ -250,7 +254,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
           final Slice.State state = oSlice.getState();
           if (state == Slice.State.ACTIVE) {
             throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
-                "Sub-shard: " + subSlice + " exists in active state. Aborting split shard.");
+                "Sub-shard: " + subSlice + " exists in active state. Aborting split shard. Parent=" + parentSlice.getName());
           } else {
             // delete the shards
             log.info("Sub-shard: {} already exists therefore requesting its deletion", subSlice);
@@ -260,7 +264,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
             propMap.put(SHARD_ID_PROP, subSlice);
             ZkNodeProps m = new ZkNodeProps(propMap);
             try {
-              ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList());
+              clusterState = ocmh.commandMap.get(DELETESHARD).call(clusterState, m, new NamedList()).clusterState;
             } catch (Exception e) {
               ParWork.propagateInterrupt(e);
               throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unable to delete already existing sub shard: " + subSlice,
@@ -275,14 +279,13 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       if (oldShardsDeleted) {
         // refresh the locally cached cluster state
         // we know we have the latest because otherwise deleteshard would have failed
-        clusterState = zkStateReader.getClusterState();
         collection = clusterState.getCollection(collectionName);
       }
 
       String nodeName = parentShardLeader.getNodeName();
 
       t = timings.sub("createSubSlicesAndLeadersInState");
-      List<Future> firstReplicaFutures = new ArrayList<>();
+      List<OverseerCollectionMessageHandler.Finalize> firstReplicaFutures = new ArrayList<>();
       Set<Runnable> firstReplicaRunAfters = ConcurrentHashMap.newKeySet();
       for (int i = 0; i < subRanges.size(); i++) {
         String subSlice = subSlices.get(i);
@@ -301,13 +304,10 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         propMap.put("shard_parent_node", nodeName);
         propMap.put("shard_parent_zk_session", leaderZnodeStat.getEphemeralOwner());
 
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
-
+        //ocmh.overseer.offerStateUpdate(Utils.toJSON(new ZkNodeProps(propMap)));
+        clusterState = new CollectionMutator(ocmh.cloudManager).createShard(clusterState, new ZkNodeProps(propMap));
         // wait until we are able to see the new shard in cluster state and refresh the local view of the cluster state
-        ocmh.waitForNewShard(collectionName, subSlice);
-
-        // refresh cluster state
-        clusterState = zkStateReader.getClusterState();
+        //ocmh.waitForNewShard(collectionName, subSlice);
 
         log.debug("Adding first replica {} as part of slice {} of collection {} on {}"
             , subShardName, subSlice, collectionName, nodeName);
@@ -329,34 +329,36 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         if (asyncId != null) {
           propMap.put(ASYNC, asyncId);
         }
-
-        Map<String,Object> finalPropMap = propMap;
-        ClusterState finalClusterState1 = clusterState;
-        Future<?> future = ocmh.tpe.submit(() -> {
-          AddReplicaCmd.Response response = null;
-          try {
-            response = ocmh.addReplicaWithResp(finalClusterState1, new ZkNodeProps(finalPropMap), results, null);
-          } catch (Exception e) {
-            log.error("", e);
-          }
-          if (response != null && response.asyncFinalRunner != null) {
-            firstReplicaRunAfters.add(response.asyncFinalRunner);
-          }
-        });
-        firstReplicaFutures.add(future);
+        AddReplicaCmd.Response resp = ocmh.addReplicaWithResp(clusterState, new ZkNodeProps(propMap), results);
+        clusterState = resp.clusterState;
+        firstReplicaFutures.add(resp.asyncFinalRunner);
+//        Map<String,Object> finalPropMap = propMap;
+//        ClusterState finalClusterState1 = clusterState;
+//        Future<?> future = ocmh.tpe.submit(() -> {
+//          AddReplicaCmd.Response response = null;
+//          try {
+//            response = ocmh.addReplicaWithResp(finalClusterState1, new ZkNodeProps(finalPropMap), results, null);
+//          } catch (Exception e) {
+//            log.error("", e);
+//          }
+////          if (response != null && response.asyncFinalRunner != null) {
+////            firstReplicaRunAfters.add(response.asyncFinalRunner);
+////          }
+//        });
+//        firstReplicaFutures.add(future);
       }
 
+      ocmh.overseer.getZkStateWriter().enqueueUpdate(clusterState, false);
+      ocmh.overseer.writePendingUpdates();
       firstReplicaFutures.forEach(future -> {
         try {
-          future.get();
-        } catch (InterruptedException e) {
-          log.error("", e);
-        } catch (ExecutionException e) {
-          log.error("", e);
+          future.call();
+        } catch (Exception e) {
+          log.error("Exception waiting for created replica", e);
         }
       });
 
-      firstReplicaRunAfters.forEach(runnable -> runnable.run());
+     // firstReplicaRunAfters.forEach(runnable -> runnable.run());
 
       {
         final ShardRequestTracker syncRequestTracker = ocmh.syncRequestTracker();
@@ -370,15 +372,16 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
         for (String subShardName : subShardNames) {
           // wait for parent leader to acknowledge the sub-shard core
-          log.debug("Asking parent leader to wait for: {} to be alive on: {}", subShardName, nodeName);
-          String coreNodeName = OverseerCollectionMessageHandler.waitForCoreNodeName(zkStateReader, collectionName, nodeName, subShardName);
+          log.info("Asking parent leader to wait for: {} to be alive on: {}", subShardName, nodeName);
+
           CoreAdminRequest.WaitForState cmd = new CoreAdminRequest.WaitForState();
           cmd.setCoreName(subShardName);
           cmd.setNodeName(nodeName);
-          cmd.setCoreNodeName(coreNodeName);
           cmd.setState(Replica.State.ACTIVE);
           cmd.setCheckLive(true);
           cmd.setOnlyIfLeader(true);
+          cmd.setCollection(collectionName);
+
 
           ModifiableSolrParams p = new ModifiableSolrParams(cmd.getParams());
           shardRequestTracker.sendShardRequest(nodeName, p, shardHandler);
@@ -401,7 +404,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       ModifiableSolrParams params = new ModifiableSolrParams();
       params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.SPLIT.toString());
       params.set(CommonAdminParams.SPLIT_METHOD, splitMethod.toLower());
-      params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+      params.set(CoreAdminParams.CORE, parentShardLeader.getName());
       for (int i = 0; i < subShardNames.size(); i++) {
         String subShardName = subShardNames.get(i);
         params.add(CoreAdminParams.TARGET_CORE, subShardName);
@@ -423,30 +426,31 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         log.debug("Index on shard: {} split into {} successfully", nodeName, subShardNames.size());
       }
 
-      t = timings.sub("applyBufferedUpdates");
-      // apply buffered updates on sub-shards
-      {
-        final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
-
-        for (int i = 0; i < subShardNames.size(); i++) {
-          String subShardName = subShardNames.get(i);
-
-          log.debug("Applying buffered updates on : {}", subShardName);
-
-          params = new ModifiableSolrParams();
-          params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
-          params.set(CoreAdminParams.NAME, subShardName);
-
-          shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
-        }
-
-        String msgOnError = "SPLITSHARD failed while asking sub shard leaders to apply buffered updates";
-        shardRequestTracker.processResponses(results, shardHandler, true, msgOnError);
-        handleFailureOnAsyncRequest(results, msgOnError);
-      }
-      t.stop();
-
-      log.debug("Successfully applied buffered updates on : {}", subShardNames);
+      // nocommit - where do we enter buffering state??
+//      t = timings.sub("applyBufferedUpdates");
+//      // apply buffered updates on sub-shards
+//      {
+//        final ShardRequestTracker shardRequestTracker = ocmh.asyncRequestTracker(asyncId, message.getStr("operation"));
+//
+//        for (int i = 0; i < subShardNames.size(); i++) {
+//          String subShardName = subShardNames.get(i);
+//
+//          log.debug("Applying buffered updates on : {}", subShardName);
+//
+//          params = new ModifiableSolrParams();
+//          params.set(CoreAdminParams.ACTION, CoreAdminParams.CoreAdminAction.REQUESTAPPLYUPDATES.toString());
+//          params.set(CoreAdminParams.NAME, subShardName);
+//
+//          shardRequestTracker.sendShardRequest(nodeName, params, shardHandler);
+//        }
+//
+//        String msgOnError = "SPLITSHARD failed while asking sub shard leaders to apply buffered updates";
+//        shardRequestTracker.processResponses(results, shardHandler, true, msgOnError);
+//        handleFailureOnAsyncRequest(results, msgOnError);
+//      }
+//      t.stop();
+//
+//      log.debug("Successfully applied buffered updates on : {}", subShardNames);
 
       // Replica creation for the new Slices
       // replica placement is controlled by the autoscaling policy framework
@@ -480,7 +484,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
           .onNodes(new ArrayList<>(clusterState.getLiveNodes()))
           .build();
       Assign.AssignStrategyFactory assignStrategyFactory = new Assign.AssignStrategyFactory(ocmh.cloudManager);
-      Assign.AssignStrategy assignStrategy = assignStrategyFactory.create(clusterState, collection);
+      Assign.AssignStrategy assignStrategy = assignStrategyFactory.create();
       List<ReplicaPosition> replicaPositions = assignStrategy.assign(ocmh.cloudManager, assignRequest);
       t.stop();
 
@@ -488,7 +492,7 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       for (ReplicaPosition replicaPosition : replicaPositions) {
         String sliceName = replicaPosition.shard;
         String subShardNodeName = replicaPosition.node;
-        String solrCoreName = Assign.buildSolrCoreName(collectionName, sliceName, replicaPosition.type, replicaPosition.index);
+        String solrCoreName = Assign.buildSolrCoreName(collection, collectionName, sliceName, replicaPosition.type);
 
         log.debug("Creating replica shard {} as part of slice {} of collection {} on {}"
             , solrCoreName, sliceName, collectionName, subShardNodeName);
@@ -497,16 +501,16 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         // avoid a race condition where Overseer may prematurely activate the new sub-slices (and deactivate
         // the parent slice) before all new replicas are added. This situation may lead to a loss of performance
         // because the new shards will be activated with possibly many fewer replicas.
-        ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
-            ZkStateReader.COLLECTION_PROP, collectionName,
-            ZkStateReader.SHARD_ID_PROP, sliceName,
-            ZkStateReader.CORE_NAME_PROP, solrCoreName,
-            ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
-            ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
-            ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
-            ZkStateReader.NODE_NAME_PROP, subShardNodeName,
-            CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
+//        ZkNodeProps props = new ZkNodeProps(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower(),
+//            ZkStateReader.COLLECTION_PROP, collectionName,
+//            ZkStateReader.SHARD_ID_PROP, sliceName,
+//            ZkStateReader.CORE_NAME_PROP, solrCoreName,
+//            ZkStateReader.REPLICA_TYPE, replicaPosition.type.name(),
+//            ZkStateReader.STATE_PROP, Replica.State.DOWN.toString(),
+//            ZkStateReader.BASE_URL_PROP, zkStateReader.getBaseUrlForNodeName(subShardNodeName),
+//            ZkStateReader.NODE_NAME_PROP, subShardNodeName,
+//            CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
+//        ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
 
         HashMap<String, Object> propMap = new HashMap<>();
         propMap.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
@@ -531,35 +535,36 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
         propMap.put(CommonAdminParams.WAIT_FOR_FINAL_STATE, Boolean.toString(waitForFinalState));
 
         replicas.add(propMap);
+        replicaToPosition.put(solrCoreName, replicaPosition);
       }
       t.stop();
       assert TestInjection.injectSplitFailureBeforeReplicaCreation();
 
-      long ephemeralOwner = leaderZnodeStat.getEphemeralOwner();
-      // compare against the ephemeralOwner of the parent leader node
-      leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null);
-      if (leaderZnodeStat == null || ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
-        // put sub-shards in recovery_failed state
-
-        Map<String, Object> propMap = new HashMap<>();
-        propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
-        for (String subSlice : subSlices) {
-          propMap.put(subSlice, Slice.State.RECOVERY_FAILED.toString());
-        }
-        propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
-        ZkNodeProps m = new ZkNodeProps(propMap);
-        ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
-
-        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!");
-        } 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 "
-                  + ephemeralOwner + " to " + leaderZnodeStat.getEphemeralOwner() + ". This can cause data loss so we must abort the split");
-        }
-      }
+//      long ephemeralOwner = leaderZnodeStat.getEphemeralOwner();
+//      // compare against the ephemeralOwner of the parent leader node
+//      leaderZnodeStat = zkStateReader.getZkClient().exists(ZkStateReader.LIVE_NODES_ZKNODE + "/" + parentShardLeader.getNodeName(), null);
+//      if (leaderZnodeStat == null || ephemeralOwner != leaderZnodeStat.getEphemeralOwner()) {
+//        // put sub-shards in recovery_failed state
+//
+//        Map<String, Object> propMap = new HashMap<>();
+//        propMap.put(Overseer.QUEUE_OPERATION, OverseerAction.UPDATESHARDSTATE.toLower());
+//        for (String subSlice : subSlices) {
+//          propMap.put(subSlice, Slice.State.RECOVERY_FAILED.toString());
+//        }
+//        propMap.put(ZkStateReader.COLLECTION_PROP, collectionName);
+//        ZkNodeProps m = new ZkNodeProps(propMap);
+//        ocmh.overseer.offerStateUpdate(Utils.toJSON(m));
+//
+//        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!");
+//        } 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 "
+//                  + ephemeralOwner + " to " + leaderZnodeStat.getEphemeralOwner() + ". This can cause data loss so we must abort the split");
+//        }
+//      }
 
       // we must set the slice state into recovery before actually creating the replica cores
       // this ensures that the logic inside ReplicaMutator to update sub-shard state to 'active'
@@ -599,24 +604,27 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
 
       List<Future> replicaFutures = new ArrayList<>();
       Set<Runnable> replicaRunAfters = ConcurrentHashMap.newKeySet();
+//      for (Map<String, Object> replica : replicas) {
+//        ocmh.addReplica(clusterState, new ZkNodeProps(replica), results);
+//      }
 
       // now actually create replica cores on sub shard nodes
-      for (Map<String, Object> replica : replicas) {
-        ClusterState finalClusterState = clusterState;
-        Future<?> future = ocmh.tpe.submit(() -> {
-          AddReplicaCmd.Response response = null;
-          try {
-            response = ocmh.addReplicaWithResp(finalClusterState, new ZkNodeProps(replica), results, null);
-          } catch (Exception e) {
-            log.error("", e);
-          }
-          if (response != null && response.asyncFinalRunner != null) {
-            replicaRunAfters.add(response.asyncFinalRunner);
-          }
-        });
-
-        replicaFutures.add(future);
-      }
+//      for (Map<String, Object> replica : replicas) {
+//        ClusterState finalClusterState = clusterState;
+//        Future<?> future = ocmh.overseer.getTaskExecutor().submit(() -> {
+//          AddReplicaCmd.Response response = null;
+//          try {
+//            response = ocmh.addReplicaWithResp(finalClusterState, new ZkNodeProps(replica), results, null);
+//          } catch (Exception e) {
+//            log.error("", e);
+//          }
+//          if (response != null && response.asyncFinalRunner != null) {
+//            replicaRunAfters.add(response.asyncFinalRunner);
+//          }
+//        });
+//
+//        replicaFutures.add(future);
+//      }
 
       assert TestInjection.injectSplitFailureAfterReplicaCreation();
 
@@ -628,15 +636,15 @@ public class SplitShardCmd implements OverseerCollectionMessageHandler.Cmd {
       }
       t.stop();
 
-      replicaFutures.forEach(future -> {
-        try {
-          future.get();
-        } catch (InterruptedException e) {
-          log.error("", e);
-        } catch (ExecutionException e) {
-          log.error("", e);
-        }
-      });
+//      replicaFutures.forEach(future -> {
+//        try {
+//          future.get();
+//        } catch (InterruptedException e) {
+//          log.error("", e);
+//        } catch (ExecutionException e) {
+//          log.error("", e);
+//        }
+//      });
 
       log.info("Successfully created all replica shards for all sub-slices {}", subSlices);
... 10276 lines suppressed ...