You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by il...@apache.org on 2024/01/23 04:01:46 UTC

(solr) 01/01: Initial code drop for ZERO replicas. SIP-20 and SOLR-17125

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

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

commit a713415c0f8bc8efff301a478c6baf21701621d5
Author: Ilan Ginzburg <ig...@salesforce.com>
AuthorDate: Tue Jan 23 04:57:18 2024 +0100

    Initial code drop for ZERO replicas. SIP-20 and SOLR-17125
    
    Co-authored-by: Andy Vuong <an...@gmail.com>
    Co-authored-by: Bilal Waheed <mb...@users.noreply.github.com>
    Co-authored-by: Megan Carey <me...@gmail.com>
    Co-authored-by: Olivier Galizzi <og...@salesforce.com>
    Co-authored-by: Pierre Salagnac <pi...@gmail.com>
    Co-authored-by: Andy Throgmorton <an...@andythrogmorton.com>
    Co-authored-by: Yonik Seeley <yo...@apache.org>
    Co-authored-by: Paul McArthur <pm...@salesforce.com>
    Co-authored-by: David Wayne Smiley <ds...@apache.org>
    Co-authored-by: Vincent P <vi...@gmail.com>
    Co-authored-by: Yohann Callea <yo...@gmail.com>
    Co-authored-by: gbellaton <66...@users.noreply.github.com>
    Co-authored-by: Bruno Roustant <br...@gmail.com>
    Co-authored-by: Mathieu Marie <mm...@salesforce.com>
    Co-authored-by: Yannis Hector <yh...@salesforce.com>
    Co-authored-by: Haythem <ha...@yahoo.fr>
    Co-authored-by: Dallas Dias <dd...@salesforce.com>
    Co-authored-by: Chockalingam Viswanathan <c....@salesforce.com>
    Co-authored-by: Prabhdeep Singh Gill <pr...@salesforce.com>
    Co-authored-by: Ilan Ginzburg <il...@murblanc.org>
---
 solr/ZERO-REPLICAS.adoc                            | 890 +++++++++++++++++++++
 .../api/model/CreateCollectionRequestBody.java     |   5 +
 .../client/api/model/CreateShardRequestBody.java   |   5 +
 solr/bin/solr                                      |  16 +
 .../solr/cloud/ShardLeaderElectionContext.java     | 161 ++--
 .../java/org/apache/solr/cloud/ZkController.java   | 161 ++--
 .../solr/cloud/api/collections/AddReplicaCmd.java  |  32 +-
 .../apache/solr/cloud/api/collections/Assign.java  |  34 +-
 .../api/collections/CollectionHandlingUtils.java   |  18 +-
 .../cloud/api/collections/CreateCollectionCmd.java |  52 +-
 .../solr/cloud/api/collections/CreateShardCmd.java |  11 +-
 .../cloud/api/collections/DeleteCollectionCmd.java |   9 +
 .../solr/cloud/api/collections/DeleteShardCmd.java |   6 +
 .../solr/cloud/api/collections/MigrateCmd.java     |   9 +-
 .../api/collections/ReindexCollectionCmd.java      |   7 +-
 .../solr/cloud/api/collections/RestoreCmd.java     |  37 +-
 .../solr/cloud/api/collections/SplitShardCmd.java  |  20 +-
 .../solr/cloud/overseer/ClusterStateMutator.java   |  12 +-
 .../solr/cloud/overseer/CollectionMutator.java     |   4 +-
 .../src/java/org/apache/solr/cluster/Replica.java  |   8 +-
 .../impl/SimpleClusterAbstractionsImpl.java        |   4 +
 .../java/org/apache/solr/core/CoreContainer.java   |  59 +-
 .../src/java/org/apache/solr/core/NodeConfig.java  |  16 +-
 .../java/org/apache/solr/core/SolrXmlConfig.java   |  27 +
 .../src/java/org/apache/solr/core/ZeroConfig.java  | 277 +++++++
 .../apache/solr/handler/ReplicationHandler.java    |  13 +
 .../solr/handler/admin/CollectionsHandler.java     |   5 +-
 .../org/apache/solr/handler/admin/SplitOp.java     |  56 +-
 .../solr/handler/admin/api/CreateCollection.java   |  49 +-
 .../handler/admin/api/RestoreCollectionAPI.java    |   4 +-
 .../apache/solr/handler/admin/api/RestoreCore.java |   9 +
 .../solr/handler/component/SearchHandler.java      |  19 +
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  74 +-
 .../apache/solr/update/DefaultSolrCoreState.java   |   7 +
 .../src/java/org/apache/solr/update/UpdateLog.java |  22 +-
 .../DistributedUpdateProcessorFactory.java         |  11 +-
 .../processor/DistributedZkUpdateProcessor.java    |  67 +-
 .../update/processor/ZeroStoreUpdateProcessor.java | 222 +++++
 .../java/org/apache/solr/zero/client/ZeroFile.java | 248 ++++++
 .../apache/solr/zero/client/ZeroStoreClient.java   | 501 ++++++++++++
 .../solr/zero/client/ZeroStoreClientFactory.java   |  93 +++
 .../org/apache/solr/zero/client/package-info.java  |  19 +
 .../zero/exception/CorruptedCoreException.java     |  25 +
 .../apache/solr/zero/exception/ZeroException.java  |  35 +
 .../solr/zero/exception/ZeroLockException.java     |  27 +
 .../apache/solr/zero/exception/package-info.java   |  19 +
 .../solr/zero/metadata/LocalCoreMetadata.java      | 256 ++++++
 .../solr/zero/metadata/MetadataCacheManager.java   | 455 +++++++++++
 .../zero/metadata/MetadataComparisonResult.java    | 195 +++++
 .../solr/zero/metadata/ZeroMetadataController.java | 504 ++++++++++++
 .../solr/zero/metadata/ZeroMetadataVersion.java    |  67 ++
 .../solr/zero/metadata/ZeroStoreShardMetadata.java | 190 +++++
 .../apache/solr/zero/metadata/package-info.java    |  19 +
 .../solr/zero/process/CollectionDeletionTask.java  |  70 ++
 .../apache/solr/zero/process/CorePullStatus.java   |  65 ++
 .../org/apache/solr/zero/process/CorePuller.java   | 800 ++++++++++++++++++
 .../solr/zero/process/CorePullerBlockingQueue.java | 232 ++++++
 .../org/apache/solr/zero/process/CorePusher.java   | 501 ++++++++++++
 .../solr/zero/process/CorePusherExecutionInfo.java | 151 ++++
 .../apache/solr/zero/process/DeleteProcessor.java  | 196 +++++
 .../org/apache/solr/zero/process/DeleterTask.java  | 181 +++++
 .../solr/zero/process/FilesDeletionTask.java       |  60 ++
 .../solr/zero/process/ShardDeletionTask.java       |  95 +++
 .../apache/solr/zero/process/ZeroAccessLocks.java  | 192 +++++
 .../process/ZeroCoreIndexingBatchProcessor.java    | 337 ++++++++
 .../apache/solr/zero/process/ZeroStoreManager.java | 692 ++++++++++++++++
 .../org/apache/solr/zero/process/package-info.java |  19 +
 .../apache/solr/zero/util/DeduplicatingList.java   | 198 +++++
 .../apache/solr/zero/util/FileTransferCounter.java |  52 ++
 .../apache/solr/zero/util/IndexInputStream.java    | 110 +++
 .../java/org/apache/solr/zero/util/ToFromJson.java |  46 ++
 .../org/apache/solr/zero/util/package-info.java    |  19 +
 .../configsets/cloud-minimal-zero/conf/schema.xml  |  29 +
 .../cloud-minimal-zero/conf/solrconfig.xml         |  53 ++
 .../zero-distrib-indexing/conf/schema.xml          |  29 +
 .../zero-distrib-indexing/conf/solrconfig.xml      |  57 ++
 .../solr/cloud/api/collections/ShardSplitTest.java |   5 +
 .../plugins/AffinityPlacementFactoryTest.java      | 134 ++--
 .../plugins/MinimizeCoresPlacementFactoryTest.java |   4 +-
 .../solr/zero/client/ZeroStoreClientTest.java      | 460 +++++++++++
 .../ZeroCollectionBackupRestoreTest.java           | 289 +++++++
 .../ZeroStoreShardMetadataEvictionTest.java        | 174 ++++
 .../ZeroStoreShardMetadataTest.java                | 161 ++++
 .../ZeroStoreSimpleCollectionTest.java             | 194 +++++
 .../zero/cloudapicollections/package-info.java     |  22 +
 .../solr/zero/metadata/FileDeleteStrategyTest.java | 172 ++++
 .../solr/zero/metadata/LocalCoreMetadataTest.java  | 412 ++++++++++
 .../ZeroMetadataVersionControllerTest.java         | 229 ++++++
 .../zero/metadata/ZeroStoreShardMetadataTest.java  | 138 ++++
 .../solr/zero/process/CoreCorruptionTest.java      | 435 ++++++++++
 .../apache/solr/zero/process/CorePullerTest.java   | 537 +++++++++++++
 .../apache/solr/zero/process/CorePusherTest.java   | 186 +++++
 .../solr/zero/process/DeleteProcessorTest.java     | 628 +++++++++++++++
 .../zero/process/PullMergeDeduplicationTest.java   | 132 +++
 .../process/SimpleZeroStoreEndToEndPullTest.java   | 144 ++++
 .../process/SimpleZeroStoreEndToEndPushTest.java   | 115 +++
 .../solr/zero/process/ZeroCollectionTestUtil.java  | 111 +++
 .../solr/zero/process/ZeroCoreConcurrencyTest.java | 882 ++++++++++++++++++++
 .../solr/zero/process/ZeroCoreDiscoveryTest.java   | 349 ++++++++
 .../ZeroCoreIndexingBatchProcessorTest.java        | 376 +++++++++
 .../zero/process/ZeroStoreDeletionProcessTest.java | 355 ++++++++
 .../process/ZeroStoreDistributedIndexingTest.java  | 358 +++++++++
 .../solr/zero/process/ZeroStoreManagerTest.java    |  84 ++
 .../zero/process/ZeroStoreMissingCoreTest.java     | 414 ++++++++++
 .../zero/process/ZeroStoreReplicaRecoveryTest.java |  65 ++
 .../zero/process/ZeroStoreSolrCloudTestCase.java   | 368 +++++++++
 .../solr/zero/process/ZeroStoreSplitTest.java      | 632 +++++++++++++++
 .../solr/zero/util/DeduplicationListTest.java      | 233 ++++++
 .../solr/zero/util/IndexInputStreamTest.java       |  93 +++
 .../client/solrj/cloud/DistribStateManager.java    |   3 +
 .../client/solrj/impl/ZkDistribStateManager.java   |  20 +
 .../apache/solr/common/cloud/ZkStateReader.java    |   2 +
 .../solrj/request/CollectionAdminRequest.java      |  74 +-
 .../apache/solr/common/cloud/DocCollection.java    |  28 +
 .../java/org/apache/solr/common/cloud/Replica.java |  16 +-
 .../org/apache/solr/common/cloud/ReplicaCount.java |  71 +-
 .../solr/common/params/CollectionAdminParams.java  |   3 +
 .../apache/solr/common/cloud/ReplicaCountTest.java |  37 +-
 .../apache/solr/cloud/MiniSolrCloudCluster.java    |  20 +-
 .../org/apache/solr/cloud/SolrCloudTestCase.java   |  11 +
 .../apache/solr/cluster/placement/Builders.java    |  19 +-
 .../solr/cluster/placement/BuildersTest.java       |   7 +-
 .../web/js/angular/controllers/collections.js      |   4 +-
 solr/webapp/web/partials/collection_overview.html  |   5 +
 solr/webapp/web/partials/collections.html          |  18 +-
 125 files changed, 18181 insertions(+), 323 deletions(-)

diff --git a/solr/ZERO-REPLICAS.adoc b/solr/ZERO-REPLICAS.adoc
new file mode 100644
index 00000000000..c829cdfd82c
--- /dev/null
+++ b/solr/ZERO-REPLICAS.adoc
@@ -0,0 +1,890 @@
+////
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+////
+
+= Zero replica type and storage
+:toc: macro
+:toclevels: 3
+
+January 2024
+
+toc::[]
+
+== Introduction
+
+The code in this branch https://github.com/apache/solr/tree/jira/solr-17125-zero-replicas[[.underline]#jira/solr-17125-zero-replicas#] is an initial and work in progress contribution
+proposal allowing for a separation of compute and storage in SolrCloud,
+as introduced by
+https://cwiki.apache.org/confluence/display/SOLR/SIP-20%3A+Separation+of+Compute+and+Storage+in+SolrCloud[[.underline]#SIP-20#]
+and referenced in Jira ticket
+https://issues.apache.org/jira/browse/SOLR-17125[[.underline]#SOLR-17125#]
+- _SIP-20 ZERO replicas, separation of compute and storage_.
+
+See <<Past plans>> on previous attempts at
+implementing this (or something similar).
+
+== Very high level overview
+
+The proposal is architected around:
+
+* Introduction of a new replica type called `ZERO` (`Replica.Type.ZERO`)
+augmenting the existing set of `NRT`, `TLOG` and `PULL` types, and a way to
+create collections based on this new replica type.
+* A shared repository called the Zero store in which the segments of each
+shard with `ZERO` replicas are persisted (each shard is persisted only
+once).
+* Local SolrCloud node disks used as caches (a better wording might be
+“non-persistent disks”). No data loss happens if a node is shut down at
+any given time or if all nodes are shut down at any given time and
+restart with empty disks
+* A guarantee that there is no data loss if two nodes consider themselves
+leaders of the same shard and try to update the Zero store concurrently.
+
+`ZERO` replicas use a local disk like other replicas usually do, push
+(write) data to the Zero store after indexing and pull (read) data from
+the Zero store when the local disk is not up to date (for serving
+queries, for indexing or any other need). This enables the local disk
+used by `ZERO` replicas to be ephemeral (i.e. can start empty on each node
+restart). By extension, a `ZERO` shard is a shard having replicas of type
+`ZERO` (and then they are all of type ZERO, mix and match with standard
+replica types is not supported) and being part of a `ZERO` collection (a
+collection that again has only `ZERO` shards).
+
+The principles governing how that new replica type is used and the
+design of the Zero store integration:
+
+* `ZERO` replicas read from and write to the Zero store (e.g. AWS S3, Google
+GCS etc.) and do not directly communicate with each other like standard
+SolrCloud replicas do. There is no Solr-to-Solr replication for these
+replicas.
+* *The Zero store content is the source of truth for index (segment)
+data*. No node specific state is ever needed to recover an index,
+regardless of how the cluster or the node was shut down or crashed.
+Nodes are stateless.
+* There is a single image on the Zero store per shard (one image and zero
+copies, hence the naming, and it also sounded cool), as opposed to an
+image per replica which implies data duplication, and in the SolrCloud
+context requires peer to peer communication between replicas.
+
+In a nutshell, the cores (replicas) that are expected on each node are
+identified by looking at ZooKeeper collection states. Replicas of type
+`ZERO` that are assigned to a given node but do not exist on its disk are
+created empty at startup (the node might have started with an empty
+disk). SolrCloud is then happy and can do the usual core opening, shard
+leader election etc. Actual data for the replica is fetched (pulled)
+from the Zero store when and if needed.
+
+Pulls may happen asynchronously when queries are served by non leader
+replicas (to refresh the locally cached state of the replica) and
+synchronously when a leader replica processes an indexing batch if that
+replica is not up to date: the replica first gets itself up to date with
+the Zero store if needed, processes locally the indexing work, pushes
+the updated segments and metadata to the Zero store and only then
+acknowledges the batch to the client.
+
+This is a current limitation whereas each indexing batch is committed
+then pushed to the Zero store before being acknowledged to the client.
+Work is planned to remedy this, see <<Future plans>>.
+
+Because in a SolrCloud cluster two replicas can consider themselves
+shard leaders at the same time (a replica might still be executing code
+assuming leadership after the quorum has voted it out, this is
+unavoidable unless leadership change waits for the outgoing leader to
+acknowledge, which does not work if the outgoing leader is really gone),
+a strategy was put in place for writes in which two concurrent replicas
+writing to the Zero store for a given shard do not overwrite each other
+and only one succeeds. This strategy uses unique filenames on the Zero
+store (no opportunity for overwrites at that level) and uses a compare
+and set operation in ZooKeeper (a.k.a. conditional update) to pick which
+one wins the race. The non winner reports a failure to the client that
+would then retry. This is expected to be a rare occurrence.
+
+== The Zero store
+
+The Zero store is a shared store accessible from all the nodes and
+stores for each `ZERO` shard the different files that constitute the
+corresponding core (segments files and `segments_N` file).
+
+The `BackupRepository` implementation is reused (it should be renamed if
+this code is merged since it would no longer be backup specific).
+Various implementations already exist in Solr code, for example for AWS
+S3 (`S3BackupRepository`), Google GCS (`GCSBackupRepository`) or a local
+file system implementation (`LocalFileSystemRepository`) for testing.
+
+The storage structure for each shard is described in
+<<The Zero store shard storage structure>>.
+
+== The Zero implementation
+
+An attempt was made to isolate Zero code in specific classes to make
+integration easier. This section describes the new classes introduced by
+the Zero implementation.
+
+These classes (and tests for that matter) are in subpackages of
+`org.apache.solr.zero` (in module `core`). This is a new package that does
+not exist in stock SolrCloud.
+
+The only new class being elsewhere is `*ZeroConfig*` which is in
+`org.apache.solr.core` like other configuration classes. It allows
+tweaking parameters of the Zero implementation and defining the
+`BackupRepository` to use for the Zero store.
+
+* In `org.apache.solr.zero.*client*`:
+** `*ZeroStoreClient*` is an “internal” client for Zero operations used
+within the Zero code.
+** `*ZeroStoreClientFactory*` this class instantiates `ZeroStoreClient` and
+also builds the `BackupRepository` used for the Zero store.
+** `*ZeroFile*` represents a file on the Zero store, possibly with a mapping
+to its local cached copy.
+* In `org.apache.solr.zero.*exception*`: Zero operations related exceptions.
+** `*CorruptedCoreException*`: used when a core pulled from the Zero store
+can’t be opened
+** `*ZeroException*`: generic exception for Zero implementation issues
+** `*ZeroLockException*`: inability to acquire a lock
+* In `org.apache.solr.zero.*metadata*`: classes dealing with Zero store and
+local metadata of a core.
+** `*ZeroStoreShardMetadata*` is a class serialized into (or deserialized
+from) a file stored in the Zero store that is the “root” file for
+finding the structure and content of a shard. This metadata includes the
+list of all files (with their Zero store names and local disk names)
+that make up the shard and can be used to create a replica of the
+shard. +
+This class also contains a list of files from prior commit points of the
+shard still present in the Zero store but no longer used and that should
+be deleted (they eventually are).
+** `*ServerSideMetadata*` the Solr node view of a replica (core) index.
+Whereas the Zero equivalent class `ZeroStoreShardMetadata` is “pure”
+metadata class, `ServerSideMetadata` also contains utility methods to
+build an instance from a local core and other helpers.
+** `*ZeroMetadataController*` deals with tracking shard metadata in ZooKeeper
+(needed in addition to actual data store in the Zero store) and provides
+comparison methods for instances of `ServerSideMetadata` and of
+`ZeroStoreShardMetadata` to compute which files need to be pulled from the
+Zero store (`diffMetadataForPull()`) or pushed from the local core to the
+Zero store (`diffMetadataForPush()`).
+** `*ZeroMetadataVersion*` a very simple class capturing the metadata suffix
+value and ZooKeeper node version, used for tracking in ZooKeeper which
+Zero store file (representing `ZeroStoreShardMetadata`) is the current
+version of the shard.
+** `*CoreMetadataComparisonResult*` captures the result of the comparison of
+the local server core and the remote Zero store metadata version of that
+shard.
+** `*MetadataCacheManager*` manages the per shard metadata cache for the
+cores (i.e. replicas) present on a node as well as some local core
+lifecycle wrt the Zero store.
+* In `org.apache.solr.zero.*process*`:
+** [red]#`*ZeroStoreManager*` is the entry point for interactions of Solr code with
+the Zero store. It is the class accessed from existing SolrCloud classes
+to do Zero store related operations.#
+** `*CorePuller*` deals with pulling a core from the Zero store and creating
+or updating a local copy.
+** `*CorePullStatus*`: an enum of the different ways a pull can end (happily
+or not).
+** `*CorePullerBlockingQueue*` provides a wrapper around a `DeduplicatingList`
+to be used in an `Executor` executing the pulls by calling `CorePuller`.
+This allows enqueuing (too) many pulls for a core without redundantly
+executing all of them. Pulls are currently enqueued from `HttpSolrCall`
+(for query requests) using
+`ZeroStoreManager.enqueueCorePullFromZeroStore`.
+** `*CorePusher*` deals with pushing a local core (or parts of it) from the
+local file system to the Zero store.
+** `*CorePusherExecutionInfo*`: captures the outcome of a core push
+** `*DeleteProcessor*` deals with deleting files on the Zero store. It
+enqueues for asynchronous execution delete tasks for different types of
+Zero file removal. Steady state file removal uses `deleteFiles()`,
+Overseer command execution uses `deleteShard()` and `deleteCollection()`.
+** `*DeleterTask*`, `*FilesDeletionTask*`, `*ShardDeletionTask*` and
+`*CollectionDeletionTask*` are different strategies to delete files by the
+`DeleteProcessor`.
+** `*ZeroAccessLocks*`: the locking logic to protect pull and push operations
+as well as indexing
+** `*ZeroCoreIndexingBatchProcessor*`: methods used at different stages of
+indexing into Zero replicas.
+* In `org.apache.solr.zero.*util*`:
+** `*DeduplicatingList*` a list-like data structure that deduplicates
+inserted entries by merging with existing entries.
+** `*FileTransferCounter*`: stats for core push and pulls
+** `*IndexInputStream*`: the stream used to push a local file to the Zero
+store
+** `*ToFromJson*`: the class converting `ZeroStoreShardMetadata` instances to
+files and vice versa
+
+== Configuring SolrCloud to support `ZERO` collections
+
+A SolrCloud cluster can support normal replica types (`NRT`, `TLOG` and
+`PULL`) and also Zero store based collections using replicas of type
+`Replica.Type.ZERO` (`ZERO` replicas for short). By extension, a collection
+based on `ZERO` replicas is referred to as a `ZERO` collection.
+
+SolrCloud needs to be configured with a Zero store to enable `ZERO`
+replicas. This is done by setting system property `zeroStoreEnabled=true`.
+During initialization of `CoreContainer` in `loadInternal()`, if that
+property is `true` the Zero implementation is initialized by creating an
+instance of `ZeroStoreManager`. The decision to enable or not the Zero
+implementation is only done at startup.
+
+In order to enable the Zero store, a repository must also be configured
+for it. This is using the configuration section `<repository>` in section
+`<repositories>` in section `<zero>` of `solr.xml`. That configuration
+contains the class of the repository implementation as well as
+additional parameters. See the backup-restore section of the deployment
+guide for details.
+
+== The Zero store shard storage structure
+
+All files of a shard, for the lifetime of that shard storage on the Zero
+store, are stored under a single “repository” (path or directory).
+
+All files corresponding to the current (latest) acknowledged commit
+point of that shard are present (using different names than the local
+file names) as well as a file that is specific to the Zero store called
+`shard.metadata._xxx_` (the `_xxx_` is a random UUID suffix) that contains
+the serialization of an instance of `ZeroStoreShardMetadata` for the
+shard. This structure contains the mapping between the Zero store names
+of the shard files and the local file system names of these files as
+expected by Solr.
+
+The Zero store file names are the Solr file names (for example
+`_y45.cfs`) to which a random suffix is added so that two nodes pushing
+Solr files with identical names do not collide or overwrite each other.
+The Zero store file name for the above file could for example be
+`_y45.cfs.84d9443a-62f8-45a7-9015-64dff429c6ca`.
+
+The suffix is the same type of suffix used for the `shard.metadata` file.
+Even though all such suffixes could be different, to make debugging
+easier, all files pushed to the Zero store in a single push “session”
+are using the same random suffix.
+
+This means that `shard.metadata.84d9443a-62f8-45a7-9015-64dff429c6ca`
+corresponds to the version of the index in which
+`_y45.cfs.84d9443a-62f8-45a7-9015-64dff429c6ca` was initially
+introduced. Of course, each `shard.metadata.xxx` can (and usually does
+except the first one for a shard) contain references to files having
+been initially added in different push sessions and therefore having
+different suffixes.
+
+The role of the suffix is to guarantee correct behavior when (unlikely
+and rare but does happen) two nodes concurrently update the same shard.
+See section
+<<Locking and managing concurrent Zero activity>>.
+
+== Creating `ZERO` collections
+
+The admin UI is changed to allow the creation of a `ZERO` collection which
+is a collection of shards having only `ZERO` replicas. A `zeroIndex`
+attribute (defaulting to `false`) is added to the `showAddCollection`
+function in `collections.js` which is referenced from the `collections.html`
+Solr admin interface.
+
+`Create` (in `CollectionAdminRequest`) is invoked by the `CollectionsHandler`
+when creating a collection. A flag is added indicating if the collection
+is `ZERO` and the number of `ZERO` replicas it should have. Collections
+backed by the Zero store can only have `ZERO` replicas.
+
+`CreateCollectionCmd`, `CreateShardCmd` and `AddReplicaCmd` (the Overseer side
+of Collection API commands) are modified accordingly to deal with the
+new replica type and attributes.
+
+The path to the data stored on the Zero store is composed of the
+collection name and the shard name. No replica or core specific names
+given the storage is per shard.
+See method `ZeroFile.getShardDirectoryURI()` that delegates part of the
+work to the underlying `BackupRepository` (the actual name/path is
+therefore implementation dependent).
+
+Note that the first push to the Zero store after collection creation is
+done when processing the first indexing batch for the shard. Remains to
+be verified if the cluster correctly deals with shards never pushed to
+the Zero store (in case of node restart, or backup being done etc).
+
+== Node startup
+
+When a node starts, it might not contain the data of its `ZERO` replicas
+and might not even contain the directory structure where these
+cores/replicas would go. The replicas assigned to a node are listed in
+ZooKeeper (in the various collections’ `state.json` files) and the
+corresponding data is available in the Zero store for `ZERO` replicas
+(after some resolution process involving Zero related metadata stored in
+ZooKeeper, see <<Access to Zero store>>).
+
+Given SolrCloud opens all cores (a replica is
+[line-through]#materialized by# a core) at startup and makes the
+replicas participate in the shard leader election for their shard, in
+order not to overwhelm the system and load large amount of data from the
+Zero store then open many “real” cores (i.e. cores with data) that might
+never get used, a tradeoff was implemented. All cores for `ZERO` replicas
+(or a shorthand: `ZERO` cores) are created empty locally and opened. That
+way they can participate in shard leader election as expected. These
+replicas are marked `ACTIVE`, even though initially they are empty! See
+also section <<Querying>>.
+
+When/if replicas are accessed and require the actual data, that data
+will be fetched (pulled) from the Zero store. Note that if the local
+disk already has data for a `ZERO` core (from a previous run of the Solr
+JVM if the volume is persistent) then the core will be open as is. It
+will eventually get refreshed to the data corresponding to it in the
+Zero store (the state of the Zero store for a shard is the source of
+truth, not the state on the local disk) if it is queried or indexed.
+
+Method `CoreContainer.discoverAdditionalCoreDescriptorsForZeroReplicas()`
+is called from `CoreContainer.loadInternal()`, it iterates over all the
+collections of the cluster and for `ZERO` collections creates a
+`CoreDescriptor` for each replica (if any) that should be on the local
+node (unless that replica was already found locally), adds it to the set
+of cores discovered locally (returned by
+`CorePropertiesLocator.discover()`) and creates the `core.properties` file
+locally at the appropriate path (this file is a standard Solr file that
+is unrelated to the `shard.metadata.xxx` file used by `ZERO` replicas).
+
+At a later stage of `CoreContainer.loadInternal()`, all cores (found
+locally and added for `ZERO` replicas) are opened (if so configured using
+`loadOnStartup` defaulting to `true`, with `false` not a recommended setting
+in SolrCloud mode).
+
+== Access to Zero store
+
+Writes to the Zero store are done by the leader replica of a shard for
+pushing newly/recently indexed documents. It is possible to have more
+than one replica considering itself as a shard leader (for the same
+shard) as leadership changes. In such a case there’s a need to both
+protect the consistency of data in the Zero store (to avoid corruption
+due to a mix and match of incompatible core files written by different
+nodes) and also make sure no data is lost due to being overwritten (e.g.
+node A write a segment, node B then overwrites that segment with an
+identically named segment, causing the data written by node A to be
+lost, even though node A might have acknowledged to the client the
+received indexing batch).
+
+In a nutshell (see
+<<The Zero store shard storage structure>>), writes to the Zero store use unique names
+and never overwrite any file (the uniqueness is guaranteed by adding a
+random suffix to the file names). The `shard.metadata._suffix_` file
+contains the list of all files on the Zero store that constitute the
+core (it references files previously pushed and still needed as well as
+the new set of files just written). Once all these writes (the segment
+files and `shard.metadata._suffix_`) have successfully completed, an
+update is made to a shard specific node in ZooKeeper to capture the
+`suffix` of the latest version of the `shard.metadata` file. That update is
+made by the indexing thread as a conditional update (checking that the
+version of the pre update ZooKeeper node hasn’t change from when
+indexing started), to make sure no update from another node sneaked in
+between the moment the indexing thread pulled the latest content from
+the Zero store before starting indexing and when it pushes its updates
+back to the Zero store after finishing indexing.
+
+If the conditional update succeeds, the indexing content and changes to
+the core have been correctly persisted and will not get overwritten. If
+the conditional update to ZooKeeper fails, the indexing batch fails. It
+will have to be executed again after having pulled the latest copy of
+the core from the Zero store. This means some other update from another
+node sneaked in and made an update to that shard on the Zero store, and
+given we don’t want to lose (overwrite) that update, the new indexing
+batch has to be redone on top of that update (the retry is not automatic
+although it could be made to be, the client gets a failure return for
+its indexing batch and has to resubmit).
+
+The above protects concurrent updates from different SolrCloud nodes to
+a shard on the Zero store. This should not happen often, but it does
+happen when a shard leader loses leadership while processing an indexing
+batch and continues processing while the new leader starts processing
+another indexing batch.
+
+Management of the ZooKeeper `metadataSuffix` node for the shard is done in
+`ZeroMetadataController`. The actual metadata is stored in
+`ZeroMetadataVersion` (the suffix itself and the corresponding ZooKeeper
+version for the conditional update), a cache of the data is maintained
+in `MetadataCacheManager`.
+
+In addition to the correctness guarantees based on the above, we don’t
+want inefficiency in the access to the Zero store from within a single
+Solr node, which is the most common case.
+
+For example when two indexing batches are executed concurrently on a
+given replica, we don’t want both to try to push data concurrently
+knowing one of the batches will have to fail. Also when two concurrent
+requests arrive for the same replica that both need to pull data from
+the Zero store, we don’t want to pull files twice.
+
+Therefore we need synchronization on pulls and pushes that happen for a
+replica on a given node. This is described in the next section.
+
+Synchronization with or more precisely protection against what happens
+through a different replica of the same shard on another node is taken
+care of by the mechanism described above.
+
+== Per core Zero related metadata
+
+`MetadataCacheManager` is the class managing the state of a core (replica)
+related to Zero store interactions as seen from a SolrCloud node. It
+tracks failures pulling a core from the Zero store (to avoid retrying
+forever). For each core it caches an instance of static inner class
+`MetadataCacheEntry`.
+
+For each local replica, the last known `metadataSuffix` and associated
+ZooKeeper node `version` are tracked. The version will be used for
+conditional updates when writes (pushes) are done to the Zero store in
+order to detect if the Zero store content has changed (in which case the
+conditional update fails). A boolean `cacheLikelyUpToDate` is a hint
+indicating the server considers that the cached `metadataSuffix` and
+version of MetadataCacheEntry are current (to skip re-reading them from
+ZooKeeper). If the values end up being not current, a conditional update
+failure will trigger a refresh from ZooKeeper and when needed retrieving
+latest content from the Zero store..
+
+=== Locking and managing concurrent Zero activity
+
+==== Overview
+
+Locks must be acquired to manipulate a replica. Locks are defined and
+used via class `ZeroAccessLocks`. An inner class `CloseableLock` wrapping
+locks allows acquiring the lock in a try-with-resources to have it
+released automatically (this is not always possible so locks are also
+acquired and released in a more usual way).
+
+There are 3 locks: for pulling, for pushing and for indexing. Internally
+these are implemented using a read-write lock for indexing and pull, and
+a simple lock for push.
+
+The idea is to allow parallel indexing batches (multiple threads can
+acquire the indexing lock concurrently), but disallow any pull while
+indexing takes place. Also only allow a single push at a time.
+
+Follows a summary of the ways different operations interact when they
+happen at the same time on the same or different nodes. The following
+sections then get into more details.
+
+Locking within a node:
+
+[width="100%",cols="26%,37%,37%",]
+|===
+| |Pull |Indexing
+
+|Pull a|
+For efficiency, serialize pulls (to avoid pulling twice the same files).
+
+_→ Pulls use a lock for mutual exclusion_
+
+a|
+No pulls while indexing is in progress to allow indexing to rely on the
+cached `metadataSuffix` node version.
+
+_→ Indexing uses a lock that allows other indexing but prohibits pulls_
+
+|Indexing >|Same as top right ↗️ a|
+Actual indexing can proceed in parallel (like stock SolrCloud), pushes
+are serialized.
+
+_→ Indexing uses a lock that allows other indexing but prohibits pulls_
+
+_→ The push to Zero store phase is done under *another mutual exclusion*
+lock_
+
+|===
+
+Interactions between two different nodes:
+
+[width="100%",cols="26%,37%,37%",]
+|===
+| |Pull on node B |Indexing on node B
+
+|Pull on node A |No limitations, can proceed in parallel |No
+limitations, can proceed in parallel
+
+|Indexing on node A >|Same as top right ↗️ |Edge case of shard leader tenure overlap. One
+batch succeeds, the other one fails. No data loss.
+|===
+
+==== Locking for pushing
+
+For pushes, acquiring the push lock guarantees there is no more than a
+single push at a time. If multiple pushes were allowed in parallel they
+would be pushing the same data and only one of them would succeed
+anyway.
+
+This lock exists for efficiency, not correctness.
+
+==== Locking for indexing and pulling
+
+Locking for indexing allows parallel indexing but prevents pulling while
+indexing is in progress (think indexing locking acquires a read lock and
+pulling acquires the write lock). If new shard content on the Zero store
+is becoming available from another node while a node is indexing (should
+be relatively rare), the indexing batch will eventually fail: the
+`metadataSuffix` node in ZooKeeper will not have the expected version due
+to the update done by another node.
+
+If pulling was allowed during indexing and updated the `version` of the
+cached `metadataSuffix` node, the ZooKeeper conditional update at the end
+of indexing could succeed. There would be a risk of data loss or
+conflict between the pulled content and locally written content (locally
+generated segment with same number as pulled segment). If pulling was
+allowed without updating the cached `version`, indexing would fail anyway.
+That’s why pulling is not permitted while indexing is in progress. The
+case this is protecting against is rare to very rare (two leaders for
+the same shard writing at more or less the same time) so no lock
+contention expected here.
+
+A naive solution to that issue would be to have each indexing batch
+record the version of the `metadataSuffix` node at the start of indexing
+(representing the version of the shard on top of which the indexing is
+done), and at the end of indexing do the conditional update using this
+version, regardless of pulls that might have occurred in the meantime on
+that node. This would work, and in case of a remote update to the Zero
+store shard data, the indexing batch would fail as expected.
+
+But this approach would be causing local inefficiencies and is not
+realistic if concurrent indexing batches on a replica are allowed: when
+two such batches run (starting from the same initial state of the
+`metadataSuffix` node), the first to finish would succeed and update the
+`metadataSuffix` node and the second would systematically fail. In
+practice this would mean no concurrent indexing.
+
+This is the reason that multiple indexing batches can proceed in
+parallel but no pulling from Zero store is allowed while an indexing is
+in progress.
+
+The indexing lock must be continuously held from before indexing starts
+to after the push that occurs at the end of indexing has completed.
+Before the push starts, the push lock must be acquired.
+
+If the indexing lock was released at the end of local indexing and
+commit and before the push to Zero store started, a pull could then
+happen. The pulled data could conflict with local core data, and
+regardless of the resolution choice (keeping the local segments or
+remote ones coming from the Zero store) the subsequent push - if not
+made to fail - would be causing some loss of data.
+
+Therefore, for correctness it is required to either prevent pulls from
+happening during indexing or making sure indexing fails if a pull does
+happen during indexing. The proposed implementation picked the first
+option but both are equivalent really (the indexing batch would fail in
+both if there was a need to do a pull, whether or not that pull was
+done).
+
+==== Indexing lifecycle with `ZERO` replicas
+
+The way concurrent indexing works with the Zero store is that
+Solr/Lucene is managing segment numbering and writes to the local index
+in the normal way. When a commit completes and the first of concurrent
+indexing batches completes, local files are pushed to the Zero store
+using unique filenames and a conditional update to the `metadataSuffix`
+ZooKeeper node is done with the `version` cached on this Solr node. This
+causes an increase of the `metadataSuffix` node version (`metadataSuffix` is
+always updated at the end of a push, this is what makes the pushed data
+visible - note this means each indexing batch with `ZERO` replicas does a
+write to ZooKeeper, this is not the case with existing replica types). A
+second indexing thread that has completed and committed waits for the
+push lock and for the first thread to complete the push and release the
+lock. The second thread then pushes to the Zero store the new content
+(diff between local index commit point and Zero store contents) and
+updates again the `metadataSuffix` node. It’s quite possible and even
+probable that the push at the end of the first batch did write to the
+Zero store the updates done by both batches (which might have resulted
+in one or more segments on disk), in which case the second thread, after
+waiting for the first push to finish, will discover that no additional
+files need to be pushed, will quickly complete and acknowledge the
+second batch.
+
+Note that when two different nodes push to the Zero store at the same
+time, only one batch succeeds and the other one fails, no data is lost
+nor overwritten. Current implementation will return a failure to the
+client (in the future the batch could be retried after refreshing the
+local replica state so it succeeds).
+
+==== Locking summary
+
+To sum it up, locking on a given SolrCloud node allows using updated
+versions of the `metadataSuffix` ZooKeeper node by indexing batches and
+therefore allows concurrent indexing on a node.
+
+Writing files with unique names to the Zero store and validating the
+actual write by a conditional update to the ZooKeeper `metadataSuffix`
+node allows managing consistency and guaranteeing no data loss in the
+Zero store when multiple nodes attempt to update the same shard on the
+Zero store (rare but not impossible, has been observed in production).
+
+Here’s a summary of the ways different operations interact when they
+happen at the same time on the same or different nodes.
+
+== Querying
+
+As described in <<Node startup>>, cores
+might start their life empty (local disk is a cache, source of truth is
+the Zero store). Currently, loading a core from the Zero store is
+triggered by either a query or an indexing request addressed to the
+core.
+
+Pulls triggered by queries are done asynchronously. When receiving a
+request which is not an `/update`, a pull is enqueued. This is done very
+early in the request lifecycle in `HttpSolrCall.init()` (this should be
+changed and moved to the handlers/components that actually need the core
+content).
+
+In `SearchHandler.handleRequestBody()`, there’s a stopgap check to ensure
+the core was pulled from Zero store at least once. If it wasn’t, the
+query is rejected right away. This ensures we never answer to a query
+with no results, mostly because we create empty cores at node startup
+for all the replicas that are discovered from ZooKeeper. An outright
+rejection causes other layers of SolrCloud to query (for a while)
+replicas on other nodes, letting the node with missing data refresh
+itself and be able to then handle queries. Rolling cluster restarts need
+somewhat more work to guarantee that there’s always at least one up to
+date replica to serve queries (if all nodes restart in quick sequence,
+all replicas of a shard might not have refreshed themselves from the
+Zero store, causing an outage). Code handling this case is not too
+complex and can be contributed later.
+
+This does not ensure the local core on a non-leader replica is always
+up-to-date. We may still process a query without having the very recent
+updates to the replica if they were done in another node. This is not so
+different from what happens with `TLOG` or `PULL` replicas, where
+replication between Solr nodes is not real time.
+
+In the Zero store case though, the update could be much older. For
+example the non leader replica got updated a week ago when it was
+leader. It will be non empty from that point on, but the quality of
+results it will be able to serve might not be very high. If queries
+target all replicas and there is a constant stream, then non leader
+replicas will always be more or less up to date. If there’s only
+indexing and no queries, the first queries on the non leader replica
+might be served based on old replica contents.
+
+Pull requests are enqueued after each query (this should be improved),
+but as they’re waiting in a queue to be executed, new requests are
+deduplicated. This means there is at most a single outstanding pull
+request for a replica, a subsequent pull request will only be enqueued
+if a new query arrives.
+
+== Indexing
+
+The first location in code where we have custom code for indexing for
+`ZERO` replicas is class `DistributedUpdateProcessorFactory`. For ZERO
+replicas, instead of using the standard `DistributedZkUpdateProcessor`,
+subclass `ZeroStoreUpdateProcessor` is used. This is mostly a hook to make
+sure the core is pulled from Zero store before processing the batch, and
+then pushed back to Zero store when the batch is complete. This happens
+respectively:
+
+[arabic]
+. In `postSetupHook()` (added in the parent class and overridden in the
+subclass) that invokes
+`ZeroCoreIndexingBatchProcessor.addOrDeleteGoingToBeIndexedLocally()`.
+. In `processCommit()` (standard method overridden from the parent class)
+that invokes
+`ZeroCoreIndexingBatchProcessor.hardCommitCompletedLocally()`.
+
+
+There is also code specific to `ZERO` replica in class `HttpSolrCall` to
+always add the `commit=true` parameter to the indexing batch. This code
+will be removed once the Transaction Log is adapted to work at the shard
+level with `ZERO` replicas. See <<Future plans>>.
+
+Indexing itself proceeds normally all the way to the commit. Push to
+Zero store happens after the new segments have been created on the local
+file system.
+
+== Shard leader elections
+
+The differences in leader election for `ZERO` replicas are minimal, and
+reside in whether we allow the replica that wins the election to
+actually become the leader replica (normal upstream leader election
+first elects a replica *then* decides if it is fit to become the leader
+based on shard terms) and as a consequence how long/if a replica waits
+for other replicas to join the election (with `ZERO` replicas no need to
+wait).
+
+When indexing to a `ZERO` replica, the latest available data will always
+be available since the source of truth (Zero store) is available to all
+replicas. Therefore the `ZERO` replica that wins the leader election is
+always legitimate to actually be the leader. In the method
+`ShardLeaderElectionContext.runLeaderProcess()`, we skip checks related to
+shard terms for `ZERO` replicas (and given that unlike `NRT` and `TLOG` there
+is no forwarding of indexing batches with `ZERO` replicas, shard terms are
+not even maintained, so it’s good news they’re not needed).
+
+Strictly speaking, for correctness and durability there is no need to
+even do shard leader elections for `ZERO` replicas. The protections around
+how the Zero store can be updated guarantees that any replica doing
+indexing will end up doing the right thing: reject an indexing request
+if it doesn’t have the last data of the shard and update itself with the
+latest data to be able to process subsequent indexing requests. But
+given shard leader election is so pervasive in SolrCloud, the ZERO
+replica implementation at least in its current state conforms to the
+general logic with only small adjustments as described above.
+
+== Admin operations
+
+=== Collection creation
+
+Collection creation in `CreateCollectionCmd` creates the `metadataSuffix`
+ZooKeeper node when creating a `ZERO` collection.
+
+=== Restore
+
+`RestoreCmd` can restore a "normal" collection as `ZERO` (this can be used
+for testing the `ZERO` feature) and vice versa.
+
+There aren’t many changes to the restore code overall to support `ZERO`.
+`RestoreCore` does push the restored core to the Zero store though.
+
+=== Shard Splits
+
+Core (shard) split requires the index. Before processing the split for a
+`ZERO` replica, the index is pulled from the Zero store in a similar way
+to pulls done before processing an indexing batch. In `SplitOp`, a wrapper
+method is executed before calling the split code, makes sure the index
+is up to date from the Zero store, then grabs the `ZERO` indexing lock on
+the parent core and executes the standard split method.
+
+That split method got slightly modified to return the list of created
+subcores (and not close them) so that the wrapper method can push these
+new cores to the Zero store then close them.
+
+Indexing batches received (and pushed to the Zero store) while a split
+is in progress might be lost if they update the Zero store after its
+contents were fetched in `SplitOp` on the node where the split happens. In
+order to prevent this from happening, indexing is rejected while a split
+is in progress (remains to be seen if that can be changed once the
+Transaction Log is adapted to the `ZERO` replicas). The rejection is
+implemented by checking if there is a splitting “lock” in ZooKeeper (a
+path of the form `/collections/_collectionName_/_shardName_-splitting`).
+The method that performs the check is
+`SplitShardCmd.shardSplitLockHeld()`.
+
+The `...-splitting` path is created in `SplitShardCmd.lockForSplit` before
+the `SplitOp` is called on the node.
+
+Note there is a race window here (a small one) in which an indexing
+batch is not rejected because no split is in progress when the check is
+done, but assuming the indexing is slow to process, by the time it
+updates the Zero store image of the shard, the split has started and
+that batch is going to be lost (not present on any of the children). A
+possible fix would be to update the `version` of the `metadataSuffix` node
+before pulling the contents of the core for the split, and making sure
+rejection during indexing in case of a split happens after the indexing
+node has pulled the core (the increased `version` would make sure the
+indexing batch ends up failing). This would have to be reconsidered once
+`ZERO` replicas use the transaction log (see
+<<Future plans>>).
+
+== Random implementation notes
+
+Listing current limitations and/or work to do.
+
+=== CollectionsHandler
+
+`MODIFYCOLLECTION` does not work with `ZERO` collections
+
+=== CoreCorruptionTest
+
+`testIndexFilePullFailure()` and `testCorruptedIndexFile()` seem to fail
+sometimes.
+
+=== CorePusherTest
+
+The cluster is shutdown in `@After` but is built at the start of
+individual tests, and not all of them...
+
+=== CreateCollectionCmd
+
+In `populateShardNames()` there’s a change (with `TODO`) that likely needs
+to be contributed to the `main` branch independently of the Zero code
+implementation (it is a general fix).
+
+=== CreateCollectionRequestBody
+
+`TODO` regarding other places related to v2 API that might need changes
+
+=== CreateShardRequestBody
+
+`TODO` regarding other places related to v2 API that might need changes
+
+=== DocCollection
+
+Checks for collection vs. replica type might be redundant
+
+=== HttpSolrCall
+
+Hard coding of the request paths that trigger a pull. This should be
+changed and delegated to the specific handlers/components that require a
+pull.
+
+=== MigrateCmd
+
+`MIGRATE` does not work with `ZERO` replicas
+
+=== ReindexCollectionCmd
+
+`REINDEXCOLLECTION` does not work with `ZERO` replicas
+
+=== SearchHandler
+
+Rejecting query if core never pulled from Zero store. Need something
+more elegant
+
+=== SplitShardCmd
+
+Method `shardSplitLockHeld()` checks the presence of a lock using a
+`ZkController` whereas existing methods `lockForSplit()` and
+`unlockForSplit()` use a `SolrCloudManager`. Unfortunately the caller of
+`shardSplitLockHeld()` does not have an instance of `SolrCloudManager`.
+
+=== ZeroCoreConcurrencyTest
+
+Flapping (and disabled) test `todoTestIndexingQueriesDeletesFailovers()`.
+
+=== ZeroCoreIndexingBatchProcessor
+
+Indexing does happen on `INACTIVE` shards when split happens, this should
+not be the case. See `ZeroCoreIndexingBatchProcessor` and run tests in
+`ZeroStoreSplitTest`.
+
+== Future plans
+
+The `ZERO` code as of this writing (January 2024) has known limitations
+forcing commits with each indexing batch and basically bypassing
+anything that relies on the transaction log (nodes are stateless, the
+transaction log is assumed to not survive a restart). Moreover (but
+relatively minor drawback), each such indexing batch also includes a
+write to a ZooKeeper node (updating `metadataSuffix`).
+
+The next step is to adapt the transaction log from its current per
+replica state and make it a per shard abstraction that is stored in the
+Zero store in a way that makes any `ZERO` replica able to add to it and to
+consume it (like any `ZERO` replica can update itself from the Zero
+store). Once this is available, some aspects of the current
+implementation of `ZERO` replicas would have to be reconsidered.
+
+== Past plans
+
+First found trace of an intention to implement something similar to the
+proposal in this branch dates back to July 2014.
+https://issues.apache.org/jira/browse/SOLR-6237[[.underline]#SOLR-6237#]
+- _An option to have only leaders write and replicas read when using a
+shared file system with SolrCloud_.
+
+A more recent attempt to contribute the code that evolved into the
+current `ZERO` replicas proposal was made in January 2019. The new replica
+type was then called SHARED. The umbrella ticket is
+https://issues.apache.org/jira/browse/SOLR-13101[[.underline]#SOLR-13101#]
+- _Shared storage via a new SHARED replica type,_ and its (long
+abandoned) branch is
+https://github.com/apache/lucene-solr/tree/jira/SOLR-13101[[.underline]#lucene-solr
+jira/SOLR-13101#].
+
+The two tickets are marked as “Unresolved” and “Won't Fix”.
+
+Hopefully history doesn’t *always* repeat itself.
diff --git a/solr/api/src/java/org/apache/solr/client/api/model/CreateCollectionRequestBody.java b/solr/api/src/java/org/apache/solr/client/api/model/CreateCollectionRequestBody.java
index 5dce5840102..018cb8be0cb 100644
--- a/solr/api/src/java/org/apache/solr/client/api/model/CreateCollectionRequestBody.java
+++ b/solr/api/src/java/org/apache/solr/client/api/model/CreateCollectionRequestBody.java
@@ -38,6 +38,11 @@ public class CreateCollectionRequestBody {
 
   @JsonProperty public Integer nrtReplicas;
 
+  @JsonProperty public Integer zeroReplicas;
+  // TODO other "v2" places where ZERO needs to be added?
+
+  @JsonProperty public Boolean isZeroIndex;
+
   @JsonProperty public Boolean waitForFinalState;
 
   @JsonProperty public Boolean perReplicaState;
diff --git a/solr/api/src/java/org/apache/solr/client/api/model/CreateShardRequestBody.java b/solr/api/src/java/org/apache/solr/client/api/model/CreateShardRequestBody.java
index ea871998cca..081ba454c7b 100644
--- a/solr/api/src/java/org/apache/solr/client/api/model/CreateShardRequestBody.java
+++ b/solr/api/src/java/org/apache/solr/client/api/model/CreateShardRequestBody.java
@@ -34,6 +34,11 @@ public class CreateShardRequestBody {
 
   @JsonProperty public Integer pullReplicas;
 
+  @JsonProperty public Integer zeroReplicas;
+  // TODO other "v2" places where ZERO needs to be added?
+
+  @JsonProperty public Boolean isZeroIndex;
+
   @JsonProperty("createReplicas")
   public Boolean createReplicas;
 
diff --git a/solr/bin/solr b/solr/bin/solr
index 6b8270d66e0..b931de3c5de 100755
--- a/solr/bin/solr
+++ b/solr/bin/solr
@@ -1669,6 +1669,22 @@ if [ "${SOLR_MODE:-}" == 'solrcloud' ]; then
     CLOUD_MODE_OPTS+=("-DcreateZkChroot=$ZK_CREATE_CHROOT")
   fi
 
+  if [ -n "$ZERO_STORE_ENABLED" ]; then
+    CLOUD_MODE_OPTS+=("-DzeroStoreEnabled=$ZERO_STORE_ENABLED")
+  fi
+
+  if [ -n "$ZERO_STORE_NUM_FILE_PUSHER_THREADS" ]; then
+    CLOUD_MODE_OPTS+=("-DzeroStoreNumFilePusherThreads=$ZERO_STORE_NUM_FILE_PUSHER_THREADS")
+  fi
+
+  if [ -n "$ZERO_STORE_NUM_FILE_PULLER_THREADS" ]; then
+    CLOUD_MODE_OPTS+=("-DzeroStoreNumFilePullerThreads=$ZERO_STORE_NUM_FILE_PULLER_THREADS")
+  fi
+
+  if [ -n "$ZERO_STORE_MAX_ASYNC_CORE_PULLS" ]; then
+    CLOUD_MODE_OPTS+=("-DzeroStoreNumCorePullerThreads=$ZERO_STORE_MUM_CORE_PULLER_THREADS")
+  fi
+
   # and if collection1 needs to be bootstrapped
   if [ -e "$SOLR_HOME/collection1/core.properties" ]; then
     CLOUD_MODE_OPTS+=('-Dbootstrap_confdir=./solr/collection1/conf' '-Dcollection.configName=myconf' '-DnumShards=1')
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 e1768e0cd37..4ae57220de7 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ShardLeaderElectionContext.java
@@ -107,6 +107,7 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
       throws KeeperException, InterruptedException {
     String coreName = leaderProps.getStr(ZkStateReader.CORE_NAME_PROP);
     ActionThrottle lt;
+    boolean isZeroReplica;
     try (SolrCore core = cc.getCore(coreName)) {
       if (core == null) {
         // shutdown or removed
@@ -114,6 +115,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
       }
       MDCLoggingContext.setCore(core);
       lt = core.getUpdateHandler().getSolrCoreState().getLeaderThrottle();
+      isZeroReplica =
+          core.getCoreDescriptor().getCloudDescriptor().getReplicaType() == Replica.Type.ZERO;
     }
 
     try {
@@ -151,7 +154,9 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
       }
 
-      if (!weAreReplacement) {
+      // Do not wait for other replicas when type is ZERO.
+      // Since we can pull data from Zero store, we don't require an up-to-date leader
+      if (!isZeroReplica && !weAreReplacement) {
         waitForReplicasToComeUp(leaderVoteWait);
       }
 
@@ -173,98 +178,104 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
 
         replicaType = core.getCoreDescriptor().getCloudDescriptor().getReplicaType();
         coreNodeName = core.getCoreDescriptor().getCloudDescriptor().getCoreNodeName();
-        // should I be leader?
-        ZkShardTerms zkShardTerms = zkController.getShardTerms(collection, shardId);
-        if (zkShardTerms.registered(coreNodeName) && !zkShardTerms.canBecomeLeader(coreNodeName)) {
-          if (!waitForEligibleBecomeLeaderAfterTimeout(
-              zkShardTerms, coreNodeName, leaderVoteWait)) {
-            rejoinLeaderElection(core);
+
+        // if ZERO replica, skip sync and recovery stages. a ZERO replica that is not up-to-date
+        // can still become leader; it will sync the latest from Zero store with the next request.
+        if (replicaType != Replica.Type.ZERO) {
+          // should I be leader?
+          ZkShardTerms zkShardTerms = zkController.getShardTerms(collection, shardId);
+          if (zkShardTerms.registered(coreNodeName)
+              && !zkShardTerms.canBecomeLeader(coreNodeName)) {
+            if (!waitForEligibleBecomeLeaderAfterTimeout(
+                zkShardTerms, coreNodeName, leaderVoteWait)) {
+              rejoinLeaderElection(core);
+              return;
+            } else {
+              // only log an error if this replica win the election
+              setTermToMax = true;
+            }
+          }
+
+          if (isClosed) {
             return;
-          } else {
-            // only log an error if this replica win the election
-            setTermToMax = true;
           }
-        }
 
-        if (isClosed) {
-          return;
-        }
+          log.info("I may be the new leader - try and sync");
 
-        log.info("I may be the new leader - try and sync");
+          // we are going to attempt to be the leader
+          // first cancel any current recovery
+          core.getUpdateHandler().getSolrCoreState().cancelRecovery();
 
-        // we are going to attempt to be the leader
-        // first cancel any current recovery
-        core.getUpdateHandler().getSolrCoreState().cancelRecovery();
+          if (weAreReplacement) {
+            // wait a moment for any floating updates to finish
+            try {
+              Thread.sleep(2500);
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+              throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, e);
+            }
+          }
 
-        if (weAreReplacement) {
-          // wait a moment for any floating updates to finish
+          PeerSync.PeerSyncResult result = null;
+          boolean success = false;
           try {
-            Thread.sleep(2500);
-          } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE, e);
+            result = syncStrategy.sync(zkController, core, leaderProps, weAreReplacement);
+            success = result.isSuccess();
+          } catch (Exception e) {
+            log.error("Exception while trying to sync", e);
+            result = PeerSync.PeerSyncResult.failure();
           }
-        }
 
-        PeerSync.PeerSyncResult result = null;
-        boolean success = false;
-        try {
-          result = syncStrategy.sync(zkController, core, leaderProps, weAreReplacement);
-          success = result.isSuccess();
-        } catch (Exception e) {
-          log.error("Exception while trying to sync", e);
-          result = PeerSync.PeerSyncResult.failure();
-        }
+          UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
 
-        UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-
-        if (!success) {
-          boolean hasRecentUpdates = false;
-          if (ulog != null) {
-            // TODO: we could optimize this if necessary
-            try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
-              hasRecentUpdates = !recentUpdates.getVersions(1).isEmpty();
+          if (!success) {
+            boolean hasRecentUpdates = false;
+            if (ulog != null) {
+              // TODO: we could optimize this if necessary
+              try (UpdateLog.RecentUpdates recentUpdates = ulog.getRecentUpdates()) {
+                hasRecentUpdates = !recentUpdates.getVersions(1).isEmpty();
+              }
             }
-          }
 
-          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;
-            } 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;
+            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;
+              } 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;
+              }
             }
           }
-        }
 
-        // solrcloud_debug
-        if (log.isDebugEnabled()) {
-          try {
-            RefCounted<SolrIndexSearcher> searchHolder = core.getNewestSearcher(false);
-            SolrIndexSearcher searcher = searchHolder.get();
+          // solrcloud_debug
+          if (log.isDebugEnabled()) {
             try {
-              if (log.isDebugEnabled()) {
-                log.debug(
-                    "{} synched {}",
-                    core.getCoreContainer().getZkController().getNodeName(),
-                    searcher.count(new MatchAllDocsQuery()));
+              RefCounted<SolrIndexSearcher> searchHolder = core.getNewestSearcher(false);
+              SolrIndexSearcher searcher = searchHolder.get();
+              try {
+                if (log.isDebugEnabled()) {
+                  log.debug(
+                      "{} synched {}",
+                      core.getCoreContainer().getZkController().getNodeName(),
+                      searcher.count(new MatchAllDocsQuery()));
+                }
+              } finally {
+                searchHolder.decref();
               }
-            } finally {
-              searchHolder.decref();
+            } catch (Exception e) {
+              log.error("Error in solrcloud_debug block", e);
             }
-          } catch (Exception e) {
-            log.error("Error in solrcloud_debug block", e);
           }
-        }
-        if (!success) {
-          rejoinLeaderElection(core);
-          return;
+          if (!success) {
+            rejoinLeaderElection(core);
+            return;
+          }
         }
       }
 
@@ -463,6 +474,8 @@ final class ShardLeaderElectionContext extends ShardLeaderElectionContextBase {
         }
 
         // on startup and after connection timeout, wait for all known shards
+        // We also wait here for Replica.Type.ZERO but none will appear since we don't mix ZERO
+        // with other replica types, and we don't execute that method when current replica is ZERO
         if (found >= slices.getReplicas(leaderEligibleReplicaTypes).size()) {
           log.info("Enough replicas found to continue.");
           return true;
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 51aa6e024c8..a7690e0a6a1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1326,87 +1326,102 @@ public class ZkController implements Closeable {
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "", e);
       }
 
-      // in this case, we want to wait for the leader as long as the leader might
-      // wait for a vote, at least - but also long enough that a large cluster has
-      // time to get its act together
-      String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
-
-      String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
-      log.debug("We are {} and leader is {}", ourUrl, leaderUrl);
-      boolean isLeader = leaderUrl.equals(ourUrl);
-      assert !isLeader || replica.getType().leaderEligible
-          : replica.getType().name() + " replica became leader!";
-
-      try (SolrCore core = cc.getCore(desc.getName())) {
-
-        // recover from local transaction log and wait for it to complete before
-        // going active
-        // TODO: should this be moved to another thread? To recoveryStrat?
-        // TODO: should this actually be done earlier, before (or as part of)
-        // leader election perhaps?
-
-        if (core == null) {
-          throw new SolrException(
-              ErrorCode.SERVICE_UNAVAILABLE, "SolrCore is no longer available to register");
-        }
+      if (replica.getType() == Type.ZERO) {
+        // ZERO replicas: leadership is only needed during indexing.
+        // Replicas need to register for potential leadership on startup but that's it.
+        // As any ZERO replica can become leader at any time, no need to wait in an election
+        // ZERO replicas do not use the update log.
+        publish(desc, Replica.State.ACTIVE);
+      } else {
+        // Non ZERO replica types (NRT, TLOG, PULL): leadership is important.  On startup,
+        // a replica needs to be designated as the leader or to synchronize with the leader.
+        // Until either happen, a replica can't be considered ACTIVE.
+
+        // in this case, we want to wait for the leader as long as the leader might
+        // wait for a vote, at least - but also long enough that a large cluster has
+        // time to get its act together
+        String leaderUrl = getLeader(cloudDesc, leaderVoteWait + 600000);
+
+        String ourUrl = ZkCoreNodeProps.getCoreUrl(baseUrl, coreName);
+        log.debug("We are {} and leader is {}", ourUrl, leaderUrl);
+        boolean isLeader = leaderUrl.equals(ourUrl);
+        assert !isLeader || replica.getType().leaderEligible
+            : replica.getType().name() + " replica became leader!";
+
+        try (SolrCore core = cc.getCore(desc.getName())) {
+
+          // recover from local transaction log and wait for it to complete before
+          // going active
+          // TODO: should this be moved to another thread? To recoveryStrat?
+          // TODO: should this actually be done earlier, before (or as part of)
+          // leader election perhaps?
+
+          if (core == null) {
+            throw new SolrException(
+                ErrorCode.SERVICE_UNAVAILABLE, "SolrCore is no longer available to register");
+          }
 
-        UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
-        boolean isTlogReplicaAndNotLeader = replica.getType() == Replica.Type.TLOG && !isLeader;
-        if (isTlogReplicaAndNotLeader) {
-          String commitVersion = ReplicateFromLeader.getCommitVersion(core);
-          if (commitVersion != null) {
-            ulog.copyOverOldUpdates(Long.parseLong(commitVersion));
+          UpdateLog ulog = core.getUpdateHandler().getUpdateLog();
+          boolean isTlogReplicaAndNotLeader = replica.getType() == Replica.Type.TLOG && !isLeader;
+          if (isTlogReplicaAndNotLeader) {
+            String commitVersion = ReplicateFromLeader.getCommitVersion(core);
+            if (commitVersion != null) {
+              ulog.copyOverOldUpdates(Long.parseLong(commitVersion));
+            }
           }
-        }
-        // we will call register again after zk expiration and on reload
-        if (!afterExpiration && !core.isReloaded() && ulog != null && !isTlogReplicaAndNotLeader) {
-          // disable recovery in case shard is in construction state (for shard splits)
-          Slice slice = getClusterState().getCollection(collection).getSlice(shardId);
-          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);
-              recoveryFuture.get(); // NOTE: this could potentially block for
-              // minutes or more!
-              // TODO: public as recovering in the mean time?
-              // TODO: in the future we could do peersync in parallel with recoverFromLog
-            } else {
-              if (log.isDebugEnabled()) {
-                log.debug("No LogReplay needed for core={} baseURL={}", core.getName(), baseUrl);
+          // we will call register again after zk expiration and on reload
+          if (!afterExpiration
+              && !core.isReloaded()
+              && ulog != null
+              && !isTlogReplicaAndNotLeader) {
+            // disable recovery in case shard is in construction state (for shard splits)
+            Slice slice = getClusterState().getCollection(collection).getSlice(shardId);
+            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);
+                recoveryFuture.get(); // NOTE: this could potentially block for
+                // minutes or more!
+                // TODO: public as recovering in the mean time?
+                // TODO: in the future we could do peersync in parallel with recoverFromLog
+              } else {
+                if (log.isDebugEnabled()) {
+                  log.debug("No LogReplay needed for core={} baseURL={}", core.getName(), baseUrl);
+                }
               }
             }
           }
-        }
-        boolean didRecovery =
-            checkRecovery(
-                recoverReloadedCores,
-                isLeader,
-                skipRecovery,
-                collection,
-                coreZkNodeName,
-                shardId,
-                core,
-                cc,
-                afterExpiration);
-        if (!didRecovery) {
-          if (isTlogReplicaAndNotLeader) {
-            startReplicationFromLeader(coreName, true);
+          boolean didRecovery =
+              checkRecovery(
+                  recoverReloadedCores,
+                  isLeader,
+                  skipRecovery,
+                  collection,
+                  coreZkNodeName,
+                  shardId,
+                  core,
+                  cc,
+                  afterExpiration);
+          if (!didRecovery) {
+            if (isTlogReplicaAndNotLeader) {
+              startReplicationFromLeader(coreName, true);
+            }
+            publish(desc, Replica.State.ACTIVE);
           }
-          publish(desc, Replica.State.ACTIVE);
-        }
 
-        if (replica.getType().leaderEligible) {
-          // the watcher is added to a set so multiple calls of this method will left only one
-          // watcher
-          shardTerms.addListener(
-              new RecoveringCoreTermWatcher(core.getCoreDescriptor(), getCoreContainer()));
+          if (replica.getType().leaderEligible) {
+            // the watcher is added to a set so multiple calls of this method will left only one
+            // watcher
+            shardTerms.addListener(
+                new RecoveringCoreTermWatcher(core.getCoreDescriptor(), getCoreContainer()));
+          }
+          core.getCoreDescriptor().getCloudDescriptor().setHasRegistered(true);
+        } catch (Exception e) {
+          unregister(coreName, desc, false);
+          throw e;
         }
-        core.getCoreDescriptor().getCloudDescriptor().setHasRegistered(true);
-      } catch (Exception e) {
-        unregister(coreName, desc, false);
-        throw e;
       }
 
       // make sure we have an update cluster state right away
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 b7ed4954fbf..bb61b706344 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
@@ -131,7 +131,9 @@ public class AddReplicaCmd implements CollApiCmds.CollectionApiCommand {
     int timeout = message.getInt(TIMEOUT, 10 * 60); // 10 minutes
     boolean parallel = message.getBool("parallel", false);
 
-    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, null, 1);
+    // Default replica type is NRT in SolrCloud, but if collection is Zero store based,
+    // default (and only acceptable type) is Replica.Type.ZERO
+    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, null, 1, coll.isZeroIndex());
 
     int totalReplicas = numReplicas.total();
     if (totalReplicas > 1) {
@@ -236,6 +238,18 @@ public class AddReplicaCmd implements CollApiCmds.CollectionApiCommand {
       boolean skipCreateReplicaInClusterState,
       CreateReplica createReplica)
       throws InterruptedException, KeeperException {
+    // Fail fast if adding wrong replicas to a collection
+    if (!coll.isZeroIndex() && createReplica.replicaType == Replica.Type.ZERO) {
+      throw new RuntimeException(
+          "Can't add a Replica.Type.ZERO to a collection not backed by Zero store");
+    }
+    if (coll.isZeroIndex() && createReplica.replicaType != Replica.Type.ZERO) {
+      throw new RuntimeException(
+          "Can't add a "
+              + createReplica.replicaType
+              + " replica to a collection backed by Zero store");
+    }
+
     if (!skipCreateReplicaInClusterState) {
       Map<String, Object> replicaProps =
           Utils.makeMap(
@@ -397,7 +411,22 @@ public class AddReplicaCmd implements CollApiCmds.CollectionApiCommand {
     boolean skipNodeAssignment = message.getBool(CollectionAdminParams.SKIP_NODE_ASSIGNMENT, false);
     String sliceName = message.getStr(SHARD_ID_PROP);
     DocCollection collection = clusterState.getCollection(collectionName);
+
     int totalReplicas = numReplicas.total();
+    int numZeroReplicas = numReplicas.get(Replica.Type.ZERO);
+    if (numZeroReplicas != 0 && numZeroReplicas != totalReplicas) {
+      throw new RuntimeException(
+          "ZERO and non ZERO replica types specified for collection "
+              + collectionName
+              + ". ZERO: "
+              + numZeroReplicas
+              + " NRT: "
+              + numReplicas.get(Replica.Type.NRT)
+              + " PULL: "
+              + numReplicas.get(Replica.Type.PULL)
+              + " TLOG: "
+              + numReplicas.get(Replica.Type.TLOG));
+    }
 
     String node = message.getStr(CoreAdminParams.NODE);
     Object createNodeSetStr = message.get(CollectionHandlingUtils.CREATE_NODE_SET);
@@ -417,6 +446,7 @@ public class AddReplicaCmd implements CollApiCmds.CollectionApiCommand {
               collection.getName(),
               sliceName,
               numReplicas,
+              collection.isZeroIndex(),
               createNodeSetStr,
               cloudManager,
               coreContainer);
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 1d7d3a4087d..b6d7fbd5f07 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
@@ -324,6 +324,7 @@ public class Assign {
       String collectionName,
       String shard,
       ReplicaCount numReplicas,
+      boolean isZeroIndex,
       Object createNodeSet,
       SolrCloudManager cloudManager,
       CoreContainer coreContainer)
@@ -358,6 +359,7 @@ public class Assign {
             .forCollection(collectionName)
             .forShard(Collections.singletonList(shard))
             .assignReplicas(numReplicas)
+            .setZeroIndex(isZeroIndex)
             .onNodes(createNodeList)
             .build();
     AssignStrategy assignStrategy = createAssignStrategy(coreContainer);
@@ -496,16 +498,38 @@ public class Assign {
     public final List<String> shardNames;
     public final List<String> nodes;
     public final ReplicaCount numReplicas;
+    public final boolean zeroIndex;
 
     public AssignRequest(
         String collectionName,
         List<String> shardNames,
         List<String> nodes,
-        ReplicaCount numReplicas) {
+        ReplicaCount numReplicas,
+        boolean zeroIndex) {
       this.collectionName = collectionName;
       this.shardNames = shardNames;
       this.nodes = nodes;
       this.numReplicas = numReplicas;
+      this.zeroIndex = zeroIndex;
+
+      // ZERO replica related verifications
+      int totalReplicas = numReplicas.total();
+      int numZeroReplicas = numReplicas.get(Replica.Type.ZERO);
+      if (numZeroReplicas != 0 && numZeroReplicas != totalReplicas) {
+        throw new RuntimeException(
+            "Illegal combination of ZERO ("
+                + numZeroReplicas
+                + ") and non ZERO ("
+                + (totalReplicas - numZeroReplicas)
+                + ") replicas for collection "
+                + collectionName);
+      }
+      if (!zeroIndex && numZeroReplicas != 0) {
+        throw new RuntimeException(
+            "Can't specify ZERO replicas for collection "
+                + collectionName
+                + " non backed by Zero store");
+      }
     }
   }
 
@@ -514,6 +538,7 @@ public class Assign {
     private List<String> shardNames;
     private List<String> nodes;
     private ReplicaCount numReplicas;
+    private boolean zeroIndex;
 
     public AssignRequestBuilder() {
       this.numReplicas = ReplicaCount.empty();
@@ -539,10 +564,15 @@ public class Assign {
       return this;
     }
 
+    public AssignRequestBuilder setZeroIndex(boolean zeroIndex) {
+      this.zeroIndex = zeroIndex;
+      return this;
+    }
+
     public AssignRequest build() {
       Objects.requireNonNull(collectionName, "The collectionName cannot be null");
       Objects.requireNonNull(shardNames, "The shard names cannot be null");
-      return new AssignRequest(collectionName, shardNames, nodes, numReplicas);
+      return new AssignRequest(collectionName, shardNames, nodes, numReplicas, zeroIndex);
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
index d73f6f4e395..cc9511d7554 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CollectionHandlingUtils.java
@@ -55,6 +55,7 @@ import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.DocCollection.CollectionStateProps;
 import org.apache.solr.common.cloud.DocRouter;
 import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Replica.Type;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
@@ -109,9 +110,16 @@ public class CollectionHandlingUtils {
 
   // Immutable Maps are null-hostile, so build our own
   public static final Map<String, Object> COLLECTION_PROPS_AND_DEFAULTS =
-      Collections.unmodifiableMap(makeCollectionPropsAndDefaults());
+      Collections.unmodifiableMap(makeCollectionPropsAndDefaults(false));
 
-  private static Map<String, Object> makeCollectionPropsAndDefaults() {
+  /**
+   * Default values for a collection based on Zero store ({@link DocCollection#isZeroIndex()}
+   * returns {@code true}, replicas are {@link Type#ZERO}).
+   */
+  public static final Map<String, Object> ZERO_INDEX_COLLECTION_PROPS_AND_DEFAULTS =
+      Collections.unmodifiableMap(makeCollectionPropsAndDefaults(true));
+
+  private static Map<String, Object> makeCollectionPropsAndDefaults(boolean zeroIndex) {
     Map<String, Object> propsAndDefaults =
         Utils.makeMap(
             CollectionStateProps.DOC_ROUTER,
@@ -119,11 +127,13 @@ public class CollectionHandlingUtils {
             CollectionStateProps.REPLICATION_FACTOR,
             "1",
             CollectionStateProps.PER_REPLICA_STATE,
-            null);
+            null,
+            ZkStateReader.ZERO_INDEX,
+            Boolean.toString(zeroIndex));
     for (Replica.Type replicaType : Replica.Type.values()) {
       propsAndDefaults.put(
           replicaType.numReplicasPropertyName,
-          replicaType == Replica.Type.defaultType() ? "1" : "0");
+          replicaType == Replica.Type.defaultType(zeroIndex) ? "1" : "0");
     }
     return propsAndDefaults;
   }
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 020146cfdc7..344117c3a67 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
@@ -17,6 +17,7 @@
 
 package org.apache.solr.cloud.api.collections;
 
+import static org.apache.solr.common.cloud.ZkStateReader.ZERO_INDEX;
 import static org.apache.solr.common.params.CollectionAdminParams.ALIAS;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.ADDREPLICA;
@@ -39,6 +40,7 @@ import java.util.NoSuchElementException;
 import java.util.Properties;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
 import org.apache.solr.client.solrj.cloud.BadVersionException;
 import org.apache.solr.client.solrj.cloud.DelegatingCloudManager;
@@ -104,11 +106,20 @@ public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
     if (ccc.getZkStateReader().aliasesManager != null) { // not a mock ZkStateReader
       ccc.getZkStateReader().aliasesManager.update();
     }
+    final boolean isZeroIndex = message.getBool(ZERO_INDEX, false);
     final Aliases aliases = ccc.getZkStateReader().getAliases();
     final String collectionName = message.getStr(NAME);
     final boolean waitForFinalState = message.getBool(WAIT_FOR_FINAL_STATE, false);
     final String alias = message.getStr(ALIAS, collectionName);
     log.info("Create collection {}", collectionName);
+
+    if (isZeroIndex && !ccc.getCoreContainer().isZeroStoreEnabled()) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "Collection "
+              + collectionName
+              + " cannot be created because Zero store is not enabled on this cluster.");
+    }
     final boolean isPRS = message.getBool(CollectionStateProps.PER_REPLICA_STATE, false);
     if (clusterState.hasCollection(collectionName)) {
       throw new SolrException(
@@ -134,7 +145,7 @@ public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
 
     // fail fast if parameters are wrong or incomplete
     List<String> shardNames = populateShardNames(message, router);
-    ReplicaCount numReplicas = getNumReplicas(message);
+    ReplicaCount numReplicas = getNumReplicas(message, isZeroIndex);
 
     DocCollection newColl = null;
     final String collectionPath = DocCollection.getCollectionPath(collectionName);
@@ -223,6 +234,15 @@ public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
               "Could not fully create collection: " + collectionName);
         }
 
+        if (isZeroIndex) {
+          for (String shardName : shardNames) {
+            ccc.getCoreContainer()
+                .getZeroStoreManager()
+                .getZeroMetadataController()
+                .createMetadataNode(collectionName, shardName);
+          }
+        }
+
         // refresh cluster state (value read below comes from Zookeeper watch firing following the
         // update done previously, be it by Overseer or by this thread when updates are distributed)
         clusterState = ccc.getSolrCloudManager().getClusterState();
@@ -521,10 +541,16 @@ public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
             "It's unusual to run two replica of the same slice on the same Solr-instance.");
       }
 
+      // true if the collection is backed by Zero store.
+      // In which case only org.apache.solr.common.cloud.Replica.Type#Zero
+      // replicas matter and no other replicas should be defined.
+      final boolean zeroIndex = message.getBool(ZERO_INDEX, false);
+
       Assign.AssignRequest assignRequest =
           new Assign.AssignRequestBuilder()
               .forCollection(collectionName)
               .forShard(shardNames)
+              .setZeroIndex(zeroIndex)
               .assignReplicas(numReplicas)
               .onNodes(nodeList)
               .build();
@@ -580,18 +606,34 @@ public class CreateCollectionCmd implements CollApiCmds.CollectionApiCommand {
         throw new SolrException(
             ErrorCode.BAD_REQUEST, CollectionHandlingUtils.NUM_SLICES + " must be > 0");
       }
+
+      // TODO unclear if the following bloc of code should be here (belongs directly in main?)
+      // When specified, retrieve shard names from the request
+      // If the list length does not match the parameter for number of shards, fallback to the
+      // default names
+      // This is mostly used when creating a collection for a restore
+      Object shards = message.get(CollectionHandlingUtils.SHARDS_PROP);
+      if (shards instanceof Map) {
+        List<String> providedNames =
+            ((Map<?, ?>) shards)
+                .keySet().stream().map(Object::toString).collect(Collectors.toList());
+        if (providedNames.size() == numSlices) {
+          return providedNames;
+        }
+      }
+
       ClusterStateMutator.getShardNames(numSlices, shardNames);
     }
     return shardNames;
   }
 
-  private ReplicaCount getNumReplicas(ZkNodeProps message) {
-    ReplicaCount numReplicas = ReplicaCount.fromMessage(message);
+  private ReplicaCount getNumReplicas(ZkNodeProps message, boolean isZeroIndex) {
+    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, isZeroIndex);
     boolean hasLeaderEligibleReplica = numReplicas.hasLeaderReplica();
-    if (!hasLeaderEligibleReplica && !numReplicas.contains(Replica.Type.defaultType())) {
+    if (!hasLeaderEligibleReplica && !numReplicas.contains(Replica.Type.defaultType(isZeroIndex))) {
       // Ensure that there is at least one replica that can become leader if the user did
       // not force a replica count.
-      numReplicas.put(Replica.Type.defaultType(), 1);
+      numReplicas.put(Replica.Type.defaultType(isZeroIndex), 1);
     } else if (!hasLeaderEligibleReplica) {
       // This can still fail if the user manually forced "0" replica counts.
       throw new SolrException(
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 b76250cfd8d..9c59fd923f8 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
@@ -66,7 +66,8 @@ public class CreateShardCmd implements CollApiCmds.CollectionApiCommand {
     }
     DocCollection collection = clusterState.getCollection(collectionName);
 
-    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, collection, 1);
+    boolean isZeroIndex = collection.isZeroIndex();
+    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, collection, 1, isZeroIndex);
     if (!numReplicas.hasLeaderReplica()) {
       throw new SolrException(
           SolrException.ErrorCode.BAD_REQUEST,
@@ -75,6 +76,14 @@ public class CreateShardCmd implements CollApiCmds.CollectionApiCommand {
               + "), there must be at least one leader-eligible replica");
     }
 
+    if (isZeroIndex) {
+      // create the shard metadataSuffix znode
+      ccc.getCoreContainer()
+          .getZeroStoreManager()
+          .getZeroMetadataController()
+          .createMetadataNode(extCollectionName, sliceName);
+    }
+
     if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
       // The message has been crafted by CollectionsHandler.CollectionOperation.CREATESHARD_OP and
       // defines the QUEUE_OPERATION to be CollectionParams.CollectionAction.CREATESHARD. Likely a
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 a1e9012aaf0..7c8081345d7 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
@@ -47,6 +47,7 @@ import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.snapshots.SolrSnapshotManager;
 import org.apache.solr.handler.admin.ConfigSetsHandler;
+import org.apache.solr.zero.process.ZeroStoreManager;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -139,6 +140,14 @@ public class DeleteCollectionCmd implements CollApiCmds.CollectionApiCommand {
         }
       }
 
+      // Delete the collection files from Zero store. We want to delete all the files before we
+      // delete the collection state from ZooKeeper.
+      DocCollection docCollection = zkStateReader.getClusterState().getCollectionOrNull(collection);
+      if (docCollection != null && docCollection.isZeroIndex()) {
+        ZeroStoreManager zeroStoreManager = ccc.getCoreContainer().getZeroStoreManager();
+        zeroStoreManager.deleteCollection(collection);
+      }
+
       ZkNodeProps m = new ZkNodeProps(Overseer.QUEUE_OPERATION, DELETE.toLower(), NAME, collection);
       if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
         ccc.getDistributedClusterStateUpdater()
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 72c93c02c62..cc942801607 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
@@ -43,6 +43,7 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.zero.process.ZeroStoreManager;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -182,6 +183,11 @@ public class DeleteShardCmd implements CollApiCmds.CollectionApiCommand {
       log.debug("Waiting for delete shard action to complete");
       cleanupLatch.await(1, TimeUnit.MINUTES);
 
+      if (clusterState.getCollection(collectionName).isZeroIndex()) {
+        ZeroStoreManager zeroStoreManager = ccc.getCoreContainer().getZeroStoreManager();
+        zeroStoreManager.deleteShard(collectionName, slice.getName());
+      }
+
       ZkNodeProps m =
           new ZkNodeProps(
               Overseer.QUEUE_OPERATION,
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 cdcb3c27449..3ffc061cbfd 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
@@ -56,6 +56,11 @@ import org.apache.solr.util.TimeOut;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+/**
+ * TODO Migration is NOT working with Zero store based collections (having replicas of type {@link
+ * org.apache.solr.common.cloud.Replica.Type#ZERO}). <br>
+ * If/once we make it work, delete this comment.
+ */
 public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private final CollectionCommandContext ccc;
@@ -307,7 +312,7 @@ public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
             CREATE.toLower(),
             NAME,
             tempSourceCollectionName,
-            Replica.Type.defaultType().numReplicasPropertyName,
+            Replica.Type.defaultType(false).numReplicasPropertyName, // TODO does not work for ZERO
             1,
             CollectionHandlingUtils.NUM_SLICES,
             1,
@@ -393,7 +398,7 @@ public class MigrateCmd implements CollApiCmds.CollectionApiCommand {
             ccc.getSolrCloudManager().getDistribStateManager(),
             zkStateReader.getClusterState().getCollection(tempSourceCollectionName),
             tempSourceSlice.getName(),
-            Replica.Type.defaultType());
+            Replica.Type.defaultType(false)); // TODO does not work for ZERO
     props = new HashMap<>();
     props.put(Overseer.QUEUE_OPERATION, ADDREPLICA.toLower());
     props.put(COLLECTION_PROP, tempSourceCollectionName);
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 91299b3c259..a65a5f8c73d 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
@@ -66,6 +66,10 @@ import org.slf4j.LoggerFactory;
 /**
  * Reindex a collection, usually in order to change the index schema.
  *
+ * <p><b>TODO Reindex is NOT working with Zero store based collections (having replicas of type
+ * {@link org.apache.solr.common.cloud.Replica.Type#ZERO}). <br>
+ * If/once we make it work, delete this comment.</b>
+ *
  * <p>WARNING: Reindexing is potentially a lossy operation - some indexed data that is not available
  * as stored fields may be irretrievably lost, so users should use this command with caution,
  * evaluating the potential impact by using different source and target collection names first, and
@@ -249,7 +253,8 @@ public class ReindexCollectionCmd implements CollApiCmds.CollectionApiCommand {
     String query = message.getStr(CommonParams.Q, "*:*");
     String fl = message.getStr(CommonParams.FL, "*");
     Integer rf = message.getInt(ZkStateReader.REPLICATION_FACTOR, coll.getReplicationFactor());
-    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, coll);
+    // TODO REINDEXCOLLECTION currently doesn't work with ZERO collections
+    ReplicaCount numReplicas = ReplicaCount.fromMessage(message, coll, false);
     int numShards = message.getInt(ZkStateReader.NUM_SHARDS_PROP, coll.getActiveSlices().size());
     DocRouter router = coll.getRouter();
     if (router == null) {
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 b774663b1e2..8b6a7f079ed 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
@@ -21,6 +21,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.ZERO_INDEX;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
@@ -214,11 +215,24 @@ public class RestoreCmd implements CollApiCmds.CollectionApiCommand {
    */
   private class RestoreOnANewCollection {
     private final ReplicaCount numReplicas;
+    private final boolean zeroIndex;
     private final ZkNodeProps message;
 
     private RestoreOnANewCollection(ZkNodeProps message, DocCollection backupCollectionState) {
       this.message = message;
-      this.numReplicas = ReplicaCount.fromMessage(message, backupCollectionState);
+
+      this.zeroIndex = message.getBool(CollectionAdminParams.ZERO_INDEX, false);
+      if (zeroIndex == backupCollectionState.isZeroIndex()) {
+        // Use the replica counts specified in the message and for those not present, in the backup
+        this.numReplicas = ReplicaCount.fromMessage(message, backupCollectionState, zeroIndex);
+      } else {
+        // Restoring a backed up Zero collection into a non Zero collection or the other way around.
+        // Can't use the replica counts in the backed up collection, they are not compatible with
+        // the collection being created.
+        // Use whatever the message asks for and if the message says nothing, ask for 2 replicas
+        // for the default type.
+        this.numReplicas = ReplicaCount.fromMessage(message, null, 2, zeroIndex);
+      }
     }
 
     public void process(NamedList<Object> results, RestoreContext rc) throws Exception {
@@ -318,11 +332,25 @@ public class RestoreCmd implements CollApiCmds.CollectionApiCommand {
       propMap.put(Overseer.QUEUE_OPERATION, CREATE.toString());
       // mostly true. Prevents autoCreated=true in the collection state.
       propMap.put("fromApi", "true");
-      propMap.put(REPLICATION_FACTOR, numReplicas.get(Replica.Type.defaultType()));
-      numReplicas.writeProps(propMap);
+      propMap.put(REPLICATION_FACTOR, numReplicas.get(Replica.Type.defaultType(zeroIndex)));
+
+      if (zeroIndex == backupCollectionState.isZeroIndex()) {
+        // When restoring into same type of collection, inherit from backed up collection replica
+        // counts not specified in the message (forces specifying zero counts for undesired replica)
+        numReplicas.writeProps(propMap);
+      } else {
+        // When restoring a ZERO collection into a non ZERO collection and vice versa, can't use
+        // incompatible defaults coming from the backed up collection as done in the loop below.
+        propMap.put(ZERO_INDEX, zeroIndex);
+        numReplicas.writePropsOrZeros(propMap);
+      }
 
       // inherit settings from input API, defaulting to the backup's setting.  Ex: replicationFactor
-      for (String collProp : CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.keySet()) {
+      for (String collProp :
+          (zeroIndex
+                  ? CollectionHandlingUtils.ZERO_INDEX_COLLECTION_PROPS_AND_DEFAULTS
+                  : CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS)
+              .keySet()) {
         Object val =
             message.getProperties().getOrDefault(collProp, backupCollectionState.get(collProp));
         if (val != null && propMap.get(collProp) == null) {
@@ -401,6 +429,7 @@ public class RestoreCmd implements CollApiCmds.CollectionApiCommand {
               .forCollection(restoreCollection)
               .forShard(sliceNames)
               .assignReplicas(numReplicas)
+              .setZeroIndex(zeroIndex)
               .onNodes(nodeList)
               .build();
       Assign.AssignStrategy assignStrategy = Assign.createAssignStrategy(ccc.getCoreContainer());
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 92c93e18b0e..7e11f405bec 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
@@ -50,6 +50,7 @@ import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.cloud.DistributedClusterStateUpdater;
 import org.apache.solr.cloud.Overseer;
+import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.api.collections.CollectionHandlingUtils.ShardRequestTracker;
 import org.apache.solr.cloud.overseer.OverseerAction;
 import org.apache.solr.common.LinkedHashMapWriter;
@@ -274,6 +275,7 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
         params.set(CoreAdminParams.GET_RANGES, "true");
         params.set(CommonAdminParams.SPLIT_METHOD, splitMethod.toLower());
         params.set(CoreAdminParams.CORE, parentShardLeader.getStr("core"));
+        params.set(CoreAdminParams.REPLICA_TYPE, leaderReplicaType.toString());
         // only 2 sub-shards are currently supported
         // int numSubShards = message.getInt(NUM_SUB_SHARDS, DEFAULT_NUM_SUB_SHARDS);
         // params.set(NUM_SUB_SHARDS, Integer.toString(numSubShards));
@@ -370,7 +372,6 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
         String subSlice = subSlices.get(i);
         String subShardName = subShardNames.get(i);
         DocRouter.Range subRange = subRanges.get(i);
-
         log.debug("Creating slice {} of collection {} on {}", subSlice, collectionName, nodeName);
 
         LinkedHashMapWriter<Object> propMap = new LinkedHashMapWriter<>();
@@ -383,6 +384,13 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
         propMap.put("shard_parent_node", nodeName);
         propMap.put("shard_parent_zk_session", leaderZnodeStat.getEphemeralOwner());
 
+        if (collection.isZeroIndex()) {
+          ccc.getCoreContainer()
+              .getZeroStoreManager()
+              .getZeroMetadataController()
+              .createMetadataNode(collectionName, subSlice);
+        }
+
         if (ccc.getDistributedClusterStateUpdater().isDistributedStateUpdate()) {
           ccc.getDistributedClusterStateUpdater()
               .doSingleStateUpdate(
@@ -488,6 +496,7 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
       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.REPLICA_TYPE, leaderReplicaType.toString());
       for (int i = 0; i < subShardNames.size(); i++) {
         String subShardName = subShardNames.get(i);
         params.add(CoreAdminParams.TARGET_CORE, subShardName);
@@ -551,6 +560,7 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
               .forCollection(collectionName)
               .forShard(subSlices)
               .assignReplicas(numReplicas)
+              .setZeroIndex(collection.isZeroIndex())
               .onNodes(
                   Assign.getLiveOrLiveAndCreateNodeSetList(
                       clusterState.getLiveNodes(),
@@ -1242,4 +1252,12 @@ public class SplitShardCmd implements CollApiCmds.CollectionApiCommand {
       }
     }
   }
+
+  public static boolean shardSplitLockHeld(
+      ZkController zkController, String collection, String shard) throws Exception {
+    // Would have been better to use a SolrCloudManager instance for the check rather than a
+    // ZkController but caller doesn't have such a reference
+    String path = ZkStateReader.COLLECTIONS_ZKNODE + "/" + collection + "/" + shard + "-splitting";
+    return zkController.pathExists(path);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
index 11b3795745b..e7081f850a5 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/ClusterStateMutator.java
@@ -71,6 +71,8 @@ public class ClusterStateMutator {
 
     Object messageShardsObj = message.get("shards");
 
+    boolean isZeroIndex = message.getBool(ZkStateReader.ZERO_INDEX, false);
+
     Map<String, Slice> slices;
     // we are being explicitly told the slice data (e.g. coll restore)
     if (messageShardsObj instanceof Map) {
@@ -104,11 +106,15 @@ public class ClusterStateMutator {
 
     Map<String, Object> collectionProps = new HashMap<>();
 
-    for (Map.Entry<String, Object> e :
-        CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.entrySet()) {
+    Map<String, Object> collectionDefaults =
+        isZeroIndex
+            ? CollectionHandlingUtils.ZERO_INDEX_COLLECTION_PROPS_AND_DEFAULTS
+            : CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS;
+
+    for (Map.Entry<String, Object> e : collectionDefaults.entrySet()) {
       Object val = message.get(e.getKey());
       if (val == null) {
-        val = CollectionHandlingUtils.COLLECTION_PROPS_AND_DEFAULTS.get(e.getKey());
+        val = collectionDefaults.get(e.getKey());
       }
       if (val != null) collectionProps.put(e.getKey(), val);
     }
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index 088c4f8fed7..397cb288db8 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -62,6 +62,7 @@ public class CollectionMutator {
     String shardId = message.getStr(ZkStateReader.SHARD_ID_PROP);
     DocCollection collection = clusterState.getCollection(collectionName);
     Slice slice = collection.getSlice(shardId);
+
     if (slice == null) {
       Map<String, Replica> replicas = Collections.emptyMap();
       Map<String, Object> sliceProps = new HashMap<>();
@@ -149,7 +150,8 @@ public class CollectionMutator {
         // SOLR-11676 : keep NRT_REPLICAS and REPLICATION_FACTOR in sync
         if (prop.equals(REPLICATION_FACTOR)) {
           props.put(
-              Replica.Type.defaultType().numReplicasPropertyName, message.get(REPLICATION_FACTOR));
+              Replica.Type.defaultType(coll.isZeroIndex()).numReplicasPropertyName,
+              message.get(REPLICATION_FACTOR));
         }
       }
     }
diff --git a/solr/core/src/java/org/apache/solr/cluster/Replica.java b/solr/core/src/java/org/apache/solr/cluster/Replica.java
index 2d6978c63f1..14a4b4a1d2c 100644
--- a/solr/core/src/java/org/apache/solr/cluster/Replica.java
+++ b/solr/core/src/java/org/apache/solr/cluster/Replica.java
@@ -36,11 +36,15 @@ public interface Replica {
   /** {@link Node} on which this {@link Replica} is located. */
   Node getNode();
 
-  /** The order of this enum is important from the most to least "important" replica type. */
+  /**
+   * The order of this enum is important from the most to least "important" replica type. {@link
+   * ReplicaType#ZERO} is different though, its position in the enum does not matter.
+   */
   enum ReplicaType {
     NRT('n'),
     TLOG('t'),
-    PULL('p');
+    PULL('p'),
+    ZERO('z');
 
     private final char suffixChar;
 
diff --git a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
index cb86dc304e5..cfae7cdb0fd 100644
--- a/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
+++ b/solr/core/src/java/org/apache/solr/cluster/placement/impl/SimpleClusterAbstractionsImpl.java
@@ -392,6 +392,8 @@ class SimpleClusterAbstractionsImpl {
           return Replica.ReplicaType.TLOG;
         case PULL:
           return Replica.ReplicaType.PULL;
+        case ZERO:
+          return Replica.ReplicaType.ZERO;
         default:
           throw new RuntimeException("Unexpected " + type);
       }
@@ -456,6 +458,8 @@ class SimpleClusterAbstractionsImpl {
           return org.apache.solr.common.cloud.Replica.Type.TLOG;
         case PULL:
           return org.apache.solr.common.cloud.Replica.Type.PULL;
+        case ZERO:
+          return org.apache.solr.common.cloud.Replica.Type.ZERO;
         default:
           throw new IllegalArgumentException("Unknown " + type);
       }
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 1da0802cf08..3455721c061 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -37,6 +37,7 @@ import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.text.SimpleDateFormat;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Date;
@@ -152,6 +153,7 @@ import org.apache.solr.util.OrderedExecutor;
 import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.StartupLoggingUtils;
 import org.apache.solr.util.stats.MetricUtils;
+import org.apache.solr.zero.process.ZeroStoreManager;
 import org.apache.zookeeper.KeeperException;
 import org.glassfish.hk2.utilities.binding.AbstractBinder;
 import org.glassfish.jersey.server.ApplicationHandler;
@@ -270,6 +272,8 @@ public class CoreContainer {
 
   protected MetricsHandler metricsHandler;
 
+  protected ZeroStoreManager zeroStoreManager;
+
   private volatile SolrClientCache solrClientCache;
 
   private volatile Map<String, SolrCache<?, ?>> caches;
@@ -1009,6 +1013,12 @@ public class CoreContainer {
 
     if (isZooKeeperAware()) {
       metricManager.loadClusterReporters(metricReporters, this);
+
+      ZeroConfig zeroConfig = getNodeConfig().getZeroConfig();
+      if (zeroConfig.isZeroStoreEnabled()) {
+        log.info("Zero store is enabled in Solr Cloud. Building ZeroStoreManager.");
+        zeroStoreManager = new ZeroStoreManager(this, zeroConfig);
+      }
     }
 
     // setup executor to load cores in parallel
@@ -1031,6 +1041,23 @@ public class CoreContainer {
             new Object[] {this});
     try {
       List<CoreDescriptor> cds = coresLocator.discover(this);
+      if (isZeroStoreEnabled()) {
+        // We can start on a clean node and might be missing cores of ZERO replicas that belong to
+        // this node.
+        // We will make sure we create those missing core descriptors (if any) and add to the list
+        // of discovered core descriptors.
+        // Later in this method the cores will be created for those additional core descriptors
+        // because isLoadOnStartup is set to true for them
+        List<CoreDescriptor> additionalCoreDescriptors =
+            getZeroStoreManager().discoverAdditionalCoreDescriptorsForZeroReplicas(cds, this);
+
+        if (!additionalCoreDescriptors.isEmpty()) {
+          // enhance the list of discovered cores with the additional ones we just
+          // discovered/created
+          cds = new ArrayList<>(cds);
+          cds.addAll(additionalCoreDescriptors);
+        }
+      }
       cds = coreSorter.sort(cds);
       checkForDuplicateCoreNames(cds);
       status |= CORE_DISCOVERY_COMPLETE;
@@ -1340,6 +1367,10 @@ public class CoreContainer {
           metricManager.closeReporters(
               SolrMetricManager.getRegistryName(SolrInfoBean.Group.cluster));
         }
+
+        if (zeroStoreManager != null) {
+          zeroStoreManager.shutdown();
+        }
       }
 
       try {
@@ -1740,6 +1771,11 @@ public class CoreContainer {
         throw e;
       }
       solrCores.removeCoreDescriptor(dcore);
+
+      if (isZeroStoreEnabled()) {
+        zeroStoreManager.evictCoreZeroMetadata(dcore);
+      }
+
       final SolrException solrException =
           new SolrException(
               ErrorCode.SERVER_ERROR, "Unable to create core [" + dcore.getName() + "]", e);
@@ -1753,6 +1789,11 @@ public class CoreContainer {
               t);
       coreInitFailures.put(dcore.getName(), new CoreLoadFailure(dcore, e));
       solrCores.removeCoreDescriptor(dcore);
+
+      if (isZeroStoreEnabled()) {
+        zeroStoreManager.evictCoreZeroMetadata(dcore);
+      }
+
       if (core != null && !core.isClosed()) IOUtils.closeQuietly(core);
       throw t;
     } finally {
@@ -2122,7 +2163,6 @@ public class CoreContainer {
    */
   public void unload(
       String name, boolean deleteIndexDir, boolean deleteDataDir, boolean deleteInstanceDir) {
-
     CoreDescriptor cd = solrCores.getCoreDescriptor(name);
 
     if (name != null) {
@@ -2136,6 +2176,9 @@ public class CoreContainer {
         // If last time around we didn't successfully load, make sure that all traces of the
         // coreDescriptor are gone.
         if (cd != null) {
+          if (isZeroStoreEnabled()) {
+            zeroStoreManager.evictCoreZeroMetadata(cd);
+          }
           solrCores.removeCoreDescriptor(cd);
           coresLocator.delete(this, cd);
         }
@@ -2152,6 +2195,9 @@ public class CoreContainer {
     boolean close = solrCores.isLoadedNotPendingClose(name);
     SolrCore core = solrCores.remove(name);
 
+    if (isZeroStoreEnabled()) {
+      zeroStoreManager.evictCoreZeroMetadata(cd);
+    }
     solrCores.removeCoreDescriptor(cd);
     coresLocator.delete(this, cd);
     if (core == null) {
@@ -2205,6 +2251,9 @@ public class CoreContainer {
         // The old coreDescriptor is obsolete, so remove it. registerCore will put it back.
         CoreDescriptor cd = core.getCoreDescriptor();
         solrCores.removeCoreDescriptor(cd);
+        if (isZeroStoreEnabled()) {
+          zeroStoreManager.evictCoreZeroMetadata(cd);
+        }
         cd.setProperty("name", toName);
         solrCores.addCoreDescriptor(cd);
         core.setName(toName);
@@ -2414,6 +2463,10 @@ public class CoreContainer {
     return zkSys.getZkController();
   }
 
+  public ZeroStoreManager getZeroStoreManager() {
+    return zeroStoreManager;
+  }
+
   public NodeConfig getConfig() {
     return cfg;
   }
@@ -2507,6 +2560,10 @@ public class CoreContainer {
     return tragicException != null;
   }
 
+  public boolean isZeroStoreEnabled() {
+    return getZeroStoreManager() != null;
+  }
+
   public ContainerPluginsRegistry getContainerPluginsRegistry() {
     return containerPluginsRegistry;
   }
diff --git a/solr/core/src/java/org/apache/solr/core/NodeConfig.java b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
index ef1cbbf2dfd..e2af4c81d19 100644
--- a/solr/core/src/java/org/apache/solr/core/NodeConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/NodeConfig.java
@@ -112,7 +112,7 @@ public class NodeConfig {
   private final String managementPath;
 
   private final PluginInfo[] backupRepositoryPlugins;
-
+  private final ZeroConfig zeroConfig;
   private final MetricsConfig metricsConfig;
 
   private final Map<String, CacheConfig> cachesConfig;
@@ -151,6 +151,7 @@ public class NodeConfig {
       SolrResourceLoader loader,
       Properties solrProperties,
       PluginInfo[] backupRepositoryPlugins,
+      ZeroConfig zeroConfig,
       MetricsConfig metricsConfig,
       Map<String, CacheConfig> cachesConfig,
       PluginInfo tracerConfig,
@@ -190,6 +191,7 @@ public class NodeConfig {
     this.loader = loader;
     this.solrProperties = solrProperties;
     this.backupRepositoryPlugins = backupRepositoryPlugins;
+    this.zeroConfig = zeroConfig;
     this.metricsConfig = metricsConfig;
     this.cachesConfig = cachesConfig == null ? Collections.emptyMap() : cachesConfig;
     this.tracerConfig = tracerConfig;
@@ -416,6 +418,10 @@ public class NodeConfig {
     return backupRepositoryPlugins;
   }
 
+  public ZeroConfig getZeroConfig() {
+    return zeroConfig;
+  }
+
   public MetricsConfig getMetricsConfig() {
     return metricsConfig;
   }
@@ -602,6 +608,7 @@ public class NodeConfig {
     private String managementPath;
     private Properties solrProperties = new Properties();
     private PluginInfo[] backupRepositoryPlugins;
+    private ZeroConfig zeroConfig;
     private MetricsConfig metricsConfig;
     private Map<String, CacheConfig> cachesConfig;
     private PluginInfo tracerConfig;
@@ -653,6 +660,7 @@ public class NodeConfig {
       setSolrDataHome(System.getProperty(SolrXmlConfig.SOLR_DATA_HOME));
       setConfigSetBaseDirectory("configsets");
       this.metricsConfig = new MetricsConfig.MetricsConfigBuilder().build();
+      this.zeroConfig = new ZeroConfig.ZeroConfigBuilder().build();
     }
 
     public NodeConfigBuilder setCoreRootDirectory(String coreRootDirectory) {
@@ -783,6 +791,11 @@ public class NodeConfig {
       return this;
     }
 
+    public NodeConfigBuilder setZeroConfig(ZeroConfig zeroConfig) {
+      this.zeroConfig = zeroConfig;
+      return this;
+    }
+
     public NodeConfigBuilder setMetricsConfig(MetricsConfig metricsConfig) {
       this.metricsConfig = metricsConfig;
       return this;
@@ -911,6 +924,7 @@ public class NodeConfig {
           loader,
           solrProperties,
           backupRepositoryPlugins,
+          zeroConfig,
           metricsConfig,
           cachesConfig,
           tracerConfig,
diff --git a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
index 4c15cd9b7a9..9097901cb3f 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrXmlConfig.java
@@ -166,6 +166,7 @@ public class SolrXmlConfig {
     if (cloudConfig != null) configBuilder.setCloudConfig(cloudConfig);
     configBuilder.setBackupRepositoryPlugins(
         getBackupRepositoryPluginInfos(root.get("backup").getAll("repository")));
+    configBuilder.setZeroConfig(getZeroConfig(root.get("zero")));
     configBuilder.setClusterPlugins(getClusterPlugins(loader, root));
     // <metrics><hiddenSysProps></metrics> will be removed in Solr 10, but until then, use it if a
     // <hiddenSysProps> is not provided under <solr>.
@@ -253,6 +254,7 @@ public class SolrXmlConfig {
     assertSingleInstance(root.getAll("logging"), "logging");
     assertSingleInstance(root.get("logging").getAll("watcher"), "logging/watcher");
     assertSingleInstance(root.getAll("backup"), "backup");
+    assertSingleInstance(root.getAll("zero"), "zero");
     assertSingleInstance(root.getAll("coreAdminHandlerActions"), "coreAdminHandlerActions");
   }
 
@@ -649,6 +651,22 @@ public class SolrXmlConfig {
     return configs;
   }
 
+  private static ZeroConfig getZeroConfig(ConfigNode zero) {
+    ZeroConfig.ZeroConfigBuilder builder = new ZeroConfig.ZeroConfigBuilder();
+    boolean enabled = zero.boolAttr("enabled", true);
+    builder.setEnabled(enabled);
+    if (enabled) {
+      builder.setBackupRepositoryPlugins(
+          getPluginsInfo(zero.get("repositories").getAll("repository")));
+    } else {
+      if (log.isInfoEnabled()) {
+        log.info("Zero store (ZERO replica) type is disabled.");
+      }
+    }
+
+    return builder.build();
+  }
+
   private static PluginInfo[] getClusterPlugins(SolrResourceLoader loader, ConfigNode root) {
     List<PluginInfo> clusterPlugins = new ArrayList<>();
 
@@ -827,4 +845,13 @@ public class SolrXmlConfig {
     if (cfg == null || !cfg.exists()) return null;
     return new PluginInfo(cfg, cfg.name(), false, true);
   }
+
+  private static PluginInfo[] getPluginsInfo(List<ConfigNode> cfgs) {
+    List<PluginInfo> configs = new ArrayList<>();
+    for (ConfigNode cfg : cfgs) {
+      PluginInfo pluginInfo = getPluginInfo(cfg);
+      if (pluginInfo != null) configs.add(pluginInfo);
+    }
+    return configs.toArray(new PluginInfo[0]);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/core/ZeroConfig.java b/solr/core/src/java/org/apache/solr/core/ZeroConfig.java
new file mode 100644
index 00000000000..1f4f87f792d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/ZeroConfig.java
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.core;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.concurrent.TimeUnit;
+import org.apache.solr.zero.process.CorePuller;
+
+/**
+ * In memory representation of the {@code <zero> ... </zero>} section of solr.xml configuration file
+ * For now contains only the description of a BackupRepository to be used as an abstraction over
+ * desired Zero store. Multiple repositories can be defined, but only one will be used : - the
+ * enabled one - the first of the enabled ones if multiple are enabled - the first one if none are
+ * enabled the zero section must contain enabled = true for the repositories configuration to be
+ * loaded
+ */
+public class ZeroConfig {
+
+  private final PluginInfo[] repositories;
+  private final boolean enabled;
+
+  @VisibleForTesting
+  public ZeroConfig() {
+    this(false, new PluginInfo[0]);
+  }
+
+  private ZeroConfig(boolean enabled, PluginInfo[] repositories) {
+    this.enabled = enabled;
+    this.repositories = repositories;
+  }
+
+  public boolean isEnabled() {
+    return enabled;
+  }
+
+  private static final PluginInfo[] NO_OP_REPOSITORIES = new PluginInfo[0];
+
+  public PluginInfo[] getBackupRepositoryPlugins() {
+    if (enabled) {
+      return repositories;
+    } else {
+      return NO_OP_REPOSITORIES;
+    }
+  }
+
+  /**
+   * Limit to the number of Zero store files to delete accepted on the delete queue (and lost in
+   * case of server crash). When the queue reaches that size, no more deletes are accepted (will be
+   * retried later for a core, next time it is pushed).
+   */
+  private final int almostMaxDeleterQueueSize = 200;
+
+  private final int deleterThreadPoolSize = 5;
+
+  private final int maxDeleteAttempts = 50;
+  private final long deleteSleepMsFailedAttempt = TimeUnit.SECONDS.toMillis(10);
+
+  /**
+   * After a successful core push has marked a file to be deleted, wait at least this long before
+   * actually deleting it from the Zero store. This is in case another node is still fetching files
+   * from the Zero store based on an older version of shard.properties that includes that file in
+   * the commit point.
+   *
+   * <p>Note the time at which a file is marked deleted is when it got added to the new
+   * shard.metadata file (or actually to the {@link
+   * org.apache.solr.zero.metadata.ZeroStoreShardMetadata}), not when the push finished in success.
+   * If the push took a very long time, by the time the new shard.metadata is visible, it's possible
+   * that all newly added files to delete are already ripe for deleting (which is likely ok, and
+   * even if that makes another node's pull to fail, upon retry that node will be able to pull the
+   * core correctly).
+   */
+  private final long deleteDelayMs = TimeUnit.MINUTES.toMillis(3);
+
+  /** Enables/disables Zero store for a Solr node. */
+  private final boolean zeroStoreEnabled =
+      Boolean.getBoolean(ZeroSystemProperty.ZeroStoreEnabled.getPropertyName());
+
+  /**
+   * Thread pool size for the executor that pushes files to Zero store. Defaults to 30 threads for
+   * now, to support higher indexing throughput.
+   */
+  private final int numFilePusherThreads =
+      Integer.getInteger(ZeroSystemProperty.numFilePusherThreads.getPropertyName(), 30);
+
+  /**
+   * Thread pool size for the executor that pulls files from Zero store. Defaults to 100 threads.
+   */
+  private final int numFilePullerThreads =
+      Integer.getInteger(ZeroSystemProperty.numFilePullerThreads.getPropertyName(), 100);
+
+  /** Max number of cores being concurrenty pulled async. Defaults to 10 cores. */
+  private final int numCorePullerThreads =
+      Integer.getInteger(ZeroSystemProperty.numCorePullerThreads.getPropertyName(), 10);
+
+  /**
+   * Number of automatic retries (after a transient IO error) to pull a core before giving up. This
+   * is the number of retries in addition to the initial pull attempt. The automatic retries are
+   * only triggered when the error is considered as transient. Defaults to 10. Attempts are spaced
+   * by at least {@link #corePullRetryDelayMs}), which means we'll retry for at least 180 seconds
+   * before giving up. This is something to adjust as the implementation of the delay between
+   * retries is cleaned up, see {@link CorePuller}.
+   */
+  private final int numCorePullAutoRetries =
+      Integer.getInteger(ZeroSystemProperty.numCorePullAutoRetries.getPropertyName(), 10);
+
+  /**
+   * Retry delay, in milliseconds, after a core pull failure when loading a core. Defaults to 20s.
+   */
+  private final long corePullRetryDelayMs =
+      TimeUnit.SECONDS.toMillis(
+          Integer.getInteger(ZeroSystemProperty.corePullRetryDelayS.getPropertyName(), 20));
+
+  /**
+   * Base delay, in milliseconds, between two attempts to pull core index files when starting an
+   * index update. Then the effective delay is computed by multiplying this base delay with the
+   * number of previous consecutive failed attempts. Defaults to 3s.
+   */
+  private final long corePullAttemptDelayMs =
+      TimeUnit.SECONDS.toMillis(
+          Integer.getInteger(ZeroSystemProperty.corePullAttemptDelayS.getPropertyName(), 3));
+
+  /**
+   * Maximum number of consecutive failed attempts to pull a core. The number of attempts takes into
+   * account the automatic retries, if any, and the request-based attempts. For example, if a core
+   * is corrupted on the Zero store side, then after this maximum number of attempts to pull it and
+   * open the index, no more pulls will be done and any attempt will fail with an exception.
+   *
+   * <p>TODO : not used as intended / seems to act as a duplicate to numCorePullAutoRetries
+   */
+  private final int maxFailedCorePullAttempts =
+      Integer.getInteger(ZeroSystemProperty.maxFailedCorePullAttempts.getPropertyName(), 20);
+
+  /**
+   * Delay, in milliseconds, before allowing another attempts when all pull attempts have been
+   * exhausted. Defaults to 5 min.
+   */
+  private final long allAttemptsExhaustedRetryDelayMs =
+      TimeUnit.MINUTES.toMillis(
+          Integer.getInteger(
+              ZeroSystemProperty.allAttemptsExhaustedRetryDelayMin.getPropertyName(), 5));
+
+  public boolean isZeroStoreEnabled() {
+    return zeroStoreEnabled;
+  }
+
+  /** Thread pool size for the executor that pushes files to Zero store. */
+  public int getNumFilePusherThreads() {
+    return numFilePusherThreads;
+  }
+
+  /** Thread pool size for the executor that pulls files from Zero store. */
+  public int getNumFilePullerThreads() {
+    return numFilePullerThreads;
+  }
+
+  /** Max number of cores being concurrenty pulled async */
+  public int getNumCorePullerThreads() {
+    return numCorePullerThreads;
+  }
+
+  /**
+   * Number of automatic retries to pull a core before giving up. This is the number of retries in
+   * addition to the initial pull attempt. The automatic retries are only triggered when the error
+   * is considered as transient.
+   *
+   * <p>TODO requires a review as this configuration is only used from tests ;(
+   */
+  public int getNumCorePullAutoRetries() {
+    return numCorePullAutoRetries;
+  }
+
+  /** Retry delay, in milliseconds, after a core pull failure when loading a core. */
+  public long getCorePullRetryDelay() {
+    return corePullRetryDelayMs;
+  }
+
+  /**
+   * Base delay, in milliseconds, between two attempts to pull core index files when starting an
+   * index update. Then the effective delay is computed by multiplying this base delay with the
+   * number of previous consecutive failed attempts.
+   */
+  public long getCorePullAttemptDelay() {
+    return corePullAttemptDelayMs;
+  }
+
+  /** Maximum number of consecutive failed attempts to pull a core. */
+  public int getMaxFailedCorePullAttempts() {
+    return maxFailedCorePullAttempts;
+  }
+
+  public long getDeleteDelayMs() {
+    return deleteDelayMs;
+  }
+
+  public int getAlmostMaxDeleterQueueSize() {
+    return almostMaxDeleterQueueSize;
+  }
+
+  public int getDeleterThreadPoolSize() {
+    return deleterThreadPoolSize;
+  }
+
+  public int getMaxDeleteAttempts() {
+    return maxDeleteAttempts;
+  }
+
+  public long getDeleteSleepMsFailedAttempt() {
+    return deleteSleepMsFailedAttempt;
+  }
+
+  /**
+   * Delay, in milliseconds, before allowing another attempts when all pull attempts have been
+   * exhausted.
+   */
+  public long getAllAttemptsExhaustedRetryDelay() {
+    return allAttemptsExhaustedRetryDelayMs;
+  }
+
+  /** Enum of constants representing the system properties used by the Zero store feature */
+  public enum ZeroSystemProperty {
+    ZeroStoreEnabled("zeroStoreEnabled"),
+    numFilePusherThreads("zeroStoreNumFilePusherThreads"),
+    numFilePullerThreads("zeroStoreNumFilePullerThreads"),
+    numCorePullerThreads("zeroStoreNumCorePullerThreads"),
+    numCorePullAutoRetries("zeroStoreNumCorePullAutoRetries"),
+    corePullRetryDelayS("zeroStoreCorePullRetryDelayS"),
+    corePullAttemptDelayS("zeroStoreCorePullAttemptDelayS"),
+    maxFailedCorePullAttempts("zeroStoreMaxFailedCorePullAttempts"),
+    allAttemptsExhaustedRetryDelayMin("zeroStoreAllAttemptsExhaustedRetryDelayMin"),
+    ;
+
+    private final String name;
+
+    ZeroSystemProperty(String propName) {
+      name = propName;
+    }
+
+    public String getPropertyName() {
+      return name;
+    }
+  }
+
+  public static class ZeroConfigBuilder {
+    private PluginInfo[] repositoryPlugins = new PluginInfo[0];
+    private boolean enabled = true;
+
+    public ZeroConfigBuilder() {}
+
+    public ZeroConfigBuilder setEnabled(boolean enabled) {
+      this.enabled = enabled;
+      return this;
+    }
+
+    public ZeroConfigBuilder setBackupRepositoryPlugins(PluginInfo[] repositoryPlugins) {
+      this.repositoryPlugins = repositoryPlugins != null ? repositoryPlugins : new PluginInfo[0];
+      return this;
+    }
+
+    public ZeroConfig build() {
+      return new ZeroConfig(enabled, repositoryPlugins);
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
index a5451dc1b2b..3f09ffec0d6 100644
--- a/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/ReplicationHandler.java
@@ -69,6 +69,7 @@ import org.apache.solr.api.JerseyResource;
 import org.apache.solr.client.api.model.SolrJerseyResponse;
 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.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
@@ -82,6 +83,7 @@ import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.SuppressForbidden;
 import org.apache.solr.core.CloseHook;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
@@ -266,6 +268,10 @@ public class ReplicationHandler extends RequestHandlerBase
     final SolrParams solrParams = req.getParams();
     String command = solrParams.required().get(COMMAND);
 
+    if (isZeroReplica(core)) {
+      throw new SolrException(ErrorCode.BAD_REQUEST, "ZERO replicas should never peer replicate!");
+    }
+
     // This command does not give the current index version of the leader
     // It gives the current 'replicateable' index version
     if (command.equals(CMD_INDEX_VERSION)) {
@@ -312,6 +318,13 @@ public class ReplicationHandler extends RequestHandlerBase
     }
   }
 
+  private boolean isZeroReplica(SolrCore core) {
+    CoreDescriptor cd = core.getCoreDescriptor();
+    return cd != null
+        && cd.getCloudDescriptor() != null
+        && cd.getCloudDescriptor().getReplicaType() == Replica.Type.ZERO;
+  }
+
   static boolean getBoolWithBackwardCompatibility(
       SolrParams params, String preferredKey, String alternativeKey, boolean defaultValue) {
     Boolean value = params.getBool(preferredKey);
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index 485670a4421..7429c1b7314 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -1053,7 +1053,10 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
             DocCollection.verifyProp(m, prop);
           }
           if (m.get(REPLICATION_FACTOR) != null) {
-            m.put(Replica.Type.defaultType().numReplicasPropertyName, m.get(REPLICATION_FACTOR));
+            // TODO assuming a non Zero index collection here! MODIFYCOLLECTION doesn't work on ZERO
+            // yet
+            m.put(
+                Replica.Type.defaultType(false).numReplicasPropertyName, m.get(REPLICATION_FACTOR));
           }
           return m;
         }),
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java b/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
index fd29d31fcd0..c970d2db6c5 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/SplitOp.java
@@ -43,10 +43,12 @@ import org.apache.solr.common.cloud.CompositeIdRouter;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.DocCollection.CollectionStateProps;
 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.params.CommonAdminParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
@@ -56,6 +58,7 @@ import org.apache.solr.update.SplitIndexCommand;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.util.RTimer;
 import org.apache.solr.util.RefCounted;
+import org.apache.solr.zero.process.ZeroStoreManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -75,8 +78,51 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
     return true;
   }
 
+  /**
+   * This method delegates the actual split work to {@link
+   * #executeInternal(CoreAdminHandler.CallInfo)} and does additional work for ZERO replicas.
+   */
+  @SuppressWarnings("try")
   @Override
   public void execute(CoreAdminHandler.CallInfo it) throws Exception {
+    SolrParams params = it.req.getParams();
+    List<SolrCore> newCores = null;
+    try {
+      if (!Replica.Type.ZERO.name().equals(params.get(CoreAdminParams.REPLICA_TYPE))) {
+        // Not a ZERO replica? Or not even SolrCloud? delegate without doing anything else
+        newCores = executeInternal(it);
+      } else {
+        // ZERO replica: need to pull from Zero store first and hold the indexing lock during the
+        // whole process
+        CoreContainer coreContainer = it.handler.getCoreContainer();
+        try (SolrCore core = coreContainer.getCore(params.get(CoreAdminParams.CORE))) {
+          ZeroStoreManager storeManager = coreContainer.getZeroStoreManager();
+          storeManager.pullCoreFromZeroStore(core);
+          try (AutoCloseable ignore = storeManager.acquireIndexingLockForSplit(core)) {
+            newCores = executeInternal(it);
+          }
+          // Now push the new cores to Zero store (if parent is ZERO, so are the children)
+          if (newCores != null) {
+            for (SolrCore newCore : newCores) {
+              newCore.getCoreContainer().getZeroStoreManager().initialCorePushToZeroStore(newCore);
+            }
+          }
+        }
+      }
+    } finally {
+      if (newCores != null) {
+        for (SolrCore newCore : newCores) {
+          newCore.close();
+        }
+      }
+    }
+  }
+
+  /**
+   * @return the list of created {@link SolrCore} or possibly <code>null</code>. If returns
+   *     non-null, caller must close all the cores.
+   */
+  private List<SolrCore> executeInternal(CoreAdminHandler.CallInfo it) throws Exception {
     SolrParams params = it.req.getParams();
     String splitKey = params.get("split.key");
     String[] newCoreNames = params.getParams("targetCore");
@@ -86,7 +132,7 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
     // to calculate the prefix ranges, and do the actual split in a separate request
     if (params.getBool(GET_RANGES, false)) {
       handleGetRanges(it, cname);
-      return;
+      return null;
     }
 
     // if not using splitByPrefix, determine split partitions
@@ -231,12 +277,10 @@ class SplitOp implements CoreAdminHandler.CoreAdminOp {
     } finally {
       if (req != null) req.close();
       if (parentCore != null) parentCore.close();
-      if (newCores != null) {
-        for (SolrCore newCore : newCores) {
-          newCore.close();
-        }
-      }
+      // closing newCores is caller's responsibility
     }
+
+    return newCores;
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
index 4042d303be1..50893d6c444 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/CreateCollection.java
@@ -33,6 +33,8 @@ import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFI
 import static org.apache.solr.common.params.CollectionAdminParams.PULL_REPLICAS;
 import static org.apache.solr.common.params.CollectionAdminParams.REPLICATION_FACTOR;
 import static org.apache.solr.common.params.CollectionAdminParams.TLOG_REPLICAS;
+import static org.apache.solr.common.params.CollectionAdminParams.ZERO_INDEX;
+import static org.apache.solr.common.params.CollectionAdminParams.ZERO_REPLICAS;
 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.CoreAdminParams.NAME;
@@ -198,16 +200,31 @@ public class CreateCollection extends AdminAPIBase implements CreateCollectionAp
     } else {
       rawProperties.put(CREATE_NODE_SET, "EMPTY");
     }
-    // 'nrtReplicas' and 'replicationFactor' are both set on the remote message, despite being
-    // functionally equivalent.
-    if (reqBody.replicationFactor != null) {
-      rawProperties.put(REPLICATION_FACTOR, reqBody.replicationFactor);
-      if (reqBody.nrtReplicas == null) rawProperties.put(NRT_REPLICAS, reqBody.replicationFactor);
-    }
-    if (reqBody.nrtReplicas != null) {
-      rawProperties.put(NRT_REPLICAS, reqBody.nrtReplicas);
-      if (reqBody.replicationFactor == null)
-        rawProperties.put(REPLICATION_FACTOR, reqBody.nrtReplicas);
+    if (reqBody.isZeroIndex == null || !reqBody.isZeroIndex) {
+      // 'nrtReplicas' and 'replicationFactor' are both set on the remote message, despite being
+      // functionally equivalent.
+      if (reqBody.replicationFactor != null) {
+        rawProperties.put(REPLICATION_FACTOR, reqBody.replicationFactor);
+        if (reqBody.nrtReplicas == null) rawProperties.put(NRT_REPLICAS, reqBody.replicationFactor);
+      }
+      if (reqBody.nrtReplicas != null) {
+        rawProperties.put(NRT_REPLICAS, reqBody.nrtReplicas);
+        if (reqBody.replicationFactor == null)
+          rawProperties.put(REPLICATION_FACTOR, reqBody.nrtReplicas);
+      }
+    } else {
+      rawProperties.put(ZERO_INDEX, true);
+      // Zero Index, there are only ZERO replicas
+      if (reqBody.replicationFactor != null) {
+        rawProperties.put(REPLICATION_FACTOR, reqBody.replicationFactor);
+        if (reqBody.zeroReplicas == null)
+          rawProperties.put(ZERO_REPLICAS, reqBody.replicationFactor);
+      }
+      if (reqBody.zeroReplicas != null) {
+        rawProperties.put(ZERO_REPLICAS, reqBody.zeroReplicas);
+        if (reqBody.replicationFactor == null)
+          rawProperties.put(REPLICATION_FACTOR, reqBody.zeroReplicas);
+      }
     }
 
     if (reqBody.properties != null) {
@@ -314,6 +331,10 @@ public class CreateCollection extends AdminAPIBase implements CreateCollectionAp
     requestBody.tlogReplicas = params.getInt(ZkStateReader.TLOG_REPLICAS);
     requestBody.pullReplicas = params.getInt(ZkStateReader.PULL_REPLICAS);
     requestBody.nrtReplicas = params.getInt(ZkStateReader.NRT_REPLICAS);
+    requestBody.zeroReplicas = params.getInt(ZkStateReader.ZERO_REPLICAS);
+    // At collection creation (only) we can decide if it is Zero store based (i.e. having only
+    // ZERO replicas)
+    requestBody.isZeroIndex = params.getBool(ZkStateReader.ZERO_INDEX);
     requestBody.waitForFinalState = params.getBool(WAIT_FOR_FINAL_STATE);
     requestBody.perReplicaState = params.getBool(PER_REPLICA_STATE);
     requestBody.alias = params.get(ALIAS);
@@ -339,6 +360,14 @@ public class CreateCollection extends AdminAPIBase implements CreateCollectionAp
           "Cannot specify both replicationFactor and nrtReplicas as they mean the same thing");
     }
 
+    if (requestBody.replicationFactor != null
+        && requestBody.zeroReplicas != null
+        && (!requestBody.replicationFactor.equals(requestBody.zeroReplicas))) {
+      throw new SolrException(
+          SolrException.ErrorCode.BAD_REQUEST,
+          "Cannot specify both replicationFactor and zeroReplicas as they mean the same thing");
+    }
+
     SolrIdentifierValidator.validateCollectionName(requestBody.name);
 
     if (requestBody.shardNames != null && !requestBody.shardNames.isEmpty()) {
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
index e1e071e6418..9d8f5dd0612 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCollectionAPI.java
@@ -26,6 +26,7 @@ import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SE
 import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SET_SHUFFLE_PARAM;
 import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_PREFIX;
 import static org.apache.solr.common.params.CollectionAdminParams.REPLICATION_FACTOR;
+import static org.apache.solr.common.params.CollectionAdminParams.ZERO_INDEX;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.params.CoreAdminParams.BACKUP_ID;
@@ -77,7 +78,8 @@ public class RestoreCollectionAPI extends BackupAPIBase {
                   COLL_CONF,
                   REPLICATION_FACTOR,
                   CREATE_NODE_SET_PARAM,
-                  CREATE_NODE_SET_SHUFFLE_PARAM))
+                  CREATE_NODE_SET_SHUFFLE_PARAM,
+                  ZERO_INDEX))
           .collect(Collectors.toUnmodifiableSet());
 
   @Inject
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCore.java b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCore.java
index 3997b1971b4..eaf988aa44c 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCore.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/api/RestoreCore.java
@@ -25,6 +25,7 @@ import org.apache.solr.client.api.model.RestoreCoreRequestBody;
 import org.apache.solr.client.api.model.SolrJerseyResponse;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
@@ -144,6 +145,14 @@ public class RestoreCore extends CoreAdminAPIBase implements RestoreCoreApi {
       if (updateLog != null) {
         updateLog.applyBufferedUpdates();
       }
+
+      // Now push the core to the Zero store if applicable
+      if (core.getCoreDescriptor()
+          .getCloudDescriptor()
+          .getReplicaType()
+          .equals(Replica.Type.ZERO)) {
+        core.getCoreContainer().getZeroStoreManager().initialCorePushToZeroStore(core);
+      }
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index 92b8f090dfc..97e61d6b942 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -73,6 +73,7 @@ import org.apache.solr.util.circuitbreaker.CircuitBreaker;
 import org.apache.solr.util.circuitbreaker.CircuitBreakerRegistry;
 import org.apache.solr.util.plugin.PluginInfoInitialized;
 import org.apache.solr.util.plugin.SolrCoreAware;
+import org.apache.solr.zero.process.ZeroStoreManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.MDC;
@@ -460,6 +461,24 @@ public class SearchHandler extends RequestHandlerBase
 
       SolrQueryTimeoutImpl.set(req);
       try {
+        /*
+         * Stopgap: This is not an end-to-end query freshness solution for ZERO replica. Rather,
+         * it is just taking care of a special case: if a core has never synced with the Zero
+         * store then it should fail the query.
+         *
+         * This stopgap is only taking care of the query path that goes through SearchHandler. This
+         * is not a generic enough place.
+         * A cleaner solution would have to find a more central place to take care of different
+         * entry points where we care about freshness.
+         *
+         * See also comment in HttpSolrCall.init()
+         */
+        CoreContainer coreContainer = core.getCoreContainer();
+        if (coreContainer.isZeroStoreEnabled()) {
+          ZeroStoreManager zeroStoreManager = coreContainer.getZeroStoreManager();
+          zeroStoreManager.ensureZeroCoreFreshness(core);
+        }
+
         // The semantics of debugging vs not debugging are different enough that
         // it makes sense to have two control loops
         if (!rb.isDebug()) {
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index e93c412676e..afd31b8e554 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -88,6 +88,7 @@ import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.JsonSchemaValidator;
@@ -331,6 +332,27 @@ public class HttpSolrCall {
           solrReq = parser.parse(core, path, req);
         }
 
+        if (cores.isZooKeeperAware()) {
+          String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
+          DocCollection collection = getCollection(collectionName);
+          if (collection != null && collection.isZeroIndex()) {
+            // TODO this is not clean, the decision to pull from Zero store should not be made here
+            // Needs to be moved to the appropriate handlers/components (enqueue pull by default
+            // from all handlers and allow handlers to disable that enqueue, in which case allow
+            // components in that handler to explicitly pull)
+            if (doesPathContainUpdate()) {
+              // Zero collection only supports hard commits therefore we always ensure one
+              addCommitIfAbsent();
+              // don't enqueue a pull on updates as those will already trigger their own synchronous
+              // pulls
+            } else {
+              if (pathRequiresZeroStorePulling(path)) {
+                cores.getZeroStoreManager().enqueueCorePullFromZeroStore(core);
+              }
+            }
+          }
+        }
+
         invalidStates =
             checkStateVersionsAreValid(solrReq.getParams().get(CloudSolrClient.STATE_VERSION));
 
@@ -345,6 +367,46 @@ public class HttpSolrCall {
     action = PASSTHROUGH;
   }
 
+  private void addCommitIfAbsent() {
+    Boolean currentValue = solrReq.getParams().getBool(UpdateParams.COMMIT);
+    if (currentValue == null || !currentValue) {
+      ModifiableSolrParams params = new ModifiableSolrParams(solrReq.getParams());
+      params.set(UpdateParams.COMMIT, "true");
+      // normally commits wait for a searcher, but we only need durability here so don't wait
+      if (params.get(UpdateParams.WAIT_SEARCHER) == null) {
+        params.set(UpdateParams.WAIT_SEARCHER, false);
+      }
+
+      solrReq.setParams(params);
+    }
+  }
+
+  /** TODO this is ugly we shouldn't hard code a path that could change */
+  protected boolean doesPathContainUpdate() {
+    return path.endsWith("/update") || path.contains("/update/");
+  }
+
+  /**
+   * Find out if our request should trigger a pull from Zero store when local core is stale.
+   *
+   * <p>TODO this is ugly we shouldn't hard code a path that could change
+   */
+  protected boolean pathRequiresZeroStorePulling(String servletPath) {
+    // get the request handler from the path (taken from SolrDispatchFilter)
+    int idx = servletPath.indexOf('/');
+    if (idx != -1) {
+      String action = servletPath.substring(idx);
+      return action.startsWith("/select")
+          || action.startsWith("/spellcheck")
+          || action.startsWith("/result_promotion")
+          || action.startsWith("/indexLookup")
+          || action.startsWith("/highlight")
+          || action.startsWith("/backup");
+    } else {
+      return false;
+    }
+  }
+
   protected void autoCreateSystemColl(String corename) throws Exception {
     if (core == null
         && SYSTEM_COLL.equals(corename)
@@ -1030,16 +1092,20 @@ public class HttpSolrCall {
     return result;
   }
 
-  protected SolrCore getCoreByCollection(String collectionName, boolean isPreferLeader) {
+  protected DocCollection getCollection(String collectionName) {
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
-
     ClusterState clusterState = zkStateReader.getClusterState();
-    DocCollection collection = clusterState.getCollectionOrNull(collectionName, true);
+
+    return clusterState.getCollectionOrNull(collectionName, true);
+  }
+
+  protected SolrCore getCoreByCollection(String collectionName, boolean isPreferLeader) {
+    DocCollection collection = getCollection(collectionName);
     if (collection == null) {
       return null;
     }
 
-    Set<String> liveNodes = clusterState.getLiveNodes();
+    Set<String> liveNodes = cores.getZkController().getClusterState().getLiveNodes();
 
     if (isPreferLeader) {
       List<Replica> leaderReplicas =
diff --git a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index 732304af8bb..da70e0c1406 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -30,9 +30,11 @@ import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.search.Sort;
 import org.apache.solr.cloud.ActionThrottle;
+import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.RecoveryStrategy;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.DirectoryFactory;
@@ -295,6 +297,11 @@ public final class DefaultSolrCoreState extends SolrCoreState
 
   @Override
   public void doRecovery(CoreContainer cc, CoreDescriptor cd) {
+    CloudDescriptor cloudDescriptor = cd.getCloudDescriptor();
+    if (cloudDescriptor != null && cloudDescriptor.getReplicaType() == Replica.Type.ZERO) {
+      log.info("Skipping recovery because ZERO replicas don't need it.");
+      return;
+    }
 
     Runnable recoveryTask =
         new Runnable() {
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateLog.java b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
index c9301356a0c..088dfa1baf9 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateLog.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateLog.java
@@ -105,6 +105,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
   private boolean debug = log.isDebugEnabled();
   private boolean trace = log.isTraceEnabled();
   private boolean usableForChildDocs;
+  private boolean rejectUpdatesWhileBuffering;
 
   public enum SyncLevel {
     NONE,
@@ -349,9 +350,12 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     dataDir = (String) info.initArgs.get("dir");
     defaultSyncLevel = SyncLevel.getSyncLevel((String) info.initArgs.get("syncLevel"));
 
+    rejectUpdatesWhileBuffering =
+        Boolean.TRUE.equals(info.initArgs.getBooleanArg("rejectUpdatesWhileBuffering"));
     numRecordsToKeep = objToInt(info.initArgs.get("numRecordsToKeep"), 100);
     maxNumLogsToKeep = objToInt(info.initArgs.get("maxNumLogsToKeep"), 10);
     numVersionBuckets = objToInt(info.initArgs.get("numVersionBuckets"), 65536);
+
     if (numVersionBuckets <= 0)
       throw new SolrException(
           SolrException.ErrorCode.SERVER_ERROR,
@@ -591,6 +595,18 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
     return Long.parseLong(last.substring(TLOG_NAME.length() + 1));
   }
 
+  /**
+   * Throw an exception if rejectUpdatesWhileBuffering=true. Method called while in BUFFERING state.
+   */
+  private void throwIfRejectUpdatesWhileBuffering() {
+    if (rejectUpdatesWhileBuffering) {
+      throw new SolrException(
+          ErrorCode.SERVICE_UNAVAILABLE,
+          "UpdateLog initialized with "
+              + "rejectUpdatesWhileBuffering=true, and we are in BUFFERING state. Rejecting update.");
+    }
+  }
+
   public void add(AddUpdateCommand cmd) {
     add(cmd, false);
   }
@@ -602,6 +618,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
 
     synchronized (this) {
       if ((cmd.getFlags() & UpdateCommand.BUFFERING) != 0) {
+        throwIfRejectUpdatesWhileBuffering();
         ensureBufferTlog();
         bufferTlog.write(cmd);
         return;
@@ -671,6 +688,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
 
     synchronized (this) {
       if ((cmd.getFlags() & UpdateCommand.BUFFERING) != 0) {
+        throwIfRejectUpdatesWhileBuffering();
         ensureBufferTlog();
         bufferTlog.writeDelete(cmd);
         return;
@@ -700,6 +718,7 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
   public void deleteByQuery(DeleteUpdateCommand cmd) {
     synchronized (this) {
       if ((cmd.getFlags() & UpdateCommand.BUFFERING) != 0) {
+        throwIfRejectUpdatesWhileBuffering();
         ensureBufferTlog();
         bufferTlog.writeDeleteByQuery(cmd);
         return;
@@ -1740,7 +1759,8 @@ public class UpdateLog implements PluginInfoInitialized, SolrMetricProducer {
       recoveryInfo = new RecoveryInfo();
 
       if (log.isInfoEnabled()) {
-        log.info("Starting to buffer updates. {}", this);
+        log.info(
+            "Starting to {} updates. {}", rejectUpdatesWhileBuffering ? "reject" : "buffer", this);
       }
 
       state = State.BUFFERING;
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
index 141ee7ff0b8..71787e9cd55 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessorFactory.java
@@ -19,6 +19,7 @@ package org.apache.solr.update.processor;
 import java.util.Collections;
 import java.util.Set;
 import java.util.TreeSet;
+import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 
@@ -56,9 +57,17 @@ public class DistributedUpdateProcessorFactory extends UpdateRequestProcessorFac
 
     DistributedUpdateProcessor distribUpdateProcessor =
         isZkAware
-            ? new DistributedZkUpdateProcessor(req, rsp, next)
+            ? getZkAwareInstance(req, rsp, next)
             : new DistributedUpdateProcessor(req, rsp, next);
     // note: will sometimes return DURP (no overhead) instead of wrapping
     return RoutedAliasUpdateProcessor.wrap(req, distribUpdateProcessor);
   }
+
+  private static DistributedUpdateProcessor getZkAwareInstance(
+      SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+    return req.getCore().getCoreDescriptor().getCloudDescriptor().getReplicaType()
+            == Replica.Type.ZERO
+        ? new ZeroStoreUpdateProcessor(req, rsp, next)
+        : new DistributedZkUpdateProcessor(req, rsp, next);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
index 35d96a7c14a..a4c09049bce 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedZkUpdateProcessor.java
@@ -76,12 +76,12 @@ import org.slf4j.LoggerFactory;
 
 public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
 
-  private final CloudDescriptor cloudDesc;
-  private final ZkController zkController;
+  protected final CloudDescriptor cloudDesc;
+  protected final ZkController zkController;
   private final SolrCmdDistributor cmdDistrib;
   protected List<SolrCmdDistributor.Node> nodes;
   private Set<String> skippedCoreNodeNames;
-  private final String collection;
+  protected final String collection;
   private boolean readOnlyCollection = false;
   private boolean broadcastDeleteById = false;
 
@@ -127,7 +127,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     }
   }
 
-  private boolean isReadOnly() {
+  protected boolean isReadOnly() {
     return readOnlyCollection || req.getCore().readOnly;
   }
 
@@ -270,6 +270,8 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     // replicationTracker to null if we aren't the leader or subShardLeader
     checkReplicationTracker(cmd);
 
+    postSetupHook();
+
     super.processAdd(cmd);
   }
 
@@ -366,6 +368,8 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     // replicationTracker to null if we aren't the leader or subShardLeader
     checkReplicationTracker(cmd);
 
+    postSetupHook();
+
     super.doDeleteById(cmd);
   }
 
@@ -459,6 +463,9 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     // check if client has requested minimum replication factor information. will set
     // replicationTracker to null if we aren't the leader or subShardLeader
     checkReplicationTracker(cmd);
+
+    postSetupHook();
+
     super.doDeleteByQuery(cmd, replicas, coll);
   }
 
@@ -712,27 +719,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
 
     clusterState = zkController.getClusterState();
     DocCollection coll = clusterState.getCollection(collection);
-    Slice slice = coll.getRouter().getTargetSlice(id, doc, route, req.getParams(), coll);
-
-    if (slice == null) {
-      // No slice found.  Most strict routers will have already thrown an exception, so a null
-      // return is a signal to use the slice of this core.
-      // TODO: what if this core is not in the targeted collection?
-      String shardId = cloudDesc.getShardId();
-      slice = coll.getSlice(shardId);
-      if (slice == null) {
-        throw new SolrException(
-            SolrException.ErrorCode.BAD_REQUEST, "No shard " + shardId + " in " + coll);
-      }
-      // if doc == null, then this is a DeleteById request with missing route, flag for forwarding
-      // to all shard leaders
-      if (doc == null
-          && coll.getRouter() instanceof CompositeIdRouter
-          && coll.getActiveSlicesMap().size() > 1) {
-        broadcastDeleteById = true;
-      }
-    }
-
+    Slice slice = getTargetSlice(coll, id, doc, route);
     DistribPhase phase = DistribPhase.parseParam(req.getParams().get(DISTRIB_UPDATE_PARAM));
 
     if (DistribPhase.FROMLEADER == phase && !couldIbeSubShardLeader(coll)) {
@@ -800,6 +787,36 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     }
   }
 
+  protected void postSetupHook() {}
+
+  protected Slice getTargetSlice(
+      DocCollection coll, String id, SolrInputDocument doc, String route) {
+    Slice slice = coll.getRouter().getTargetSlice(id, doc, route, req.getParams(), coll);
+
+    if (slice == null) {
+      // No slice found.  Most strict routers will have already thrown an exception, so a null
+      // return is a signal to use the slice of this core.
+      // TODO: what if this core is not in the targeted collection?
+      String shardId = cloudDesc.getShardId();
+      slice = coll.getSlice(shardId);
+
+      if (slice == null) {
+        throw new SolrException(
+            SolrException.ErrorCode.BAD_REQUEST, "No shard " + shardId + " in " + coll);
+      }
+
+      // if doc == null, then this is a DeleteById request with missing route, flag for forwarding
+      // to all shard leaders
+      if (doc == null
+          && coll.getRouter() instanceof CompositeIdRouter
+          && coll.getActiveSlicesMap().size() > 1) {
+        broadcastDeleteById = true;
+      }
+    }
+
+    return slice;
+  }
+
   @Override
   protected boolean shouldCloneCmdDoc() {
     boolean willDistrib = isLeader && nodes != null && nodes.size() > 0;
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ZeroStoreUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/ZeroStoreUpdateProcessor.java
new file mode 100644
index 00000000000..97706486382
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/update/processor/ZeroStoreUpdateProcessor.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update.processor;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.solr.cloud.api.collections.SplitShardCmd;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica.Type;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.SolrCmdDistributor;
+import org.apache.solr.update.UpdateCommand;
+import org.apache.solr.zero.process.ZeroCoreIndexingBatchProcessor;
+
+/**
+ * This class is extending and replacing {@link DistributedZkUpdateProcessor} to be used in {@link
+ * DistributedUpdateProcessorFactory} when dealing with cores for {@link
+ * org.apache.solr.common.cloud.Replica.Type#ZERO} replicas.
+ *
+ * <p>It the glue for adding Zero related code to normal SolrCloud processing: Rejection if shard
+ * splits are in progress is done in this class, and actual work delegated to {@link
+ * ZeroCoreIndexingBatchProcessor} (this only happens if the request is to be processed locally as
+ * opposed to being forwarded to other nodes/replicas).
+ *
+ * <p>Delegated work is calling {@link
+ * ZeroCoreIndexingBatchProcessor#addOrDeleteGoingToBeIndexedLocally} when an indexing batch starts
+ * and {@link ZeroCoreIndexingBatchProcessor#hardCommitCompletedLocally} after local hard commit was
+ * done.
+ *
+ * <p>Plugging into the {@code updateRequestProcessorChain} of {@code solrconfig.xml} is not ideal,
+ * it assumes {@link DistributedUpdateProcessorFactory} is configured there in the right place.
+ * Would be nice to find another way of plugging the Zero processing into SolrCloud (pulling before
+ * data is needed, pushing after indexing has completed).
+ *
+ * <p>Given the locking in use, it is assumed that indexing of a given batch (from the first doc to
+ * the hard commit, sent by the client or added automatically in {@link
+ * org.apache.solr.servlet.HttpSolrCall#addCommitIfAbsent}) is done by a single thread.
+ */
+public class ZeroStoreUpdateProcessor extends DistributedZkUpdateProcessor {
+  private ZeroCoreIndexingBatchProcessor zeroCoreIndexingBatchProcessor;
+
+  public ZeroStoreUpdateProcessor(
+      SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+    super(req, rsp, next);
+  }
+
+  @Override
+  public void processCommit(CommitUpdateCommand cmd) throws IOException {
+
+    if (isReadOnly()) {
+      throw new SolrException(
+          SolrException.ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
+    if (!cmd.softCommit) {
+      doLocalCommit(cmd);
+      getZeroCoreIndexingBatchProcessor().hardCommitCompletedLocally();
+    }
+  }
+
+  /**
+   * Override the behavior of setupRequest so that it rejects an update if the target shard is
+   * splitting
+   */
+  @Override
+  protected List<SolrCmdDistributor.Node> setupRequest(
+      String id, SolrInputDocument doc, String route, UpdateCommand cmd) {
+    DocCollection coll = clusterState.getCollection(cloudDesc.getCollectionName());
+    Slice slice = getTargetSlice(coll, id, doc, route);
+
+    rejectUpdateIfSplitting(coll, slice);
+
+    return super.setupRequest(id, doc, route, cmd);
+  }
+
+  @Override
+  protected void postSetupHook() {
+    // this should be called after setupRequest and before the doc is indexed locally
+    // because it needs to know whether current replica is leader or subShardLeader for the doc or
+    // not
+    if (willBeIndexedLocally()) {
+      getZeroCoreIndexingBatchProcessor().addOrDeleteGoingToBeIndexedLocally();
+    }
+  }
+
+  /**
+   * Override the behavior of doDeleteByQuery so that it rejects an update if any shard in the
+   * collection is splitting
+   */
+  @Override
+  public void doDeleteByQuery(DeleteUpdateCommand cmd) throws IOException {
+    DocCollection coll = clusterState.getCollection(cloudDesc.getCollectionName());
+
+    // This will reject the update if any shard in the collection is currently being split
+    rejectIfConstructing(coll.getSlices());
+    super.doDeleteByQuery(cmd);
+  }
+
+  /** Reject the update if the target shard is splitting. */
+  private void rejectUpdateIfSplitting(DocCollection coll, Slice shard) {
+    // Find the sub shards, if any
+    Set<Slice> subShards =
+        coll.getSlices().stream()
+            .filter(s -> (s.getParent() != null && s.getParent().equals(shard.getName())))
+            .collect(Collectors.toSet());
+    if (!subShards.isEmpty()) {
+      // Reject if they are in construction or recovery state
+      rejectIfConstructing(subShards);
+    }
+  }
+
+  /**
+   * Reject the update if any of the shards in the provided collection are in construction or
+   * recovery state, indicating that the parent is splitting. This is because the transaction log is
+   * not trusted to work (nodes are stateless) so we avoid using it.
+   *
+   * <p>Do note though that there is a related issue, see comment at the end of constructor of
+   * {@link ZeroCoreIndexingBatchProcessor#ZeroCoreIndexingBatchProcessor}
+   *
+   * @param shards The collection of shards to check for construction or recovery state
+   */
+  private void rejectIfConstructing(Collection<Slice> shards) {
+
+    Set<String> splittingShards =
+        shards.stream()
+            .filter(
+                s ->
+                    (s.getState() == Slice.State.CONSTRUCTION
+                        || s.getState() == Slice.State.RECOVERY))
+            .map(s -> s.getParent())
+            .collect(Collectors.toSet());
+
+    for (String splittingShard : splittingShards) {
+      // To avoid potential deadlock in the case of catastrophic Overseer failure during a split, it
+      // is necessary
+      // to verify that the split lock exists, and that the split is actually "in progress"
+      try {
+        if (SplitShardCmd.shardSplitLockHeld(
+            zkController, cloudDesc.getCollectionName(), splittingShard)) {
+          throw new SolrException(
+              SolrException.ErrorCode.SERVICE_UNAVAILABLE,
+              "Rejecting updates to shard " + splittingShard + " while a split is in progress.");
+        }
+      } catch (SolrException e) {
+        throw e;
+      } catch (Exception e) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVICE_UNAVAILABLE,
+            "Rejecting updates to shard "
+                + splittingShard
+                + ", can't check if a split is in progress.",
+            e);
+      }
+    }
+  }
+
+  @Override
+  protected void doClose() {
+    if (zeroCoreIndexingBatchProcessor != null) {
+      zeroCoreIndexingBatchProcessor.close();
+    }
+    super.doClose();
+  }
+
+  private boolean willBeIndexedLocally() {
+    // forwardToLeader: if true, then the update is going to be forwarded to its rightful leader.
+    // The doc being added or deleted might not even belong to the current core's (req.getCore())
+    // shard.
+    // isLeader: if true, then the current core (req.getCore()) is the leader of the shard to which
+    // the doc being added or deleted belongs to.
+    // For ZERO replicas only leader replicas do local indexing. Follower ZERO replicas do not do
+    // any local indexing and forward the add/delete updates to the leader replica.
+    // isSubShardLeader: if true, then the current core (req.getCore()) is the leader of a sub shard
+    // being built (why is this rejected then? Should be rejected if the subshard is not active yet
+    // but logically it would end up being indexed locally).
+    //
+    // Therefore, only the leader replicas of the ZERO active shards will process docs locally and
+    // thus need to pull/push from the Zero store during indexing.
+    return !forwardToLeader && isLeader && !isSubShardLeader;
+  }
+
+  private ZeroCoreIndexingBatchProcessor getZeroCoreIndexingBatchProcessor() {
+    assert Type.ZERO.equals(replicaType);
+
+    if (zeroCoreIndexingBatchProcessor == null) {
+      updateCoreIndexingBatchProcessor(
+          new ZeroCoreIndexingBatchProcessor(req.getCore(), clusterState, rsp));
+    }
+    return zeroCoreIndexingBatchProcessor;
+  }
+
+  @VisibleForTesting
+  public void updateCoreIndexingBatchProcessor(
+      ZeroCoreIndexingBatchProcessor zeroCoreIndexingBatchProcessor) {
+    this.zeroCoreIndexingBatchProcessor = zeroCoreIndexingBatchProcessor;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/client/ZeroFile.java b/solr/core/src/java/org/apache/solr/zero/client/ZeroFile.java
new file mode 100644
index 00000000000..035fb543f9b
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/client/ZeroFile.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.client;
+
+import com.fasterxml.jackson.annotation.JsonGetter;
+import com.google.common.annotations.VisibleForTesting;
+import java.net.URI;
+import java.time.Instant;
+import java.util.Objects;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.zero.util.ToFromJson;
+
+/**
+ * The parent representation of a file on the Zero store. Contains the information required to
+ * locate and name that file in the store, but nothing else. This class is instantiated directly
+ * when listing files on the Zero store.
+ *
+ * <p>ZeroFile instances (including existing subclasses) are equal if they represent the same Zero
+ * store file, even if other parameters of the instance are different. This makes it easier to
+ * compare for example {@link ZeroFile} files coming from a directory listing of the Zero store and
+ * {@link WithLocal} files of a core that are present both locally and on the Zero store.
+ */
+public class ZeroFile {
+  protected final String collectionName;
+  protected final String shardName;
+  protected final String zeroFileName;
+
+  /** For Jackson deserialization. See {@link ToFromJson}. */
+  protected ZeroFile() {
+    collectionName = shardName = zeroFileName = null;
+  }
+
+  public ZeroFile(String collectionName, String shardName, String zeroFileName) {
+    this.collectionName = collectionName;
+    this.shardName = shardName;
+    this.zeroFileName = zeroFileName;
+  }
+
+  protected ZeroFile(ZeroFile f) {
+    this.collectionName = f.collectionName;
+    this.shardName = f.shardName;
+    this.zeroFileName = f.zeroFileName;
+  }
+
+  public String getCollectionName() {
+    return collectionName;
+  }
+
+  public String getShardName() {
+    return shardName;
+  }
+
+  public String getZeroFileName() {
+    return zeroFileName;
+  }
+
+  public URI getFileURI(BackupRepository repo) {
+    URI pathURI = getShardDirectoryURI(repo);
+    return repo.resolve(pathURI, getZeroFileName());
+  }
+
+  public URI getShardDirectoryURI(BackupRepository repo) {
+    URI collectionURI = getCollectionDirectoryURI(repo);
+    return repo.resolveDirectory(collectionURI, shardName);
+  }
+
+  public URI getCollectionDirectoryURI(BackupRepository repo) {
+    URI baseURI = repo.createURI(repo.getBackupLocation(null));
+    return repo.resolveDirectory(baseURI, collectionName);
+  }
+
+  public boolean pathEmpty() {
+    return (collectionName == null
+        || collectionName.isEmpty()
+        || shardName == null
+        || shardName.isEmpty());
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof ZeroFile) {
+      ZeroFile other = (ZeroFile) obj;
+      return Objects.equals(collectionName, other.collectionName)
+          && Objects.equals(shardName, other.shardName)
+          && Objects.equals(zeroFileName, other.zeroFileName);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(collectionName, shardName, zeroFileName);
+  }
+
+  /**
+   * A file stored in the Zero store with its matching local copy name. The names of the two are
+   * totally independent of each other, even though in general the Zero store name is built using
+   * the local Solr name plus a random suffix.
+   *
+   * <p>This class relies on {@link ZeroFile#equals(Object)}. Two local files representing the same
+   * Zero store file but having different Solr names are considered equal. In practice this should
+   * never happen, but relying on the superclass comparison allows for example comparing for
+   * equality files that are part of an index and files coming from a listing of files on the Zero
+   * store.
+   */
+  public static class WithLocal extends ZeroFile {
+    protected final String solrFileName;
+
+    private final boolean checksumPresent;
+
+    /**
+     * Name the file should have on a Solr server retrieving it, not including the core specific
+     * part of the filename (i.e. the path)
+     */
+    private final long fileSize;
+
+    /**
+     * Lucene generated checksum of the file. It is used in addition to file size to compare local
+     * and Zero store files.
+     */
+    private final long checksum;
+
+    /** For Jackson deserialization. See {@link ToFromJson}. */
+    protected WithLocal() {
+      solrFileName = null;
+      checksumPresent = false;
+      fileSize = checksum = 0L;
+    }
+
+    @VisibleForTesting
+    WithLocal(String collectionName, String shardName, String solrFileName, String zeroFileName) {
+      this(collectionName, shardName, solrFileName, zeroFileName, false, -1L, -1L);
+    }
+
+    public WithLocal(
+        String collectionName,
+        String shardName,
+        String solrFileName,
+        String zeroFileName,
+        long fileSize,
+        long checksum) {
+      this(collectionName, shardName, solrFileName, zeroFileName, true, fileSize, checksum);
+    }
+
+    private WithLocal(
+        String collectionName,
+        String shardName,
+        String solrFileName,
+        String zeroFileName,
+        boolean checksumPresent,
+        long fileSize,
+        long checksum) {
+      super(collectionName, shardName, zeroFileName);
+      this.solrFileName = solrFileName;
+      this.checksumPresent = checksumPresent;
+      this.fileSize = fileSize;
+      this.checksum = checksum;
+    }
+
+    public String getSolrFileName() {
+      return solrFileName;
+    }
+
+    public boolean isChecksumPresent() {
+      return checksumPresent;
+    }
+
+    public long getFileSize() {
+      return this.fileSize;
+    }
+
+    public long getChecksum() {
+      return this.checksum;
+    }
+  }
+
+  /**
+   * A file stored in the Zero store that is no longer used and should be deleted.
+   *
+   * <p>Deletes on the Zero store do not need to know the local Solr file name, therefore this class
+   * does not extend {@link WithLocal}.
+   *
+   * <p>This class relies on {@link ZeroFile#equals(Object)}. Two files to delete representing the
+   * same Zero store file but having different {@link #deletedAt} values are considered equal.
+   */
+  public static class ToDelete extends ZeroFile {
+
+    // Even though deletes are only done when it is clear the file is no longer needed (a successful
+    // push updated the Zero store commit point), it is possible that a slow pull on another node is
+    // still pulling files using an older commit point. Even if this happens qnd the pull fails, a
+    // subsequent pull would likely succeed. Maybe this timestamp is overdesign.
+    private Instant deletedAt;
+
+    /** For easy Jackson serialization of Instant type. See {@link ToFromJson}. */
+    @JsonGetter("deletedAt")
+    public long getMyInstantValEpoch() {
+      return deletedAt.toEpochMilli();
+    }
+
+    /** For Jackson deserialization of Instant type. See {@link ToFromJson}. */
+    @JsonGetter("deletedAt")
+    public void setMyInstantValEpoch(long epochMilli) {
+      this.deletedAt = Instant.ofEpochMilli(epochMilli);
+    }
+
+    /** For Jackson deserialization. See {@link ToFromJson}. */
+    protected ToDelete() {
+      deletedAt = null;
+    }
+
+    public ToDelete(ZeroFile zeroFile, Instant deletedAt) {
+      super(zeroFile);
+      this.deletedAt = deletedAt;
+    }
+
+    public ToDelete(String collectionName, String shardName, String zeroFileName) {
+      this(collectionName, shardName, zeroFileName, Instant.now());
+    }
+
+    public ToDelete(
+        String collectionName, String shardName, String zeroFileName, Instant deletedAt) {
+      super(collectionName, shardName, zeroFileName);
+      this.deletedAt = deletedAt;
+    }
+
+    /**
+     * @return time in milliseconds (converted from nanotime) that file was marked as deleted
+     */
+    public Instant getDeletedAt() {
+      return this.deletedAt;
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClient.java b/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClient.java
new file mode 100644
index 00000000000..8dadcc52231
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClient.java
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.client;
+
+import com.codahale.metrics.Counter;
+import com.codahale.metrics.InstrumentedExecutorService;
+import com.codahale.metrics.MetricRegistry;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.invoke.MethodHandles;
+import java.net.URI;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.FileAlreadyExistsException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutorService;
+import java.util.stream.Collectors;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.SolrInfoBean;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.LocalFileSystemRepository;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.apache.solr.metrics.SolrMetricProducer;
+import org.apache.solr.metrics.SolrMetricsContext;
+import org.apache.solr.zero.exception.ZeroException;
+import org.apache.solr.zero.metadata.ZeroStoreShardMetadata;
+import org.apache.solr.zero.util.IndexInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class is a wrapper around BackupRepository for easy reading and writing to an arbitrary
+ * underlying Zero store.
+ */
+public class ZeroStoreClient {
+
+  static final int CHUNK_SIZE = 16 * 1024 * 1024; // 16 MBs
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Using {@link BackupRepository} to instantiate the Zero store. That class will have to be
+   * eventually renamed as with this change its role extends beyond backups (keeping it as is to
+   * simplify looking at the Zero branch).
+   */
+  protected final BackupRepository zeroRepository;
+
+  private static final String METRIC_BASE_NAME = "ZeroStore";
+
+  /**
+   * Identifiers for file pusher/puller thread pools that run on all Solr nodes containing Zero
+   * collections
+   */
+  private static final String ZERO_STORE_FILE_PUSHER_THREAD_POOL = "ZeroStoreFilePusherThreadPool";
+
+  private static final String ZERO_STORE_FILE_PULLER_THREAD_POOL = "ZeroStoreFilePullerThreadPool";
+  private final ExecutorService filePushExecutor;
+  private final ExecutorService filePullExecutor;
+
+  private final Counter bytesPushed;
+  private final Counter numFilesSuccessfullyPushed;
+  private final Counter bytesPulled;
+  private final Counter numFilesSuccessfullyPulled;
+
+  public ZeroStoreClient(
+      BackupRepository zeroRepository,
+      SolrMetricManager metricManager,
+      final int numFilePusherThreads,
+      final int numFilePullerThreads) {
+    this.zeroRepository = zeroRepository;
+
+    String registryName = SolrMetricManager.getRegistryName(SolrInfoBean.Group.node);
+    MetricRegistry registry = metricManager.registry(registryName);
+
+    // Executor for pushing individual files to Zero store
+    ExecutorService unwrappedFilePushExecutor =
+        ExecutorUtil.newMDCAwareFixedThreadPool(
+            numFilePusherThreads, new NamedThreadFactory(ZERO_STORE_FILE_PUSHER_THREAD_POOL));
+    this.filePushExecutor =
+        new InstrumentedExecutorService(
+            unwrappedFilePushExecutor, registry, METRIC_BASE_NAME + ".push.execution");
+
+    // Executor for pulling individual files from Zero store
+    ExecutorService unwrappedFilePullExecutor =
+        ExecutorUtil.newMDCAwareFixedThreadPool(
+            numFilePullerThreads, new NamedThreadFactory(ZERO_STORE_FILE_PULLER_THREAD_POOL));
+    this.filePullExecutor =
+        new InstrumentedExecutorService(
+            unwrappedFilePullExecutor, registry, METRIC_BASE_NAME + ".pull.execution");
+
+    SolrMetricsContext solrMetricsContext =
+        new SolrMetricsContext(
+            metricManager, registryName, SolrMetricProducer.getUniqueMetricTag(this, null));
+
+    bytesPushed = solrMetricsContext.counter("push.numBytes", METRIC_BASE_NAME);
+    numFilesSuccessfullyPushed =
+        solrMetricsContext.counter("push.numFilesSuccessful", METRIC_BASE_NAME);
+    solrMetricsContext.gauge(() -> numFilePusherThreads, true, "push.numThreads", METRIC_BASE_NAME);
+
+    bytesPulled = solrMetricsContext.counter("pull.numBytes", METRIC_BASE_NAME);
+    numFilesSuccessfullyPulled =
+        solrMetricsContext.counter("pull.numFilesSuccessful", METRIC_BASE_NAME);
+    solrMetricsContext.gauge(() -> numFilePullerThreads, true, "pull.numThreads", METRIC_BASE_NAME);
+  }
+
+  /**
+   * Replaces the special SHARD_METADATA_ZERO_FILENAME file on the Zero store for the core by a new
+   * version passed as a {@link ZeroStoreShardMetadata} instance.
+   *
+   * @param zeroShardMetadataFile description of the Zero shard index data for which to write the
+   *     metadata file
+   * @param shardMetadata Zero shard metadata content to be serialized and written to the Zero store
+   */
+  public void pushShardMetadata(
+      ZeroFile.WithLocal zeroShardMetadataFile, ZeroStoreShardMetadata shardMetadata)
+      throws ZeroException {
+    if (zeroShardMetadataFile.pathEmpty())
+      throw new ZeroException("Can't write shard metadata file to empty path");
+    try {
+      createCoreStorage(zeroShardMetadataFile);
+      String zcmJson = shardMetadata.toJson();
+      OutputStream os =
+          zeroRepository.createOutput(zeroShardMetadataFile.getFileURI(zeroRepository));
+      os.write(zcmJson.getBytes(StandardCharsets.UTF_8));
+      os.close();
+    } catch (IOException e) {
+      throw handleIOException(e);
+    } catch (Exception ex) {
+      throw new ZeroException(ex);
+    }
+  }
+
+  /**
+   * Reads the special CORE_METADATA_ZERO_FILENAME file from the Zero store for the core and returns
+   * the corresponding {@link ZeroStoreShardMetadata} object.
+   *
+   * @param zeroShardMetadataFile description of the Zero shard index data to get metadata for
+   * @return <code>null</code> if the core does not exist on the Zero store or method {@link
+   *     #pushShardMetadata} was never called for it. Otherwise returns the latest value written
+   *     using {@link #pushShardMetadata} ("latest" here based on the consistency model of the
+   *     underlying store, in practice the last value written by any server given the strong
+   *     consistency of the Salesforce S3 implementation).
+   */
+  public ZeroStoreShardMetadata pullShardMetadata(ZeroFile zeroShardMetadataFile)
+      throws ZeroException {
+    try {
+
+      if (!shardMetadataExists(zeroShardMetadataFile)) {
+        return null;
+      }
+      try (IndexInput ii = pullStream(zeroShardMetadataFile)) {
+        OutputStream os = new ByteArrayOutputStream();
+        readIndexInput(ii, os);
+        String decodedJson = os.toString();
+        os.close();
+        return ZeroStoreShardMetadata.fromJson(decodedJson);
+      }
+    } catch (IOException e) {
+      throw handleIOException(e);
+    } catch (Exception ex) {
+      throw new ZeroException(ex);
+    }
+  }
+
+  /**
+   * Returns an input stream for the given Zero store file. The caller must close the stream when
+   * done.
+   *
+   * @param zeroFile description of the Zero shard index data to get a stream from
+   * @return the file's input stream
+   */
+  public IndexInput pullStream(ZeroFile zeroFile) throws ZeroException {
+    if (zeroFile.pathEmpty()) throw new ZeroException("Can't read from empty Zero store path");
+    try {
+      return zeroRepository.openInput(
+          zeroFile.getShardDirectoryURI(zeroRepository),
+          zeroFile.getZeroFileName(),
+          IOContext.DEFAULT);
+    } catch (IOException e) {
+      throw handleIOException(e);
+    } catch (Exception ex) {
+      throw new ZeroException(ex);
+    }
+  }
+
+  /**
+   * There are two interfaces to push a Zero core (shard) to the Zero store. In any case writes are
+   * done file by file. This interface writes to the external Zero store using an input stream for
+   * the given Zero store file.
+   *
+   * @param zeroFile description of the Zero store file to push data to
+   * @param is input stream of the file to push, which will be closed by this method after use
+   */
+  public void pushStream(ZeroFile zeroFile, InputStream is) throws ZeroException {
+    if (zeroFile.pathEmpty()) throw new ZeroException("Can't write to empty Zero store path");
+    try {
+      createCoreStorage(zeroFile);
+      OutputStream os = zeroRepository.createOutput(zeroFile.getFileURI(zeroRepository));
+      is.transferTo(os);
+      is.close();
+      os.close();
+    } catch (IOException e) {
+      throw handleIOException(e);
+    } catch (Exception ex) {
+      throw new ZeroException(ex);
+    }
+  }
+
+  /**
+   * Checks if the shard index data with the given Zero store shard metadata file exists. This
+   * method could actually be used to check the existence of any Zero store file, but it happens to
+   * only be used for shard metadata files.
+   *
+   * @param zeroShardMetadataFile description of the Zero store core (shard) metadata file to check
+   * @return true if the metadata file exists on the Zero store
+   */
+  public boolean shardMetadataExists(ZeroFile zeroShardMetadataFile) throws ZeroException {
+    try {
+      return zeroRepository.exists(zeroShardMetadataFile.getFileURI(zeroRepository));
+    } catch (IOException e) {
+      throw handleIOException(e);
+    } catch (Exception ex) {
+      throw new ZeroException(ex);
+    }
+  }
+
+  /**
+   * Batch delete files from the Zero store. Any file path that specifies a non-existent file will
+   * not be treated as an error and should return success.
+   *
+   * @param zeroFiles list of Zero file to be deleted
+   */
+  public void deleteZeroFiles(Collection<ZeroFile> zeroFiles) {
+    // TODO do we need to do something specific around deleting directories?
+    var files =
+        zeroFiles.stream()
+            .collect(Collectors.groupingBy(f -> f.getShardDirectoryURI(zeroRepository)));
+
+    for (Map.Entry<URI, List<ZeroFile>> entry : files.entrySet()) {
+      URI baseURI = entry.getKey();
+      List<String> filesName =
+          entry.getValue().stream().map(ZeroFile::getZeroFileName).collect(Collectors.toList());
+      try {
+        zeroRepository.delete(baseURI, filesName);
+      } catch (IOException e) {
+        if (log.isErrorEnabled()) {
+          log.error(
+              "Could not delete one of the following files in directory {} : {}",
+              baseURI.toString(),
+              StrUtils.join(filesName, ','),
+              e);
+        }
+      }
+    }
+  }
+
+  /** Lists all file names within the given path */
+  public Set<ZeroFile> listShardZeroFiles(String collectionName, String shardName) {
+    try {
+      URI shardURI = getShardURI(collectionName, shardName);
+      return Arrays.stream(zeroRepository.listAll(shardURI))
+          .map(zeroFileName -> new ZeroFile(collectionName, shardName, zeroFileName))
+          .collect(Collectors.toSet());
+    } catch (Exception e) {
+      log.error(
+          "Error while listing file for collection={} shard={}", collectionName, shardName, e);
+      return new LinkedHashSet<>();
+    }
+  }
+
+  public Set<ZeroFile> listCollectionZeroFiles(String collectionName) {
+    try {
+      URI collectionURI = getCollectionURI(collectionName);
+      return Arrays.stream(zeroRepository.listAll(collectionURI))
+          .flatMap(shardName -> listShardZeroFiles(collectionName, shardName).stream())
+          .collect(Collectors.toSet());
+    } catch (Exception e) {
+      log.error("Error while listing file for collection={}", collectionName, e);
+      return new LinkedHashSet<>();
+    }
+  }
+
+  /** Closes any resources used by the client */
+  public void shutdown() {
+    if (log.isInfoEnabled()) {
+      log.info("ZeroStoreClient is shutting down");
+    }
+
+    ExecutorUtil.shutdownAndAwaitTermination(filePushExecutor);
+    if (log.isInfoEnabled()) {
+      log.info("{} has shutdown", ZERO_STORE_FILE_PUSHER_THREAD_POOL);
+    }
+    ExecutorUtil.shutdownAndAwaitTermination(filePullExecutor);
+    if (log.isInfoEnabled()) {
+      log.info("{} has shutdown", ZERO_STORE_FILE_PULLER_THREAD_POOL);
+    }
+
+    try {
+      zeroRepository.close();
+      log.info("Zero store repository has shutdown");
+    } catch (IOException e) {
+      log.error("Zero store repository couldn't close properly");
+    }
+  }
+
+  /**
+   * Helper function to wrap a generic IOException into a ZeroException
+   *
+   * @param ioe exception to wrap
+   * @return wrapping exception
+   */
+  private ZeroException handleIOException(IOException ioe) {
+    String errMessage =
+        String.format(Locale.ROOT, "A Zero store IOException was thrown! %s", ioe.toString());
+    return new ZeroException(errMessage, ioe);
+  }
+
+  /**
+   * Required for some {@link BackupRepository} like {@link LocalFileSystemRepository}. Others do
+   * not need that like S3 since there are no directories, just keys with a prefix meaning no need
+   * to create the empty prefix before creating the file method is synchronized to avoid multi
+   * threading issues
+   *
+   * @param zeroFile file for which we want to be sure the directory hierarchy exists
+   */
+  private synchronized void createCoreStorage(ZeroFile zeroFile) throws IOException {
+    try {
+      URI collectionDirURI = zeroFile.getCollectionDirectoryURI(zeroRepository);
+      try {
+        // TODO the collection directory should be created once when collection created
+        if (!zeroRepository.exists(collectionDirURI)) {
+          zeroRepository.createDirectory(collectionDirURI);
+        }
+      } catch (FileAlreadyExistsException e) {
+        log.warn(
+            "Could not create collection directory as it already exists, collection={}",
+            zeroFile.getCollectionName(),
+            e);
+      }
+
+      // Maybe the collection directory was already created by another Solr node, but we anyway
+      // need to try creating the shard directory.
+      URI shardDirURI = zeroFile.getShardDirectoryURI(zeroRepository);
+      try {
+        if (!zeroRepository.exists(shardDirURI)) {
+          zeroRepository.createDirectory(shardDirURI);
+        }
+      } catch (FileAlreadyExistsException e) {
+        log.warn(
+            "Could not create shard directory as it already exists, collection={} shard={}",
+            zeroFile.getCollectionName(),
+            zeroFile.getShardName(),
+            e);
+      }
+    } catch (IOException e) {
+      log.warn(
+          "Error while creating directory for collection={} shard={}",
+          zeroFile.getCollectionName(),
+          zeroFile.getShardName(),
+          e);
+      throw e;
+    }
+  }
+
+  public URI getCollectionURI(String collectionName) {
+    URI baseURI = zeroRepository.createURI(zeroRepository.getBackupLocation(null));
+    return zeroRepository.resolveDirectory(baseURI, collectionName);
+  }
+
+  public URI getShardURI(String collectionName, String shardName) {
+    URI baseURI = zeroRepository.createURI(zeroRepository.getBackupLocation(null));
+    return zeroRepository.resolveDirectory(baseURI, collectionName, shardName);
+  }
+
+  public static void readIndexInput(IndexInput ii, OutputStream os) throws IOException {
+    byte[] buffer = new byte[CHUNK_SIZE];
+    int bufferLen;
+    long remaining = ii.length();
+    while (remaining > 0) {
+      bufferLen = remaining >= CHUNK_SIZE ? CHUNK_SIZE : (int) remaining;
+      ii.readBytes(buffer, 0, bufferLen);
+      os.write(buffer, 0, bufferLen);
+      remaining -= bufferLen;
+    }
+  }
+
+  public void deleteShardDirectory(String collectionName, String shardName) throws IOException {
+    URI shardURI = getShardURI(collectionName, shardName);
+    zeroRepository.deleteDirectory(shardURI);
+  }
+
+  public void deleteCollectionDirectory(String collectionName) throws IOException {
+    URI collectionURI = getCollectionURI(collectionName);
+    List<String> shardDirectoryName =
+        new ArrayList<>(Arrays.asList(zeroRepository.listAll(collectionURI)));
+    for (String d : shardDirectoryName) {
+      deleteShardDirectory(collectionName, d);
+    }
+    zeroRepository.deleteDirectory(collectionURI);
+  }
+
+  /** Asynchronously push shard.metadata to Zero store, using the push thread pool. */
+  public CompletableFuture<Void> pushShardMetadataAsync(
+      ZeroFile.WithLocal zeroShardMetadataFile, ZeroStoreShardMetadata shardMetadata) {
+    return CompletableFuture.runAsync(
+        () -> {
+          try {
+            pushShardMetadata(zeroShardMetadataFile, shardMetadata);
+          } catch (ZeroException e) {
+            throw new RuntimeException(
+                "Exception while pushing " + zeroShardMetadataFile.getZeroFileName(), e);
+          }
+        },
+        filePushExecutor);
+  }
+
+  /** Asynchronously push a local file to Zero store, using the push thread pool. */
+  public CompletableFuture<Void> pushFileAsync(Directory dir, ZeroFile.WithLocal localZeroFile) {
+    return CompletableFuture.runAsync(
+        () -> pushFileToZeroStore(dir, localZeroFile), filePushExecutor);
+  }
+
+  private void pushFileToZeroStore(Directory dir, ZeroFile.WithLocal localZeroFile) {
+    // This method pushes the normal segment files (as opposed to shard.metadata) and they should
+    // have file size and checksum set
+    assert localZeroFile.isChecksumPresent();
+    // Use Lucene-level IndexInput to lock on file lifecycle; don't want files deleted from
+    // underneath us
+    try (IndexInput ii = dir.openInput(localZeroFile.getSolrFileName(), IOContext.READONCE);
+        InputStream is = new IndexInputStream(ii)) {
+
+      pushStream(localZeroFile, is);
+
+      numFilesSuccessfullyPushed.inc();
+      long pushedFileLength = dir.fileLength(localZeroFile.getSolrFileName());
+      bytesPushed.inc(pushedFileLength);
+      assert pushedFileLength == localZeroFile.getFileSize();
+    } catch (Exception e) {
+      throw new RuntimeException(
+          "Exception while pushing file: " + localZeroFile.getSolrFileName(), e);
+    }
+  }
+
+  /** Asynchronously pull a local file from Zero store, using the pull thread pool, */
+  public CompletableFuture<Void> pullFileAsync(Directory dir, ZeroFile.WithLocal fileToDownload) {
+    return CompletableFuture.runAsync(
+        () -> pullFileFromZeroStore(dir, fileToDownload), filePullExecutor);
+  }
+
+  private void pullFileFromZeroStore(Directory destDir, ZeroFile.WithLocal fileToDownload) {
+    try (IndexOutput io =
+            destDir.createOutput(
+                fileToDownload.getSolrFileName(), DirectoryFactory.IOCONTEXT_NO_CACHE);
+        IndexInput ii = pullStream(fileToDownload)) {
+
+      if (log.isDebugEnabled()) {
+        log.debug(
+            "Copying {} from Zero store",
+            fileToDownload
+                .getSolrFileName()); // Logline has MDC, no need to add core/shard/replica info
+      }
+      io.copyBytes(ii, ii.length());
+      numFilesSuccessfullyPulled.inc();
+      bytesPulled.inc(fileToDownload.getFileSize());
+    } catch (ZeroException | IOException ex) {
+      throw new RuntimeException(
+          "Exception while pulling file: " + fileToDownload.getSolrFileName(), ex);
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClientFactory.java b/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClientFactory.java
new file mode 100644
index 00000000000..c32501a29d1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/client/ZeroStoreClientFactory.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.client;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.solr.common.util.StrUtils;
+import org.apache.solr.core.NodeConfig;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.core.ZeroConfig;
+import org.apache.solr.core.backup.repository.BackupRepository;
+import org.apache.solr.core.backup.repository.BackupRepositoryFactory;
+import org.apache.solr.metrics.SolrMetricManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Used to create a new ZeroStoreClient with underlying {@link BackupRepository}. BackupRepository
+ * properties are stored in solr.xml in the {@code <zero> ... </zero>} section
+ */
+public class ZeroStoreClientFactory {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static ZeroStoreClient newInstance(NodeConfig cfg, SolrMetricManager metricManager) {
+
+    return new ZeroStoreClient(
+        newZeroRepository(cfg),
+        metricManager,
+        cfg.getZeroConfig().getNumFilePusherThreads(),
+        cfg.getZeroConfig().getNumFilePullerThreads());
+  }
+
+  public static BackupRepository newZeroRepository(NodeConfig cfg) {
+
+    SolrResourceLoader ressourceLoader = cfg.getSolrResourceLoader();
+    ZeroConfig zeroConfig = cfg.getZeroConfig();
+    PluginInfo[] repoPlugins = zeroConfig.getBackupRepositoryPlugins();
+    BackupRepositoryFactory zeroRepositoryFactory = new BackupRepositoryFactory(repoPlugins);
+
+    BackupRepository repository;
+
+    List<String> enabledRepos =
+        Arrays.stream(repoPlugins)
+            .filter(PluginInfo::isEnabled)
+            .map(p -> p.name)
+            .collect(Collectors.toList());
+    if (enabledRepos.isEmpty()) {
+      List<String> allRepos =
+          Arrays.stream(repoPlugins).map(p -> p.name).collect(Collectors.toList());
+      if (allRepos.isEmpty()) {
+        log.warn("No Zero store BackupRepository defined, using default");
+        repository = zeroRepositoryFactory.newInstance(ressourceLoader);
+      } else if (allRepos.size() == 1) {
+        repository = zeroRepositoryFactory.newInstance(ressourceLoader, allRepos.get(0));
+      } else {
+        log.warn(
+            "Multiple not enabled Zero store BackupRepository defined ({}), using first: {}",
+            StrUtils.join(allRepos, ','),
+            allRepos.get(0));
+        repository = zeroRepositoryFactory.newInstance(ressourceLoader, allRepos.get(0));
+      }
+    } else if (enabledRepos.size() == 1) {
+      repository = zeroRepositoryFactory.newInstance(ressourceLoader, enabledRepos.get(0));
+    } else {
+      log.warn(
+          "Multiple enabled Zero store BackupRepository defined ({}), using first: {}",
+          StrUtils.join(enabledRepos, ','),
+          enabledRepos.get(0));
+      repository = zeroRepositoryFactory.newInstance(ressourceLoader, enabledRepos.get(0));
+    }
+
+    return repository;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/client/package-info.java b/solr/core/src/java/org/apache/solr/zero/client/package-info.java
new file mode 100644
index 00000000000..2608726223e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/client/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Zero store client implementations, utils, and exceptions */
+package org.apache.solr.zero.client;
diff --git a/solr/core/src/java/org/apache/solr/zero/exception/CorruptedCoreException.java b/solr/core/src/java/org/apache/solr/zero/exception/CorruptedCoreException.java
new file mode 100644
index 00000000000..8ec96a3963d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/exception/CorruptedCoreException.java
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.exception;
+
+/** Exception thrown when a core is corrupted locally. */
+public class CorruptedCoreException extends Exception {
+
+  public CorruptedCoreException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/exception/ZeroException.java b/solr/core/src/java/org/apache/solr/zero/exception/ZeroException.java
new file mode 100644
index 00000000000..e6964066262
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/exception/ZeroException.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.exception;
+
+/**
+ * Parent class of Zero store related issues. Likely to change and maybe disappear but good enough
+ * for a PoC.
+ */
+public class ZeroException extends Exception {
+  public ZeroException(Throwable cause) {
+    super(cause);
+  }
+
+  public ZeroException(String message) {
+    super(message);
+  }
+
+  public ZeroException(String message, Throwable cause) {
+    super(message, cause);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/exception/ZeroLockException.java b/solr/core/src/java/org/apache/solr/zero/exception/ZeroLockException.java
new file mode 100644
index 00000000000..27cccff9a5d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/exception/ZeroLockException.java
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.exception;
+
+import java.util.Locale;
+
+public class ZeroLockException extends Exception {
+
+  public ZeroLockException(String format, Object... args) {
+    super(String.format(Locale.ROOT, format, args));
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/exception/package-info.java b/solr/core/src/java/org/apache/solr/zero/exception/package-info.java
new file mode 100644
index 00000000000..fcd31c4be76
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/exception/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** Zero store exception types */
+package org.apache.solr.zero.exception;
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/LocalCoreMetadata.java b/solr/core/src/java/org/apache/solr/zero/metadata/LocalCoreMetadata.java
new file mode 100644
index 00000000000..e4adf58f451
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/LocalCoreMetadata.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.metadata;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.NoSuchFileException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.core.IndexDeletionPolicyWrapper;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.zero.exception.ZeroException;
+import org.apache.solr.zero.process.CorePuller;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Object capturing the metadata of a Solr core on a Solr node. */
+public class LocalCoreMetadata {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * When an instance of this class is created for pushing data to the Zero store, reserve the
+   * commit point for a short while to give the caller time to save it while it works on it. Can't
+   * save it here directly as it would be awkward to try to release it on all execution paths.
+   */
+  private static final long RESERVE_COMMIT_DURATION = TimeUnit.SECONDS.toMillis(1L);
+
+  private static final int MAX_ATTEMPTS_TO_CAPTURE_COMMIT_POINT = 5;
+
+  /**
+   * Files composing the core. They are referenced from the core's current commit point's segments_N
+   * file which is ALSO included in this collection.
+   */
+  private Collection<String> latestCommitFiles;
+
+  /**
+   * Names of all files (all of them, no exception) in the local index directory. These files do not
+   * matter when pushing contents to Zeo store but they do matter if Zero store content being pulled
+   * conflicts with them.
+   */
+  private Set<String> allFiles;
+
+  /**
+   * Hash of the directory content used to make sure the content doesn't change as we proceed to
+   * pull new files from the Zero store (if we need to pull new files from there)
+   */
+  private String directoryHash;
+
+  /**
+   * Generation number of the local index. This generation number is only meant to identify a
+   * scenario where local index generation number is higher than what we have in Zero store. In that
+   * scenario we would switch index to a new directory when pulling contents from the Zero store.
+   * Because in the presence of higher generation number locally, simply pulling the content of the
+   * Zero store for the core does not work (Solr favors the highest generation locally...). It is
+   * also used for saving (reserving) local commit while doing pushes.
+   */
+  private long generation;
+
+  protected final SolrCore core;
+
+  /** Given a core name, builds the local metadata */
+  public LocalCoreMetadata(SolrCore core) {
+
+    this.core = core;
+  }
+
+  public void readMetadata(boolean reserveCommit, boolean captureDirHash) throws Exception {
+
+    Directory coreDir = getCoreDirectory();
+    if (coreDir == null) return;
+
+    try {
+      Set<String> latestCommitBuilder;
+      IndexCommit latestCommit;
+      int attempt = 1;
+      // we don't have an atomic way of capturing a commit point or taking a snapshot of the
+      // entire index directory to compute a directory hash. In the commit point case, there
+      // is a slight chance of losing files between getting a latest commit and reserving it.
+      // Likewise, there is a chance of losing files between the time we list all file names
+      // in the directory and computing the directory hash. Therefore, we try to capture the
+      // commit point and compute the directory hash in a loop with maximum number of attempts.
+      String hash = null;
+      String[] fileNames;
+      while (true) {
+        try {
+          // Work around possible bug returning same file multiple times by using a set here
+          // See org.apache.solr.handler.ReplicationHandler.getFileList()
+          latestCommitBuilder = new HashSet<>();
+          latestCommit = tryCapturingLatestCommit(latestCommitBuilder, reserveCommit);
+
+          // Capture now the hash and verify again after files have been pulled and before the
+          // directory is updated (or before the index is switched to use a new directory) to
+          // make sure there are no local changes at the same time that might lead to a
+          // corruption in case of interaction with the download or might be a sign of other
+          // problems (it is not expected that indexing can happen on a local directory of a
+          // ZERO replica if that replica is not up-to-date with the Zero store version).
+          fileNames = coreDir.listAll();
+          if (captureDirHash) {
+            hash = getSolrDirectoryHash(coreDir, fileNames);
+          }
+          break;
+        } catch (FileNotFoundException | NoSuchFileException ex) {
+          attempt++;
+          if (attempt > MAX_ATTEMPTS_TO_CAPTURE_COMMIT_POINT) {
+            throw ex;
+          }
+          String reason = ex.getMessage();
+          log.error(
+              "Failed to capture directory snapshot for either commit point or entire directory listing: core={} attempt={} reason={}",
+              getCoreName(),
+              attempt,
+              reason);
+        }
+      }
+
+      generation = latestCommit.getGeneration();
+      latestCommitFiles = Collections.unmodifiableSet(latestCommitBuilder);
+      directoryHash = hash;
+
+      // Need to inventory all local files in case files that need to be pulled from Zero store
+      // conflict
+      // with them.
+      allFiles = Set.of(fileNames);
+    } finally {
+      releaseCoreDirectory(coreDir);
+    }
+  }
+
+  private IndexCommit tryCapturingLatestCommit(
+      Set<String> latestCommitBuilder, boolean reserveCommit) throws ZeroException, IOException {
+    IndexDeletionPolicyWrapper deletionPolicy = core.getDeletionPolicy();
+    IndexCommit latestCommit = deletionPolicy.getLatestCommit();
+    if (latestCommit == null) {
+      throw new ZeroException("Core " + getCoreName() + " has no available commit point");
+    }
+
+    if (reserveCommit) {
+      // Caller will save the commit point shortly. See CorePushPull.pushFilesToZeroStore()
+      deletionPolicy.setReserveDuration(latestCommit.getGeneration(), RESERVE_COMMIT_DURATION);
+    }
+    // Note we add here all segment related files as well as the commit point's segments_N file
+    // Commit points do not contain lock (write.lock) files.
+    latestCommitBuilder.addAll(latestCommit.getFileNames());
+    return latestCommit;
+  }
+
+  /**
+   * Computes a hash of a Solr Directory in order to make sure the directory doesn't change as we
+   * pull content into it (if we need to pull content into it)
+   */
+  private String getSolrDirectoryHash(Directory coreDir, String[] filesNames)
+      throws NoSuchAlgorithmException, IOException {
+    MessageDigest digest =
+        MessageDigest.getInstance("sha1"); // not sure MD5 is available in Solr jars
+
+    // Computing the hash requires items to be submitted in the same order...
+    Arrays.sort(filesNames);
+
+    for (String fileName : filesNames) {
+      // .lock files come and go. Ignore them (we're closing the Index Writer before adding any
+      // pulled files to the Core)
+      if (!fileName.endsWith(".lock")) {
+        // Hash the file name and file size so we can tell if any file has changed (or files
+        // appeared or vanished)
+        digest.update(fileName.getBytes(StandardCharsets.UTF_8));
+        try {
+          digest.update(
+              Long.toString(coreDir.fileLength(fileName)).getBytes(StandardCharsets.UTF_8));
+        } catch (FileNotFoundException | NoSuchFileException fnf) {
+          // The file was deleted between the listAll() and the check, use an impossible size to not
+          // match a digest
+          // for which the file is completely present or completely absent (which will cause this
+          // hash to never match that directory again).
+          digest.update(Long.toString(-42).getBytes(StandardCharsets.UTF_8));
+        }
+      }
+    }
+
+    return new String(Hex.encodeHex(digest.digest()));
+  }
+
+  protected String getCoreName() {
+    return this.core.getName();
+  }
+
+  protected String getCollectionName() {
+    return core.getCoreDescriptor().getCollectionName();
+  }
+
+  protected String getShardName() {
+    return core.getCoreDescriptor().getCloudDescriptor().getShardId();
+  }
+
+  public long getGeneration() {
+    return this.generation;
+  }
+
+  public String getDirectoryHash() {
+    return directoryHash;
+  }
+
+  public Set<String> getAllFiles() {
+    return allFiles;
+  }
+
+  public Collection<String> getLatestCommitFiles() {
+    return latestCommitFiles;
+  }
+
+  @Override
+  public String toString() {
+    return "collectionName="
+        + getCollectionName()
+        + " shardName="
+        + getShardName()
+        + " coreName="
+        + getCoreName()
+        + " generation="
+        + generation;
+  }
+
+  public Directory getCoreDirectory() throws IOException {
+    return CorePuller.getDirectory(core, core.getIndexDir());
+  }
+
+  public void releaseCoreDirectory(Directory coreDir) throws IOException {
+    if (coreDir != null) core.getDirectoryFactory().release(coreDir);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/MetadataCacheManager.java b/solr/core/src/java/org/apache/solr/zero/metadata/MetadataCacheManager.java
new file mode 100644
index 00000000000..7cffbb7e555
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/MetadataCacheManager.java
@@ -0,0 +1,455 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.metadata;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.invoke.MethodHandles;
+import java.util.Locale;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.ZeroConfig;
+import org.apache.solr.zero.process.CorePuller;
+import org.apache.solr.zero.process.ZeroAccessLocks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class maintains a cache of Zero store shard metadata for local cores of ZERO replicas. For
+ * each local core, an instance of {@link MetadataCacheEntry} is stored. The data comes in part from
+ * the Zero store, in part from ZooKeeper.
+ */
+public class MetadataCacheManager {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Value for a newly initialized MetadataCacheEntry instance. Not expected to make it to ZK,
+   * unlike real UUID's or the default value {@link
+   * ZeroMetadataController#METADATA_NODE_DEFAULT_VALUE} written at the very beginning of the ZERO
+   * shard's life cycle
+   */
+  private static final String METADATA_SUFFIX_CACHE_INITIAL_VALUE = "iNiTiAl";
+
+  private final ZeroConfig config;
+  private final CoreContainer coreContainer;
+
+  /**
+   * This map maintains for a local core information about Zero store shard metadata, corresponding
+   * metadataSuffix and version, as well as the locks for Zero store operations done on this node
+   */
+  private final ConcurrentHashMap<String, MetadataCacheEntry> coresMetadata;
+
+  /**
+   * Counters for the failed core pull attempts. Key is the core name, value is the counter which
+   * also includes the last attempt time.
+   */
+  private final ConcurrentHashMap<String, PullAttempts> coresPullAttempts;
+
+  /** Used by tests to provide their own {@link PullAttempts} factory. */
+  private final Supplier<PullAttempts> pullAttemptsFactory;
+
+  public MetadataCacheManager(ZeroConfig config, CoreContainer coreContainer) {
+    this(config, coreContainer, PullAttempts::new);
+  }
+
+  public MetadataCacheManager(
+      ZeroConfig config, CoreContainer coreContainer, Supplier<PullAttempts> pullAttemptsFactory) {
+    this.config = config;
+    this.coreContainer = coreContainer;
+    this.pullAttemptsFactory = pullAttemptsFactory;
+    coresMetadata = buildMetadataCache();
+    coresPullAttempts = new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Logs the current {@link ZeroCoreStage} a core is at. Increments the pull attempts ({@link
+   * ZeroCoreStage#PULL_STARTED}) to abort pulling if there were more than {@link
+   * ZeroConfig#getMaxFailedCorePullAttempts()} unsuccessful pulls (without the corresponding {@link
+   * ZeroCoreStage#PULL_SUCCEEDED}).
+   */
+  public void recordState(SolrCore core, ZeroCoreStage stage) {
+    String coreName = getCoreName(core);
+    String collectionName = getCollectionName(core);
+    String shardName = getShardName(core);
+    log.info(
+        "RecordZeroCoreStage: collection={} shard={} core={} stage={}",
+        collectionName,
+        shardName,
+        coreName,
+        stage);
+    switch (stage) {
+      case PULL_STARTED:
+        coresPullAttempts.compute(
+            coreName,
+            (__, attempts) -> {
+              if (attempts == null) {
+                attempts = pullAttemptsFactory.get();
+              }
+              return attempts.checkAndIncrement(coreName, config);
+            });
+        break;
+      case PULL_SUCCEEDED:
+        coresPullAttempts.remove(coreName);
+        break;
+      case PULL_FAILED:
+        coresPullAttempts.computeIfPresent(coreName, (__, attempts) -> attempts.onFailedAttempt());
+        break;
+      case PULL_FAILED_WITH_CORRUPTION:
+        // We only want to retry once to pull a corrupted core.
+        coresPullAttempts.compute(
+            coreName,
+            (__, attempts) -> {
+              if (attempts == null) {
+                attempts = pullAttemptsFactory.get();
+              }
+              if (!attempts.onFailedAttempt().exhaustAllButOne(config)) {
+                // Done with attempts to pull the corrupted core. Set the replica state to
+                // RECOVERY_FAILED
+                // to prevent updates and queries on it, and to alert.
+                setReplicaStateToRecoveryFailed(collectionName, shardName, coreName);
+              }
+              return attempts;
+            });
+        break;
+      default:
+        break;
+    }
+  }
+
+  /**
+   * Sets the replica state to RECOVERY_FAILED for a corrupted core that cannot be pulled from the
+   * Zero store. This state will prevent any update or query on the replica, and it will be
+   * monitored with a metric.
+   */
+  private void setReplicaStateToRecoveryFailed(
+      String collectionName, String shardName, String coreName) {
+    try (SolrCore core = coreContainer.getCore(coreName)) {
+      try {
+        coreContainer
+            .getZkController()
+            .publish(core.getCoreDescriptor(), Replica.State.RECOVERY_FAILED);
+      } catch (Exception e) {
+        log.error(
+            "Cannot set replica state to {} for core {} shard {} collection {}",
+            Replica.State.RECOVERY_FAILED,
+            coreName,
+            shardName,
+            collectionName);
+      }
+    }
+  }
+
+  /**
+   * Updates the {@link MetadataCacheEntry} for the core with passed in {@link ZeroMetadataVersion},
+   * {@link ZeroStoreShardMetadata} and {@code cacheLikelyUpToDate}
+   */
+  public void updateCoreMetadata(
+      String coreName,
+      ZeroMetadataVersion shardMetadataVersion,
+      ZeroStoreShardMetadata zeroStoreShardMetadata,
+      boolean cacheLikelyUpToDate) {
+    MetadataCacheEntry currentMetadata = getOrCreateCoreMetadata(coreName);
+    MetadataCacheEntry updatedMetadata =
+        currentMetadata.updatedOf(
+            shardMetadataVersion, zeroStoreShardMetadata, cacheLikelyUpToDate);
+    updateCoreMetadata(coreName, currentMetadata, updatedMetadata);
+  }
+
+  /** Updates {@link MetadataCacheEntry} for the core with passed in {@code cacheLikelyUpToDate} */
+  public void updateCoreMetadata(String coreName, boolean cacheLikelyUpToDate) {
+    MetadataCacheEntry currentMetadata = getOrCreateCoreMetadata(coreName);
+    MetadataCacheEntry updatedMetadata = currentMetadata.updatedOf(cacheLikelyUpToDate);
+    updateCoreMetadata(coreName, currentMetadata, updatedMetadata);
+  }
+
+  /**
+   * Evicts an entry from the {@link #coresMetadata} if one exists for the given core name.
+   *
+   * @return {@code true} if an entry was actually removed
+   */
+  public boolean removeCoreMetadata(String coreName) {
+    coresPullAttempts.remove(coreName);
+    return coresMetadata.remove(coreName) != null;
+  }
+
+  private void updateCoreMetadata(
+      String coreName, MetadataCacheEntry currentMetadata, MetadataCacheEntry updatedMetadata) {
+    if (currentMetadata.getZeroAccessLocks().canUpdateCoreMetadata()) {
+      if (log.isInfoEnabled()) {
+        log.info(
+            "updateCoreMetadata: core={}  current={} updated={}",
+            coreName,
+            currentMetadata,
+            updatedMetadata);
+      }
+      coresMetadata.put(coreName, updatedMetadata);
+    } else {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error updating the MetadataCacheEntry because the proper "
+              + "locks are not held by the current thread. "
+              + currentMetadata.locksHeldToString()
+              + " core="
+              + coreName);
+    }
+  }
+
+  /**
+   * Returns the cached {@link MetadataCacheEntry} representing Zero store metadata of the core or
+   * creates a new entry for the core if none exists.
+   */
+  public MetadataCacheEntry getOrCreateCoreMetadata(String coreName) {
+    return coresMetadata.computeIfAbsent(coreName, k -> new MetadataCacheEntry());
+  }
+
+  protected ConcurrentHashMap<String, MetadataCacheEntry> buildMetadataCache() {
+    return new ConcurrentHashMap<>();
+  }
+
+  public boolean hasInitialCacheSuffixValue(MetadataCacheEntry coreMetadata) {
+    return MetadataCacheManager.METADATA_SUFFIX_CACHE_INITIAL_VALUE.equals(
+        coreMetadata.getMetadataVersion().getMetadataSuffix());
+  }
+
+  protected String getCoreName(SolrCore solrCore) {
+    return solrCore.getName();
+  }
+
+  protected String getShardName(SolrCore solrCore) {
+    return solrCore.getCoreDescriptor().getCloudDescriptor().getShardId();
+  }
+
+  protected String getCollectionName(SolrCore solrCore) {
+    return solrCore.getCoreDescriptor().getCollectionName();
+  }
+
+  /**
+   * This represents metadata that needs to be cached for a core of a Zero collection {@link
+   * DocCollection#isZeroIndex()} so that it can be properly synchronized for concurrent indexing,
+   * pushes and pulls.
+   */
+  public static class MetadataCacheEntry {
+    /**
+     * Value originating from a ZooKeeper node used to handle conditionally and safely update the
+     * shard.metadata file written to the Zero store.
+     */
+    private final ZeroMetadataVersion metadataVersion;
+
+    /**
+     * {@link ZeroStoreShardMetadata} representing the state corresponding to {@link
+     * #metadataVersion}
+     */
+    private final ZeroStoreShardMetadata zeroStoreShardMetadata;
+
+    /**
+     * Whether cache likely up to date with {@link ZeroMetadataVersion} of the shard. In steady
+     * state this is set for leader cores when they push and pull {@link
+     * CorePuller#pullCoreWithRetries(boolean, long, long)} since followers cannot index. In
+     * presence of this flag we can skip consulting zookeeper before processing an indexing batch
+     * (if the flag proved wrong, indexing will fail and the cached state will get updated).
+     */
+    private final boolean cacheLikelyUpToDate;
+
+    /**
+     * The locks only need to be initialized once. When the metadata changes (and a new instance of
+     * {@link MetadataCacheEntry} created), the locks from the previous one must be reused.
+     */
+    private final ZeroAccessLocks locks;
+
+    /** Constructor for initial insertion of an entry into the cache. */
+    public MetadataCacheEntry() {
+      // Metadata suffix that will never be actually used in the Zero store
+      // Start with version 0 since when we use this entry to update the ZooKeeper node, it's
+      // the first update after that node got created.
+      this.metadataVersion = new ZeroMetadataVersion(METADATA_SUFFIX_CACHE_INITIAL_VALUE, 0);
+      this.zeroStoreShardMetadata = new ZeroStoreShardMetadata();
+      this.cacheLikelyUpToDate = false;
+      this.locks = new ZeroAccessLocks();
+    }
+
+    /**
+     * Used by {@link #updatedOf(boolean)} and {@link #updatedOf(ZeroMetadataVersion,
+     * ZeroStoreShardMetadata, boolean)} in this class. Visible outside the clas only for tests.
+     */
+    @VisibleForTesting
+    public MetadataCacheEntry(
+        ZeroMetadataVersion metadataVersion,
+        ZeroStoreShardMetadata zeroStoreShardMetadata,
+        boolean cacheLikelyUpToDate,
+        ZeroAccessLocks locks) {
+      this.metadataVersion = metadataVersion;
+      this.zeroStoreShardMetadata = zeroStoreShardMetadata;
+      this.cacheLikelyUpToDate = cacheLikelyUpToDate;
+      this.locks = locks;
+    }
+
+    public ZeroMetadataVersion getMetadataVersion() {
+      return metadataVersion;
+    }
+
+    public ZeroStoreShardMetadata getZeroShardMetadata() {
+      return zeroStoreShardMetadata;
+    }
+
+    public boolean isCacheLikelyUpToDate() {
+      return cacheLikelyUpToDate;
+    }
+
+    public ZeroAccessLocks getZeroAccessLocks() {
+      return locks;
+    }
+
+    private MetadataCacheEntry updatedOf(
+        ZeroMetadataVersion metadataVersion,
+        ZeroStoreShardMetadata zeroStoreShardMetadata,
+        boolean cacheLikelyUpToDate) {
+      return new MetadataCacheEntry(
+          metadataVersion, zeroStoreShardMetadata, cacheLikelyUpToDate, locks);
+    }
+
+    private MetadataCacheEntry updatedOf(boolean cacheLikelyUpToDate) {
+      return new MetadataCacheEntry(
+          metadataVersion, zeroStoreShardMetadata, cacheLikelyUpToDate, locks);
+    }
+
+    public String locksHeldToString() {
+      return String.format(
+          Locale.ROOT, "%s cacheLikelyUpToDate=%s", metadataVersion, cacheLikelyUpToDate);
+    }
+  }
+
+  /**
+   * Various stages a core of a Zero collection {@link DocCollection#isZeroIndex()} might go through
+   * during indexing and querying.
+   */
+  public enum ZeroCoreStage {
+    /** Necessary locks have been acquired and we have started to pull from the Zero store. */
+    PULL_STARTED,
+    /** Pull has ended and succeeded, and we are about to release the necessary locks. */
+    PULL_SUCCEEDED,
+    /** Pull has ended but failed, and we are about to release the necessary locks. */
+    PULL_FAILED,
+    /** Pull has ended but failed because of a corrupted core index. */
+    PULL_FAILED_WITH_CORRUPTION,
+    /** We have received an indexing batch but necessary locks have not been acquired yet. */
+    INDEXING_BATCH_RECEIVED,
+    /**
+     * We have passed the Zero store pull stage (if applicable) and are in sync with Zero store. Now
+     * we will proceed with local indexing.
+     */
+    LOCAL_INDEXING_STARTED,
+    /** Local indexing finished but has not been pushed to Zero store. */
+    LOCAL_INDEXING_FINISHED,
+    /** Necessary locks have been acquired and push to Zero store has started. */
+    PUSH_STARTED,
+    /** Files have been pushed to Zero store. */
+    FILE_PUSHED,
+    /** Zookeeper has been successfully updated with new metadata. */
+    ZK_UPDATE_FINISHED,
+    /** Local cache {@link #coresMetadata} has been successfully updated with new metadata. */
+    LOCAL_CACHE_UPDATE_FINISHED,
+    /**
+     * Push (either successful or failed) has ended and we are about to release the necessary locks.
+     */
+    PUSH_FINISHED,
+    /**
+     * Indexing batch (either successful or failed) has ended and we are about to release the
+     * necessary locks.
+     */
+    INDEXING_BATCH_FINISHED
+  }
+
+  /** Records the number of failed pull attempts and the last attempt time. */
+  @VisibleForTesting
+  public static class PullAttempts {
+    private int count;
+    private boolean pullAllowed;
+    private long lastAttemptTimeNs;
+
+    /**
+     * Checks the pull attempts count and either increments it, or throws a {@link SolrException} if
+     * the pull is not allowed until some delay elapses since the previous attempt. Sets
+     * lastAttemptTimeNs to the current time if the attempt is allowed.
+     */
+    synchronized PullAttempts checkAndIncrement(String coreName, ZeroConfig config) {
+      long timeNs = nanoTime();
+      long retryInNs = computeRemainingTimeNs(timeNs, config);
+      if (retryInNs > 0) {
+        throw new SolrException(
+            SolrException.ErrorCode.INVALID_STATE,
+            coreName
+                + " will not be pulled from the Zero store; canceling pull attempt due to "
+                + count
+                + " failed attempts, next attempt allowed in "
+                + TimeUnit.NANOSECONDS.toSeconds(retryInNs)
+                + " s");
+      }
+      count++;
+      pullAllowed = true;
+      // Resetting lastAttemptTimeNs here is not required, just in case onFailedAttempt() is not
+      // called.
+      lastAttemptTimeNs = timeNs;
+      return this;
+    }
+
+    synchronized PullAttempts onFailedAttempt() {
+      // Only reset lastAttemptTimeNs if the pull was allowed and failed.
+      if (pullAllowed) {
+        lastAttemptTimeNs = nanoTime();
+        pullAllowed = false;
+      }
+      return this;
+    }
+
+    private long computeRemainingTimeNs(long timeNs, ZeroConfig config) {
+      long remainingTimeNs;
+      if (count == 0) {
+        remainingTimeNs = 0;
+      } else {
+        long delayMs =
+            count >= config.getMaxFailedCorePullAttempts()
+                ?
+                // All pull attempts are exhausted. Still retry once periodically.
+                config.getAllAttemptsExhaustedRetryDelay()
+                :
+                // Allow the next pull attempt only if the last attempt was long enough ago.
+                count * config.getCorePullAttemptDelay();
+        remainingTimeNs = TimeUnit.MILLISECONDS.toNanos(delayMs) - (timeNs - lastAttemptTimeNs);
+      }
+      return remainingTimeNs;
+    }
+
+    synchronized boolean exhaustAllButOne(ZeroConfig config) {
+      if (count >= config.getMaxFailedCorePullAttempts()) {
+        return false;
+      }
+      count = config.getMaxFailedCorePullAttempts() - 1;
+      return true;
+    }
+
+    @VisibleForTesting
+    protected long nanoTime() {
+      return System.nanoTime();
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/MetadataComparisonResult.java b/solr/core/src/java/org/apache/solr/zero/metadata/MetadataComparisonResult.java
new file mode 100644
index 00000000000..252e8ba8f46
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/MetadataComparisonResult.java
@@ -0,0 +1,195 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.metadata;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.NoSuchFileException;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import org.apache.commons.codec.binary.Hex;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.zero.client.ZeroFile;
+
+/**
+ * Class capturing the differences between a local Solr core metadata {@link LocalCoreMetadata} and
+ * the Zero store metadata of the corresponding shard {@link ZeroStoreShardMetadata}.
+ */
+public class MetadataComparisonResult {
+
+  private final String metadataSuffix;
+
+  /** See {@link LocalCoreMetadata#generation} */
+  private final long localGeneration;
+
+  private final long distantGeneration;
+
+  /** Zero store files that need to be pulled */
+  private final Collection<ZeroFile.WithLocal> filesToPull;
+
+  /** Zero store files that need to be pushed */
+  private final Collection<ZeroFile.WithLocal> filesToPush;
+
+  /** Zero store files that need to be deleted */
+  private final Collection<ZeroFile.WithLocal> filesToDelete;
+
+  /**
+   * True if the local index contents conflict with contents to be pulled from the Zero store. If
+   * they conflict the core will be moved to a new index dir when pulling Zero store contents Two
+   * cases that result in a conflict: 1. local index is at higher generation number than Zero store
+   * generation number 2. index file with given name exists in both places but with a different size
+   * or checksum
+   */
+  private final boolean localConflictingWithZero;
+
+  /**
+   * Hash of the directory content used to make sure the content doesn't change as we proceed to
+   * pull new files from Zero store (if we need to pull new files from Zero store)
+   */
+  private final String directoryHash;
+
+  /**
+   * Result of a comparison of a local core and the Zero store version of the corresponding shard to
+   * decide how to align the two. Used when the goal is to pull newer content from the Zero store.
+   *
+   * @param localConflictingWithZero when true, some local files and Zero store files conflict.
+   *     After pulling, the core will be switched to a new directory.
+   */
+  public MetadataComparisonResult(
+      long localGeneration,
+      long distantGeneration,
+      String directoryHash,
+      boolean localConflictingWithZero,
+      Collection<ZeroFile.WithLocal> filesToPull) {
+    this.localGeneration = localGeneration;
+    this.distantGeneration = distantGeneration;
+    this.directoryHash = directoryHash;
+    this.filesToPush = Collections.emptySet();
+    this.filesToDelete = Collections.emptySet();
+    this.filesToPull = filesToPull;
+    this.localConflictingWithZero = localConflictingWithZero;
+    this.metadataSuffix = null;
+  }
+
+  /**
+   * Result of a comparison of a local core and the Zero store version of the corresponding shard to
+   * decide how to align the two. Used when the goal is to push local updates to the Zero store.
+   */
+  public MetadataComparisonResult(
+      long localGeneration,
+      long distantGeneration,
+      String directoryHash,
+      String metadataSuffix,
+      Collection<ZeroFile.WithLocal> filesToPush,
+      Collection<ZeroFile.WithLocal> filesToDelete) {
+    this.localGeneration = localGeneration;
+    this.distantGeneration = distantGeneration;
+    this.directoryHash = directoryHash;
+    this.metadataSuffix = metadataSuffix;
+    this.filesToPush = filesToPush;
+    this.filesToDelete = filesToDelete;
+    this.filesToPull = Collections.emptySet();
+    this.localConflictingWithZero = false;
+  }
+
+  public Collection<ZeroFile.WithLocal> getFilesToPush() {
+    return filesToPush;
+  }
+
+  public Collection<ZeroFile.WithLocal> getFilesToPull() {
+    return filesToPull;
+  }
+
+  public Collection<ZeroFile.WithLocal> getFilesToDelete() {
+    return filesToDelete;
+  }
+
+  public boolean isLocalConflictingWithZero() {
+    return localConflictingWithZero;
+  }
+
+  public long getLocalGeneration() {
+    return this.localGeneration;
+  }
+
+  public long getDistantGeneration() {
+    return this.distantGeneration;
+  }
+
+  public String getMetadataSuffix() {
+    return metadataSuffix;
+  }
+
+  /**
+   * Returns {@code true} if the contents of the directory passed into this method is identical to
+   * the contents of the directory of the Solr core of this instance, taken at instance creation
+   * time. If the directory hash was not computed at the instance creation time, then we throw an
+   * IllegalStateException indicating a programming error.
+   *
+   * <p>Passing in the Directory (expected to be the directory of the same core used during
+   * construction) because it seems safer than trying to get it again here...
+   *
+   * @throws IllegalStateException if this instance was not created with a computed directoryHash
+   */
+  public boolean isSameDirectoryContent(Directory coreDir)
+      throws NoSuchAlgorithmException, IOException {
+    if (directoryHash == null) {
+      throw new IllegalStateException("Directory hash was not computed for the given core");
+    }
+    return directoryHash.equals(getSolrDirectoryHash(coreDir, coreDir.listAll()));
+  }
+
+  /**
+   * Computes a hash of a Solr Directory in order to make sure the directory doesn't change as we
+   * pull content into it (if we need to pull content into it)
+   */
+  private String getSolrDirectoryHash(Directory coreDir, String[] filesNames)
+      throws NoSuchAlgorithmException, IOException {
+    MessageDigest digest =
+        MessageDigest.getInstance("sha1"); // not sure MD5 is available in Solr jars
+
+    // Computing the hash requires items to be submitted in the same order...
+    Arrays.sort(filesNames);
+
+    for (String fileName : filesNames) {
+      // .lock files come and go. Ignore them (we're closing the Index Writer before adding any
+      // pulled files to the Core)
+      if (!fileName.endsWith(".lock")) {
+        // Hash the file name and file size so we can tell if any file has changed (or files
+        // appeared or vanished)
+        digest.update(fileName.getBytes(StandardCharsets.UTF_8));
+        try {
+          digest.update(
+              Long.toString(coreDir.fileLength(fileName)).getBytes(StandardCharsets.UTF_8));
+        } catch (FileNotFoundException | NoSuchFileException fnf) {
+          // The file was deleted between the listAll() and the check, use an impossible size to not
+          // match a digest
+          // for which the file is completely present or completely absent (which will cause this
+          // hash to never match that directory again).
+          digest.update(Long.toString(-42).getBytes(StandardCharsets.UTF_8));
+        }
+      }
+    }
+
+    return new String(Hex.encodeHex(digest.digest()));
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataController.java b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataController.java
new file mode 100644
index 00000000000..a68540da405
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataController.java
@@ -0,0 +1,504 @@
+/*
+
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.solr.zero.metadata;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.solr.client.solrj.cloud.AlreadyExistsException;
+import org.apache.solr.client.solrj.cloud.BadVersionException;
+import org.apache.solr.client.solrj.cloud.SolrCloudManager;
+import org.apache.solr.client.solrj.cloud.VersionedData;
+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.ZkStateReader;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Class that manages metadata for Zero Index-based collections in Solr Cloud and ZooKeeper. */
+public class ZeroMetadataController {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * This is the initial value written to ZooKeeper for the {@link #SUFFIX_NODE_NAME} for a shard.
+   * See also {@link MetadataCacheManager#METADATA_SUFFIX_CACHE_INITIAL_VALUE} used as the initial
+   * value in the cache but not written to ZK.
+   */
+  public static final String METADATA_NODE_DEFAULT_VALUE = "DeFauLT";
+
+  public static final String SUFFIX_NODE_NAME = "metadataSuffix";
+
+  /**
+   * This file captures the shard metadata stored in the Zero store with each shard. When present
+   * locally (before being pushed to the Zero store), that file represents the metadata of the local
+   * core. Core vs shard, there is some naming ambiguity but eventually the two mean the same thing
+   * in the context of this file.
+   */
+  public static final String SHARD_METADATA_ZERO_FILENAME = "shard.metadata";
+
+  private static final String SEGMENTS_N_PREFIX = "segments_";
+
+  private final SolrCloudManager cloudManager;
+
+  public ZeroMetadataController(SolrCloudManager cloudManager) {
+    this.cloudManager = cloudManager;
+  }
+
+  public boolean hasDefaultNodeSuffix(ZeroMetadataVersion shardMetadataVersion) {
+    return METADATA_NODE_DEFAULT_VALUE.equals(shardMetadataVersion.getMetadataSuffix());
+  }
+
+  public boolean hasDefaultNodeSuffix(MetadataCacheManager.MetadataCacheEntry coreMetadata) {
+    return METADATA_NODE_DEFAULT_VALUE.equals(
+        coreMetadata.getMetadataVersion().getMetadataSuffix());
+  }
+
+  /**
+   * Creates a new metadataSuffix node if it doesn't exist for the shard and overwrites it if it
+   * does. This is only used when creating a shard, not when updating it (after indexing)
+   */
+  public void createMetadataNode(String collectionName, String shardName) {
+    ClusterState clusterState = cloudManager.getClusterStateProvider().getClusterState();
+    DocCollection collection = clusterState.getCollection(collectionName);
+    if (!collection.isZeroIndex()) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Can't create a metadataNode for collection "
+              + collectionName
+              + " that is not"
+              + " of type Zero for shard "
+              + shardName);
+    }
+
+    String metadataPath = getMetadataBasePath(collectionName, shardName) + "/" + SUFFIX_NODE_NAME;
+    Map<String, Object> nodeProps = new HashMap<>();
+    nodeProps.put(SUFFIX_NODE_NAME, METADATA_NODE_DEFAULT_VALUE);
+
+    createPersistentNode(metadataPath, Utils.toJSON(nodeProps));
+  }
+
+  /**
+   * If the update is successful, the returned {@link ZeroMetadataVersion} will contain the new
+   * version as well as the value of the data just written.
+   *
+   * @param value the value to be written to ZooKeeper
+   * @param version the ZooKeeper node version to conditionally update on. Except in tests, this is
+   *     never -1 (that allows the update to succeed regardless of existing node version)
+   */
+  public ZeroMetadataVersion updateMetadataValueWithVersion(
+      String collectionName, String shardName, String value, int version) {
+    String metadataPath = getMetadataBasePath(collectionName, shardName) + "/" + SUFFIX_NODE_NAME;
+    try {
+      Map<String, Object> nodeProps = new HashMap<>();
+      nodeProps.put(SUFFIX_NODE_NAME, value);
+
+      VersionedData data =
+          cloudManager
+              .getDistribStateManager()
+              .setAndGetResult(metadataPath, Utils.toJSON(nodeProps), version);
+      Map<?, ?> nodeUserData = (Map<?, ?>) Utils.fromJSON(data.getData());
+      String metadataSuffix = (String) nodeUserData.get(ZeroMetadataController.SUFFIX_NODE_NAME);
+      return new ZeroMetadataVersion(metadataSuffix, data.getVersion());
+    } catch (BadVersionException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error updating path: " + metadataPath + " due to mismatching versions",
+          e);
+    } catch (NoSuchElementException | KeeperException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error updating path: " + metadataPath + " in ZooKeeper",
+          e);
+    }
+  }
+
+  /** Reads the {@link ZeroMetadataVersion} for the shard from ZooKeeper. */
+  public ZeroMetadataVersion readMetadataValue(String collectionName, String shardName) {
+    String metadataPath = getMetadataBasePath(collectionName, shardName) + "/" + SUFFIX_NODE_NAME;
+    try {
+      VersionedData data = cloudManager.getDistribStateManager().getData(metadataPath, null);
+      Map<?, ?> nodeUserData = (Map<?, ?>) Utils.fromJSON(data.getData());
+      String metadataSuffix = (String) nodeUserData.get(ZeroMetadataController.SUFFIX_NODE_NAME);
+      return new ZeroMetadataVersion(metadataSuffix, data.getVersion());
+    } catch (IOException | NoSuchElementException | KeeperException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error reading data from path: " + metadataPath + " in ZooKeeper",
+          e);
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error reading data from path: " + metadataPath + " in ZooKeeper",
+          e);
+    }
+  }
+
+  /** Removes the metadata node on zookeeper for the given collection and shard name. */
+  public void cleanUpMetadataNodes(String collectionName, String shardName) {
+    String metadataPath = getMetadataBasePath(collectionName, shardName) + "/" + SUFFIX_NODE_NAME;
+    try {
+      cloudManager.getDistribStateManager().removeRecursively(metadataPath, true, true);
+    } catch (Exception e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error deleting path " + metadataPath + " in Zookeeper",
+          e);
+    }
+  }
+
+  private void createPersistentNode(String path, byte[] data) {
+    try {
+      // if path already exists, overwrite it
+      cloudManager.getDistribStateManager().makePath(path, data, CreateMode.PERSISTENT, false);
+    } catch (InterruptedException e) {
+      Thread.interrupted();
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR, "Error creating path " + path + " in Zookeeper", e);
+    } catch (IOException | AlreadyExistsException | KeeperException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR, "Error creating path " + path + " in Zookeeper", e);
+    }
+  }
+
+  protected String getMetadataBasePath(String collectionName, String shardName) {
+    return ZkStateReader.COLLECTIONS_ZKNODE
+        + "/"
+        + collectionName
+        + "/"
+        + ZkStateReader.SHARD_LEADERS_ZKNODE
+        + "/"
+        + shardName;
+  }
+
+  public MetadataComparisonResult diffMetadataForPush(
+      LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantMetadata, String suffix)
+      throws Exception {
+    Directory coreDir = localMetadata.getCoreDirectory();
+    try {
+      checkSegmentFilesIntegrity(localMetadata, distantMetadata);
+      return diffMetadataForPush(coreDir, localMetadata, distantMetadata, suffix);
+    } finally {
+      localMetadata.releaseCoreDirectory(coreDir);
+    }
+  }
+
+  public MetadataComparisonResult diffMetadataforPull(
+      LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantMetadata) throws Exception {
+    Directory coreDir = localMetadata.getCoreDirectory();
+    try {
+      checkSegmentFilesIntegrity(localMetadata, distantMetadata);
+      return diffMetadataForPull(coreDir, localMetadata, distantMetadata);
+    } finally {
+      localMetadata.releaseCoreDirectory(coreDir);
+    }
+  }
+
+  private MetadataComparisonResult diffMetadataForPull(
+      Directory coreDir, LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantMetadata) {
+
+    boolean localConflictingWithZero = false;
+
+    Map<String, ZeroFile.WithLocal> zeroFilesMissingLocally =
+        getZeroFilesMissingLocally(localMetadata, distantMetadata);
+    Map<String, ZeroFile.WithLocal> latestCommitFilesAlreadyOnZeroButDifferent =
+        getLatestCommitFilesAlreadyOnZeroButDifferent(coreDir, localMetadata, distantMetadata);
+    if (!latestCommitFilesAlreadyOnZeroButDifferent.isEmpty()) {
+      localConflictingWithZero = true;
+      zeroFilesMissingLocally = distantMetadata.getZeroFilesAsMap();
+    }
+
+    Map<String, ZeroFile.WithLocal> filesOnBothSideButNotInLatestCommit =
+        getFilesOnBothSideButNotInLatestCommit(localMetadata, zeroFilesMissingLocally);
+    if (!filesOnBothSideButNotInLatestCommit.isEmpty()) {
+      // We are resolving metadata for a PULL, it means we expect the local core to be behind or
+      // at the Zero version. We therefore do not expect local files not in the commit point that
+      // are still in the Zero store commit point. It might mean the local core did some indexing
+      // and merged segments then dropped from the commit point.
+      //
+      // This can happen if the core was the leader, indexed locally but then failed to push to the
+      // Zero store, and is now pulling again after another replica did a push. In these case we do
+      // want to align on the Zero store version.
+      localConflictingWithZero = true;
+      zeroFilesMissingLocally = distantMetadata.getZeroFilesAsMap();
+
+      filesOnBothSideButNotInLatestCommit
+          .values()
+          .forEach(
+              bf -> {
+                if (log.isInfoEnabled()) {
+                  log.info(
+                      "File exists locally outside of current commit point. collectionName={} shardName={}"
+                          + " coreName={}  solrFileName={} zeroFileName={}",
+                      localMetadata.getCollectionName(),
+                      localMetadata.getShardName(),
+                      localMetadata.getCoreName(),
+                      bf.getSolrFileName(),
+                      bf.getZeroFileName());
+                }
+              });
+    }
+
+    // If local index generation is higher than Zero's we will download to new directory because
+    // if both segments_N files are present in same directory Solr will open the higher one (and
+    // we really want the lower one from the Zero store as we're currently pulling content from
+    // there).
+    if (!distantMetadata.getZeroFiles().isEmpty()
+        && localMetadata.getGeneration() > distantMetadata.getGeneration()) {
+      localConflictingWithZero = true;
+      zeroFilesMissingLocally = distantMetadata.getZeroFilesAsMap();
+      if (log.isInfoEnabled()) {
+        log.info(
+            "local generation higher than Zero store.  coreName={} localGeneration={} zeroGeneration={}",
+            localMetadata.getCoreName(),
+            localMetadata.getGeneration(),
+            distantMetadata.getGeneration());
+      }
+    }
+
+    return new MetadataComparisonResult(
+        localMetadata.getGeneration(),
+        distantMetadata.getGeneration(),
+        localMetadata.getDirectoryHash(),
+        localConflictingWithZero,
+        zeroFilesMissingLocally.values());
+  }
+
+  private MetadataComparisonResult diffMetadataForPush(
+      Directory coreDir,
+      LocalCoreMetadata localMetadata,
+      ZeroStoreShardMetadata distantMetadata,
+      String newSuffix) {
+
+    Collection<ZeroFile.WithLocal> filesToPush =
+        getLocalFilesMissingOnZero(localMetadata, distantMetadata).stream()
+            .map(
+                f ->
+                    getNewZeroLocalFile(
+                        coreDir,
+                        f,
+                        localMetadata.getCollectionName(),
+                        localMetadata.getShardName(),
+                        newSuffix))
+            .collect(Collectors.toSet());
+    Collection<ZeroFile.WithLocal> filesToDelete =
+        getZeroFilesMissingLocally(localMetadata, distantMetadata).values();
+    return new MetadataComparisonResult(
+        localMetadata.getGeneration(),
+        distantMetadata.getGeneration(),
+        localMetadata.getDirectoryHash(),
+        newSuffix,
+        filesToPush,
+        filesToDelete);
+  }
+
+  /**
+   * @return intersection(all local files, distant files - latest committed files)
+   */
+  private Map<String, ZeroFile.WithLocal> getFilesOnBothSideButNotInLatestCommit(
+      LocalCoreMetadata localMetadata, Map<String, ZeroFile.WithLocal> zeroFilesMissingLocally) {
+
+    Map<String, ZeroFile.WithLocal> filesOnBothSideButNotInLatestCommit =
+        new HashMap<>(zeroFilesMissingLocally);
+    filesOnBothSideButNotInLatestCommit.keySet().retainAll(localMetadata.getAllFiles());
+    return filesOnBothSideButNotInLatestCommit;
+  }
+
+  /**
+   * @return intersection(distant files, latest committed files).filter(size or checksum different)
+   */
+  private Map<String, ZeroFile.WithLocal> getLatestCommitFilesAlreadyOnZeroButDifferent(
+      Directory coreDir, LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantZeroFiles) {
+    Map<String, ZeroFile.WithLocal> latestCommitFilesAlreadyOnZero =
+        distantZeroFiles.getZeroFilesAsMap();
+    latestCommitFilesAlreadyOnZero.keySet().retainAll(localMetadata.getLatestCommitFiles());
+    return latestCommitFilesAlreadyOnZero.entrySet().stream()
+        .filter(
+            f ->
+                areDistantAndLocalFileDifferent(
+                    coreDir, f.getValue(), f.getKey(), localMetadata.getCoreName()))
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+  }
+
+  /**
+   * @return distant files - latest committed files
+   */
+  private Map<String, ZeroFile.WithLocal> getZeroFilesMissingLocally(
+      LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantZeroFiles) {
+
+    Map<String, ZeroFile.WithLocal> zeroFilesMissingLocally = distantZeroFiles.getZeroFilesAsMap();
+    zeroFilesMissingLocally.keySet().removeAll(localMetadata.getLatestCommitFiles());
+    return zeroFilesMissingLocally;
+  }
+
+  /**
+   * @return latest committed files - distant files
+   */
+  private Set<String> getLocalFilesMissingOnZero(
+      LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantZeroFiles) {
+    Set<String> localFileNamesMissingOnZero = new HashSet<>(localMetadata.getLatestCommitFiles());
+    localFileNamesMissingOnZero.removeAll(distantZeroFiles.getZeroFileNames());
+    return localFileNamesMissingOnZero;
+  }
+
+  private void checkSegmentFilesIntegrity(
+      LocalCoreMetadata localMetadata, ZeroStoreShardMetadata distantMetadata)
+      throws SolrException {
+    List<ZeroFile.WithLocal> segmentFiles =
+        distantMetadata.getZeroFiles().stream()
+            .filter(this::isSegmentsNFilename)
+            .limit(2)
+            .collect(Collectors.toList());
+    if (segmentFiles.size() == 2) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          String.format(
+              Locale.ROOT,
+              "Zero store for collection %s and shard %s has conflicting files %s and %s",
+              localMetadata.getCollectionName(),
+              localMetadata.getShardName(),
+              segmentFiles.get(0),
+              segmentFiles.get(1)));
+    } else if (segmentFiles.isEmpty() && !distantMetadata.getZeroFiles().isEmpty()) {
+      // TODO - for now just log and propagate the error up, this class shouldn't do corruption
+      // checking now
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR, "Zero store has missing segments file");
+    }
+  }
+
+  protected ZeroFile.WithLocal getNewZeroLocalFile(
+      Directory coreDir, String fileName, String collectionName, String shardName, String suffix) {
+    long fileSize = 0L;
+    long fileChecksum = 0L;
+    try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
+      fileSize = indexInput.length();
+      fileChecksum = CodecUtil.retrieveChecksum(indexInput);
+    } catch (IOException e) {
+      log.warn("Failed to get file metadata {}", fileName, e);
+      // IO error when opening the file (probably not found, for example it could be wiped
+      // in CorePushPull if a corruption was detected).
+      // Consider the file empty so that it is pulled again from the Zero store.
+    }
+    return new ZeroFile.WithLocal(
+        collectionName,
+        shardName,
+        fileName,
+        buildFileName(fileName, suffix),
+        fileSize,
+        fileChecksum);
+  }
+
+  protected boolean areDistantAndLocalFileDifferent(
+      Directory coreDir, ZeroFile.WithLocal distantFile, String localFileName, String coreName) {
+    long localFileSize = 0L;
+    long localFileChecksum = 0L;
+    try (final IndexInput indexInput = coreDir.openInput(localFileName, IOContext.READONCE)) {
+      localFileSize = indexInput.length();
+      localFileChecksum = CodecUtil.retrieveChecksum(indexInput);
+    } catch (IOException e) {
+      log.warn("Error getting file metadata {}", localFileName, e);
+      // IO error when opening the file (probably not found, for example it could be wiped
+      // in CorePushPull if a corruption was detected).
+      // Consider the file empty so that it is pulled again from the Zero store.
+    }
+
+    boolean different =
+        (localFileSize != distantFile.getFileSize()
+            || localFileChecksum != distantFile.getChecksum());
+
+    if (different) {
+      String message =
+          String.format(
+              Locale.ROOT,
+              "Size/Checksum conflicts. collectionName=%s shardName=%s coreName=%s fileName=%s zeroStoreFileName=%s"
+                  + " localSize=%s zeroStoreSize=%s localChecksum=%s zeroStoreChecksum=%s",
+              distantFile.getCollectionName(),
+              distantFile.getShardName(),
+              coreName,
+              localFileName,
+              distantFile.getZeroFileName(),
+              localFileSize,
+              distantFile.getFileSize(),
+              localFileChecksum,
+              distantFile.getChecksum());
+      log.info(message);
+    }
+    return different;
+  }
+
+  /** Identify the segments_N file in local Zero files. */
+  public boolean isSegmentsNFilename(ZeroFile.WithLocal file) {
+    return file.getSolrFileName().startsWith(SEGMENTS_N_PREFIX);
+  }
+
+  public ZeroFile.WithLocal newShardMetadataZeroFile(
+      String collectionName, String shardName, String suffix) {
+    return new ZeroFile.WithLocal(
+        collectionName,
+        shardName,
+        SHARD_METADATA_ZERO_FILENAME,
+        buildFileName(ZeroMetadataController.SHARD_METADATA_ZERO_FILENAME, suffix),
+        ZeroStoreShardMetadata.FAKE_CORE_METADATA_LENGTH,
+        ZeroStoreShardMetadata.FAKE_CORE_METADATA_CHECKSUM);
+  }
+
+  /**
+   * This method implements the naming convention for files on the Zero store.
+   *
+   * <p>There really are two naming conventions, one for the {@link #SHARD_METADATA_ZERO_FILENAME},
+   * and one for all other files. For simplicity the two conventions are identical.
+   *
+   * <ol>
+   *   <li>{@link #SHARD_METADATA_ZERO_FILENAME} (<code>shard.metadata</code>) is named on the Zero
+   *       store <code>shard.metadata.randomSuffix</code>, the random suffix is stored in Zookeeper
+   *       and that's how the current <code>shard.metadata</code> file is identified (there might be
+   *       older such files, they will have a different suffix from the one stored in Zookeeper)
+   *   <li>All other files (composing a Solr core) can have any name, as long as it's unique so no
+   *       chance of overwriting an existing file on the Zero store. The mapping from Zero store
+   *       file name to local file name is maintained in the <code>shard.metadata</code> file so
+   *       there's no need for a specific naming convention (same convention used nonetheless).
+   * </ol>
+   */
+  public static String buildFileName(String prefix, String suffix) {
+    return prefix + "." + suffix;
+  }
+
+  public ZeroFile.ToDelete newShardMetadataZeroFileToDelete(
+      String collectionName, String shardName, String metadataSuffix) {
+    return new ZeroFile.ToDelete(
+        collectionName, shardName, buildFileName(SHARD_METADATA_ZERO_FILENAME, metadataSuffix));
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataVersion.java b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataVersion.java
new file mode 100644
index 00000000000..f0d26e71ab9
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroMetadataVersion.java
@@ -0,0 +1,67 @@
+/*
+
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.solr.zero.metadata;
+
+import org.apache.solr.common.cloud.DocCollection;
+
+/**
+ * This represents the metadata suffix and the corresponding Zookeeper version of the node for a
+ * shard of a Zero collection {@link DocCollection#isZeroIndex()}
+ */
+public class ZeroMetadataVersion {
+  /** The metadataSuffix for the shard stored in the Zero store. */
+  private final String metadataSuffix;
+
+  /** version of zookeeper node maintaining the metadata */
+  private final int version;
+
+  public ZeroMetadataVersion(String metadataSuffix, int version) {
+    assert metadataSuffix != null;
+    this.metadataSuffix = metadataSuffix;
+    this.version = version;
+  }
+
+  public String getMetadataSuffix() {
+    return metadataSuffix;
+  }
+
+  public int getVersion() {
+    return version;
+  }
+
+  @Override
+  public String toString() {
+    return "metadataSuffix=" + metadataSuffix + " version=" + version;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || obj.getClass() != this.getClass()) {
+      return false;
+    }
+
+    final ZeroMetadataVersion other = (ZeroMetadataVersion) obj;
+    return this.version == other.version && this.metadataSuffix.equals(other.metadataSuffix);
+  }
+
+  @Override
+  public int hashCode() {
+    // This object is never inserted into a collection
+    return 0;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/ZeroStoreShardMetadata.java b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroStoreShardMetadata.java
new file mode 100644
index 00000000000..caecea1fc77
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/ZeroStoreShardMetadata.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.metadata;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.process.CorePusher;
+import org.apache.solr.zero.util.ToFromJson;
+
+/**
+ * Object defining metadata stored in Zero store for a Zero Collection shard and its builders
+ * (storage in Zero store is a single copy per shard and not per core or replica as usual in
+ * SolrCloud). This metadata includes all actual segment files as well as the segments_N file of the
+ * commit point.
+ *
+ * <p>This object is serialized to/from Json and stored in the Zero store as a file named <code>
+ * shard.metadata</code> (with a random suffix).
+ *
+ * <p>Although in the Zero store storage is per shard, each Solr node manages this file for its
+ * specific core (replica) of the shard. The update strategy via unique filenames on the Zero store
+ * and a tiebreaker using zookeeper guarantees consistency at the shard level (a replica doesn't
+ * overwrite an update to the shard in the Zero store done by another replica on another node).
+ */
+public class ZeroStoreShardMetadata {
+
+  /** shard.metadata file doesn't have its length or checksum compared so using fake values */
+  public static final long FAKE_CORE_METADATA_LENGTH = -10;
+
+  public static final long FAKE_CORE_METADATA_CHECKSUM = -10;
+  public static final long UNDEFINED_GENERATION = -1L;
+
+  /**
+   * Generation number of index represented by this metadata. Generation numbers across replicas
+   * don't carry any meaning since each replica can be doing its own indexing. We are only using
+   * this generation for optimizations in the context of a single replica. It should not be used
+   * beyond following stated purposes. If its mere existence is popping it into consideration for
+   * solving unintended use cases, then we can remove it and implement the suggested alternatives
+   * for following usages. 1. identify a scenario where local index generation number is higher than
+   * what we have in Zero store. In that scenario we would switch index to a new directory when
+   * pulling contents from Zero store. Because in the presence of higher generation number locally,
+   * Zero store contents cannot establish their legitimacy. Storing this number is just an
+   * optimization. We can always infer that number from segment_N file and get rid of this usage.
+   * {@link ZeroMetadataController#diffMetadataforPull(LocalCoreMetadata, ZeroStoreShardMetadata)}
+   * ()} 2. {@link CorePusher#endToEndPushCoreToZeroStore()} piggy backs on cached
+   * ZeroStoreShardMetadata's generation number instead of carrying a separate lastGenerationPushed
+   * property in the cache. This is also an optimization, we can always add lastGenerationPushed
+   * property to cache and get rid of this usage.
+   */
+  private final long generation;
+
+  /**
+   * The array of files that constitute the current commit point of the core (as known by the Zero
+   * store). This array is not ordered! There are no duplicate entries in it either
+   */
+  private final Set<ZeroFile.WithLocal> zeroFiles;
+
+  /**
+   * Files marked for delete but not yet removed from the Zero store. Files are only actually
+   * deleted after a new version of shard.metadata has been successfully written to the Zero store
+   * and "validated" by the ZooKeeper metadataSuffix update.
+   */
+  private final Set<ZeroFile.ToDelete> zeroFilesToDelete;
+
+  /** For Jackson deserialization. See {@link ToFromJson}. */
+  public ZeroStoreShardMetadata() {
+    this(UNDEFINED_GENERATION);
+  }
+
+  /** Always builds non "isCorrupt" and non "isDeleted" metadata. */
+  public ZeroStoreShardMetadata(
+      Set<ZeroFile.WithLocal> zeroFiles,
+      Set<ZeroFile.ToDelete> zeroFilesToDelete,
+      long generation) {
+    this.zeroFiles = zeroFiles;
+    this.zeroFilesToDelete = zeroFilesToDelete;
+    this.generation = generation;
+  }
+
+  @VisibleForTesting
+  public ZeroStoreShardMetadata(long generation) {
+    this.zeroFiles = new HashSet<>();
+    this.zeroFilesToDelete = new HashSet<>();
+    this.generation = generation;
+  }
+
+  /** Adds a file to the set of "active" files listed in the metadata */
+  public void addFile(ZeroFile.WithLocal f) {
+    this.zeroFiles.add(f);
+  }
+
+  /** Removes a file from the set of "active" files listed in the metadata */
+  public void removeFile(ZeroFile.WithLocal f) {
+    boolean removed = this.zeroFiles.remove(f);
+    assert removed; // If we remove things that are not there, likely a bug in our code
+  }
+
+  /**
+   * Adds a file to the set of files to delete listed in the metadata
+   *
+   * <p>This method should always be called with {@link #removeFile(ZeroFile.WithLocal)} above
+   * (except when adding for delete the current shard.metadata file).
+   */
+  public void addFileToDelete(ZeroFile.ToDelete f) {
+    this.zeroFilesToDelete.add(f);
+  }
+
+  public long getGeneration() {
+    return this.generation;
+  }
+
+  public Set<ZeroFile.WithLocal> getZeroFiles() {
+    return zeroFiles;
+  }
+
+  public Set<ZeroFile.ToDelete> getZeroFilesToDelete() {
+    return zeroFilesToDelete;
+  }
+
+  public Map<String, ZeroFile.WithLocal> getZeroFilesAsMap() {
+    return zeroFiles.stream()
+        .collect(Collectors.toMap(ZeroFile.WithLocal::getSolrFileName, Function.identity()));
+  }
+
+  public Set<String> getZeroFileNames() {
+    return zeroFiles.stream().map(ZeroFile.WithLocal::getSolrFileName).collect(Collectors.toSet());
+  }
+
+  /** Removes a file from the set of "deleted" files listed in the metadata */
+  public void removeFilesFromDeleted(Set<ZeroFile.ToDelete> files) {
+    int originalSize = this.zeroFilesToDelete.size();
+    boolean removed = this.zeroFilesToDelete.removeAll(files);
+    int totalRemoved = originalSize - this.zeroFilesToDelete.size();
+
+    // If we remove things that are not there, likely a bug in our code
+    assert removed && (totalRemoved == files.size());
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof ZeroStoreShardMetadata)) return false;
+
+    ZeroStoreShardMetadata that = (ZeroStoreShardMetadata) o;
+
+    if (this.generation != that.generation) return false;
+    if (!this.zeroFiles.equals(that.zeroFiles)) return false;
+    return this.zeroFilesToDelete.equals(that.zeroFilesToDelete);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(generation, this.zeroFiles, this.zeroFilesToDelete);
+  }
+
+  public static String generateMetadataSuffix() {
+    return UUID.randomUUID().toString();
+  }
+
+  public String toJson() throws Exception {
+    ToFromJson<ZeroStoreShardMetadata> converter = new ToFromJson<>();
+    return converter.toJson(this);
+  }
+
+  public static ZeroStoreShardMetadata fromJson(String json) throws Exception {
+    ToFromJson<ZeroStoreShardMetadata> converter = new ToFromJson<>();
+    return converter.fromJson(json, ZeroStoreShardMetadata.class);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/metadata/package-info.java b/solr/core/src/java/org/apache/solr/zero/metadata/package-info.java
new file mode 100644
index 00000000000..623c6d818a2
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/metadata/package-info.java
@@ -0,0 +1,19 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/** APIs for handling Zero store metadata */
+package org.apache.solr.zero.metadata;
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CollectionDeletionTask.java b/solr/core/src/java/org/apache/solr/zero/process/CollectionDeletionTask.java
new file mode 100644
index 00000000000..00c709893a6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CollectionDeletionTask.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.NoSuchFileException;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class CollectionDeletionTask extends FilesDeletionTask {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public CollectionDeletionTask(
+      ZeroStoreClient zeroStoreClient,
+      String collectionName,
+      boolean allowRetry,
+      int maxRetryAttempt) {
+    super(
+        zeroStoreClient,
+        collectionName,
+        zeroStoreClient.listCollectionZeroFiles(collectionName),
+        allowRetry,
+        maxRetryAttempt);
+  }
+
+  @Override
+  public String getActionName() {
+    return "DELETE_COLLECTION_FILES";
+  }
+
+  @Override
+  public DeleterTask.Result call() {
+    DeleterTask.Result result = super.call();
+    if (result.isSuccess()) {
+      try {
+        zeroStoreClient.deleteCollectionDirectory(collectionName);
+      } catch (NoSuchFileException ex) {
+        if (log.isWarnEnabled())
+          log.warn(
+              "Could not delete Zero store directory for collection={} as it does not exists",
+              collectionName);
+      } catch (IOException ex) {
+        result.updateSuccess(false);
+        if (log.isWarnEnabled())
+          log.warn(
+              "Could not delete Zero store directory for collection={} after all files have been deleted",
+              collectionName);
+      }
+    }
+    return result;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CorePullStatus.java b/solr/core/src/java/org/apache/solr/zero/process/CorePullStatus.java
new file mode 100644
index 00000000000..557ca3d1a1f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CorePullStatus.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.process;
+
+/** Enumerates all possible outcomes of core pull tasks */
+public enum CorePullStatus {
+  /** Core pulled successfully from Zero store */
+  SUCCESS(true, false),
+  /** There was no need to pull the core from the Zero store */
+  NOT_NEEDED(true, false),
+  /** The pull task was merged with another duplicate task in the queue (deduplicated) */
+  DUPLICATE_REQUEST(true, false),
+
+  /** Core failed to pull from the Zero store, unknown cause */
+  FAILURE(false, true),
+  /** Pulling process was interrupted */
+  INTERRUPTED(false, true),
+  /**
+   * Core not pulled from Zero store because corresponding shard.metadata file was not found there
+   */
+  ZERO_METADATA_MISSING(false, false),
+  /** The pull lock was not acquired immediately, task can be retried */
+  FAILED_TO_ACQUIRE_LOCK(false, true);
+
+  private final boolean isTransientError;
+  private final boolean isSuccess;
+
+  CorePullStatus(boolean isSuccess, boolean isTransientError) {
+    assert !(isSuccess && isTransientError);
+    this.isSuccess = isSuccess;
+    this.isTransientError = isTransientError;
+  }
+
+  /**
+   * Only defined for a status when ({@link #isSuccess()} returns {@code false}).
+   *
+   * @return {@code true} when it makes sense to retry (a few times) running the same task again,
+   *     i.e. when the cause of the failure might disappear.
+   */
+  public boolean isTransientError() {
+    return isTransientError;
+  }
+
+  /**
+   * @return {@code true} if content was successfully synced from Zero store OR if no sync was
+   *     required.
+   */
+  public boolean isSuccess() {
+    return isSuccess;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CorePuller.java b/solr/core/src/java/org/apache/solr/zero/process/CorePuller.java
new file mode 100644
index 00000000000..456da6c3214
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CorePuller.java
@@ -0,0 +1,800 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.reflect.Array;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.BiConsumer;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.apache.solr.zero.exception.CorruptedCoreException;
+import org.apache.solr.zero.exception.ZeroException;
+import org.apache.solr.zero.exception.ZeroLockException;
+import org.apache.solr.zero.metadata.LocalCoreMetadata;
+import org.apache.solr.zero.metadata.MetadataCacheManager;
+import org.apache.solr.zero.metadata.MetadataComparisonResult;
+import org.apache.solr.zero.metadata.ZeroMetadataController;
+import org.apache.solr.zero.metadata.ZeroMetadataVersion;
+import org.apache.solr.zero.metadata.ZeroStoreShardMetadata;
+import org.apache.solr.zero.util.DeduplicatingList;
+import org.apache.solr.zero.util.FileTransferCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Code for pulling updates for a specific core from the Zero store. */
+public class CorePuller implements DeduplicatingList.Deduplicatable<String> {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final ZeroStoreClient zeroStoreClient;
+  private final SolrCore solrCore;
+  private final MetadataCacheManager metadataCacheManager;
+  private final ZeroMetadataController metadataController;
+  private final Instant queuedTime;
+  private final AtomicInteger currentPullAttempt;
+  private final int maxPullAttempts;
+  private final Instant lastPullAttemptTime;
+
+  /**
+   * A callback method for end pull notification for async pulls (queries). Will be a no-op in
+   * production but some tests do set it. Note no notification will be received when the pull throws
+   * an exception rather than return a failure status code, which is the case for most though not
+   * all errors.
+   */
+  private final BiConsumer<CorePuller, CorePullStatus> pullEndNotification;
+
+  CorePuller(
+      SolrCore solrCore,
+      ZeroStoreClient zeroStoreClient,
+      MetadataCacheManager metadataCacheManager,
+      ZeroMetadataController metadataController,
+      Instant queuedTime,
+      int currentPullAttempt,
+      int maxPullAttempts,
+      Instant lastPullAttemptTime,
+      BiConsumer<CorePuller, CorePullStatus> pullEndNotification) {
+    this.zeroStoreClient = zeroStoreClient;
+    this.metadataCacheManager = metadataCacheManager;
+    this.metadataController = metadataController;
+    this.solrCore = solrCore;
+    this.queuedTime = queuedTime;
+    this.currentPullAttempt = new AtomicInteger(currentPullAttempt);
+    this.lastPullAttemptTime = lastPullAttemptTime;
+    this.maxPullAttempts = maxPullAttempts;
+    this.pullEndNotification = pullEndNotification != null ? pullEndNotification : (u, v) -> {};
+  }
+
+  public CorePuller(
+      SolrCore solrCore,
+      ZeroStoreClient zeroStoreClient,
+      MetadataCacheManager metadataCacheManager,
+      ZeroMetadataController metadataController,
+      int maxPullAttempts,
+      BiConsumer<CorePuller, CorePullStatus> pullEndNotification) {
+    this(
+        solrCore,
+        zeroStoreClient,
+        metadataCacheManager,
+        metadataController,
+        Instant.now(),
+        0,
+        maxPullAttempts,
+        Instant.EPOCH,
+        pullEndNotification);
+  }
+
+  /** Needed for the {@link CorePuller} to be used in a {@link DeduplicatingList}. */
+  @Override
+  public String getDedupeKey() {
+    return this.getCoreName();
+  }
+
+  public static DeduplicatingList<String, CorePuller> getDeduplicatingList(int maxCapacity) {
+    return new DeduplicatingList<>(maxCapacity, new CorePuller.CorePullerMerger());
+  }
+
+  public void incCurrentPullAttempts() {
+    currentPullAttempt.incrementAndGet();
+  }
+
+  public int getCurrentPullAttempt() {
+    return currentPullAttempt.get();
+  }
+
+  public int getMaxPullAttempts() {
+    return maxPullAttempts;
+  }
+
+  private CorePuller newUpdatedPuller(
+      Instant queuedTime, int attempts, int maxAttempts, Instant lastAttemptTime) {
+
+    return new CorePuller(
+        solrCore,
+        zeroStoreClient,
+        metadataCacheManager,
+        metadataController,
+        queuedTime,
+        attempts,
+        maxAttempts,
+        lastAttemptTime,
+        pullEndNotification);
+  }
+
+  /**
+   * This method is only used in this class for now because the "re-enqueue with delay"
+   * implementation is imperfect. Longer term, such a re-enqueue should be handled outside this
+   * class.
+   */
+  public Instant getLastPullAttemptTime() {
+    return lastPullAttemptTime;
+  }
+
+  public Instant getQueuedTime() {
+    return this.queuedTime;
+  }
+
+  /**
+   * Manages multiple retries calling {@link #lockAndPullCore} to pull a core from Zero store when
+   * transient failures are encountered.
+   */
+  @SuppressWarnings("try")
+  public CorePullStatus pullCoreWithRetries(
+      boolean isLeaderPulling, long secondsToWaitPullLock, long retryDelay) throws ZeroException {
+    MDCLoggingContext.setCore(getCore());
+    try {
+      if (skipPullIfShardAlreadySynced(isLeaderPulling)) return CorePullStatus.NOT_NEEDED;
+
+      // Initialization needed if we don't pass in the loop even once (tests...). Then
+      // CorePullStatus.FAILURE
+      // being a transient error, the log will be correct that all retries exhausted.
+      CorePullStatus pullStatus = CorePullStatus.FAILURE;
+
+      while (getCurrentPullAttempt() < maxPullAttempts) {
+        // Sleep a bit if we're retrying, to let the previous error disappear naturally
+        try {
+          sleepIfNeeded(retryDelay);
+        } catch (InterruptedException ie) {
+          // No retry if the wait is interrupted (no timeout here)
+          Thread.currentThread().interrupt();
+          String message = "Wait done before core pull got interrupted, likely shutdown?";
+          log.error(message, ie);
+          throw new ZeroException(message);
+        }
+        // Do the actual pull
+        pullStatus = lockAndPullCore(isLeaderPulling, secondsToWaitPullLock);
+        // Check the return code and throw exceptions for most errors
+        incCurrentPullAttempts();
+        if (pullStatus.isSuccess()) {
+          if (log.isInfoEnabled()) {
+            log.info(
+                String.format(
+                    Locale.ROOT,
+                    "Pulling core for collection %s shard %s succeeded. Last status=%s attempts=%s",
+                    getCollectionName(),
+                    getShardName(),
+                    pullStatus,
+                    getCurrentPullAttempt()));
+          }
+          notifyPullEnd(pullStatus);
+          return pullStatus;
+        } else if (!pullStatus.isTransientError()) {
+          break;
+        }
+        if (log.isInfoEnabled()) {
+          log.info(
+              String.format(
+                  Locale.ROOT,
+                  "Pulling core for collection %s shard %s failed with transient error. Retrying. Last status=%s attempts=%s",
+                  getCollectionName(),
+                  getShardName(),
+                  pullStatus,
+                  getCurrentPullAttempt()));
+        }
+      }
+      // Give up if unrecoverable or attempts exhausted
+      if (log.isWarnEnabled()) {
+        log.warn(
+            String.format(
+                Locale.ROOT,
+                "Pulling core for collection %s shard %s failed %s. Giving up. Last status=%s attempts=%s",
+                getCollectionName(),
+                getShardName(),
+                pullStatus.isTransientError()
+                    ? "after max attempts reached"
+                    : "with non transient error",
+                pullStatus,
+                getCurrentPullAttempt()));
+      }
+      throw new ZeroException(pullStatus.name());
+    } finally {
+      MDCLoggingContext.clear();
+    }
+  }
+
+  @VisibleForTesting
+  void notifyPullEnd(CorePullStatus pullStatus) {
+    // Tests count down on a latch to observe the end of the pull
+    pullEndNotification.accept(this, pullStatus);
+  }
+
+  /**
+   * Acquires the required locks and calls {@link #pullCoreFilesFromZeroStore(boolean)} to do the
+   * actual pulling work. Also checks a few conditions to skip the pull completely when not needed.
+   * No pull completion notification ({@link #notifyPullEnd(CorePullStatus)}) done from this method
+   * because if there are retries, we'd only notify after multiple failures (or a success).
+   */
+  @SuppressWarnings("try")
+  public CorePullStatus lockAndPullCore(boolean isLeaderPulling, long secondsToWaitPullLock) {
+    if (skipPullIfShardAlreadySynced(isLeaderPulling)) return CorePullStatus.NOT_NEEDED;
+    try (ZeroAccessLocks.NoThrowAutoCloseable ignore =
+        metadataCacheManager
+            .getOrCreateCoreMetadata(solrCore.getName())
+            .getZeroAccessLocks()
+            .acquirePullLock(secondsToWaitPullLock)) {
+      if (skipPullIfShardHasDefaultSuffix(isLeaderPulling)) {
+        return CorePullStatus.NOT_NEEDED;
+      }
+      return pullCoreFilesFromZeroStore(isLeaderPulling);
+    } catch (ZeroLockException ex) {
+      log.warn("Failed to acquire lock for pull", ex);
+      return CorePullStatus.FAILED_TO_ACQUIRE_LOCK;
+    } catch (ZeroException ex) {
+      log.warn("Failure pulling core", ex);
+      return CorePullStatus.FAILURE;
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      log.warn("Pull interrupted", ex);
+      return CorePullStatus.INTERRUPTED;
+    }
+  }
+
+  private String getTaskDescription() {
+    return String.format(
+        Locale.ROOT,
+        "(col=%s,shard=%s,name=%s,attempt=%d,thread=%s)",
+        getCollectionName(),
+        getShardName(),
+        getCoreName(),
+        getCurrentPullAttempt() + 1,
+        Thread.currentThread().getName());
+  }
+
+  /**
+   * This method is used to quickly ends core pulling if shard has never been updated since creation
+   * Requires read lock to be taken otherwise fails
+   */
+  private boolean skipPullIfShardHasDefaultSuffix(boolean isLeaderPulling) throws ZeroException {
+    ZeroMetadataVersion shardMetadataVersion = getShardMetadataVersion();
+    if (metadataController.hasDefaultNodeSuffix(shardMetadataVersion)) {
+      ZeroStoreShardMetadata emptyZeroStoreShardMetadata = new ZeroStoreShardMetadata();
+      metadataCacheManager.updateCoreMetadata(
+          solrCore.getName(), shardMetadataVersion, emptyZeroStoreShardMetadata, isLeaderPulling);
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * This method is used to quickly end core pulling if server side and remote shard have the same
+   * version
+   */
+  private boolean skipPullIfShardAlreadySynced(boolean isLeaderPulling) {
+    MetadataCacheManager.MetadataCacheEntry coreMetadata =
+        metadataCacheManager.getOrCreateCoreMetadata(solrCore.getName());
+    ZeroMetadataVersion shardMetadataVersion = getShardMetadataVersion();
+    // If leader is pulling, compare metadata to cache only if cache assumed up to date
+    return (coreMetadata.isCacheLikelyUpToDate() || !isLeaderPulling)
+        && coreMetadata.getMetadataVersion().equals(shardMetadataVersion);
+  }
+
+  /**
+   * Method that sleeps for a while depending on the number of pull attempts already done (in order
+   * not to retry pulling in short sequence and likely hit the same transient errors)
+   *
+   * @param retryDelay milliseconds to let elapse since previous pull attempt, by sleeping if needed
+   */
+  private void sleepIfNeeded(long retryDelay) throws InterruptedException {
+    if (getCurrentPullAttempt() != 0) {
+      long sleepTime =
+          retryDelay - getLastPullAttemptTime().until(Instant.now(), ChronoUnit.MILLIS);
+      if (sleepTime > 0) {
+        Thread.sleep(sleepTime);
+      }
+    }
+  }
+
+  /**
+   * Pulls required files representing the core update from the Zero store and returns the success
+   * or failure status. Pull lock ({@link ZeroAccessLocks#acquirePullLock(long)}) must be held.
+   */
+  private CorePullStatus pullCoreFilesFromZeroStore(boolean isLeaderPulling) throws ZeroException {
+    if (log.isInfoEnabled()) {
+      log.info("Initiating core pull {}", getTaskDescription());
+    }
+
+    ZeroMetadataVersion shardMetadataVersion = getShardMetadataVersion();
+
+    MetadataCacheManager.ZeroCoreStage stage = null;
+    ZeroStoreShardMetadata zeroMetadata;
+    CorePullStatus corePullStatus = CorePullStatus.FAILURE;
+
+    try {
+      metadataCacheManager.recordState(solrCore, MetadataCacheManager.ZeroCoreStage.PULL_STARTED);
+
+      // Get Zero store shard metadata
+      ZeroFile.WithLocal localZeroFile =
+          metadataController.newShardMetadataZeroFile(
+              getCollectionName(), getShardName(), shardMetadataVersion.getMetadataSuffix());
+      zeroMetadata = zeroStoreClient.pullShardMetadata(localZeroFile);
+
+      // Handle callback
+      if (zeroMetadata == null) {
+        return CorePullStatus.ZERO_METADATA_MISSING;
+      }
+
+      // Diff local metadata against Zero store metadata. Given we're doing a pull, don't
+      // need to reserve the commit point.
+      // We do need to compute a directory hash to verify after pulling or before switching index
+      // dirs that no local changes occurred concurrently
+      LocalCoreMetadata localCoreMetadata = new LocalCoreMetadata(solrCore);
+      localCoreMetadata.readMetadata(false, true);
+      MetadataComparisonResult metadataComparisonResult =
+          metadataController.diffMetadataforPull(localCoreMetadata, zeroMetadata);
+
+      // If there is nothing to pull, we should report SUCCESS_EQUIVALENT and do nothing.
+      if (!metadataComparisonResult.getFilesToPull().isEmpty()) {
+        pullUpdateFromZero(metadataComparisonResult, true);
+        corePullStatus = CorePullStatus.SUCCESS;
+      } else if (getOpenTimeForSearcher().isBefore(getLastPullAttemptTime())) {
+        // If the searcher's opened ts is less than the last time this task ran, we assume it failed
+        // to open successfully
+        log.warn(
+            "Looks like a searcher was never opened after the last pull from Zero store, doing that now. {} ",
+            getTaskDescription());
+        openSearcher(true);
+        corePullStatus = CorePullStatus.SUCCESS;
+      } else {
+        log.warn(
+            "There are no files to pull and we don't need to re-open a new searcher, even though we do not match the version in zk? {}",
+            getTaskDescription());
+        corePullStatus = CorePullStatus.NOT_NEEDED;
+      }
+
+      metadataCacheManager.updateCoreMetadata(
+          solrCore.getName(), shardMetadataVersion, zeroMetadata, isLeaderPulling);
+
+    } catch (InterruptedException e) {
+      corePullStatus = CorePullStatus.INTERRUPTED;
+      stage = MetadataCacheManager.ZeroCoreStage.PULL_FAILED;
+      throwZeroException(e, "Failed attempt to pull core");
+    } catch (CorruptedCoreException e) {
+      corePullStatus = CorePullStatus.FAILURE;
+      stage = MetadataCacheManager.ZeroCoreStage.PULL_FAILED_WITH_CORRUPTION;
+      throwZeroException(e, "Failed attempt to pull core");
+    } catch (Exception e) {
+      corePullStatus = CorePullStatus.FAILURE;
+      throwZeroException(e, "Failed attempt to pull core");
+    } finally {
+      if (stage == null) {
+        stage =
+            corePullStatus.isSuccess()
+                ? MetadataCacheManager.ZeroCoreStage.PULL_SUCCEEDED
+                : MetadataCacheManager.ZeroCoreStage.PULL_FAILED;
+      }
+      metadataCacheManager.recordState(solrCore, stage);
+    }
+    return corePullStatus;
+  }
+
+  /**
+   * We're doing here what replication does in {@link
+   * org.apache.solr.handler.IndexFetcher#fetchLatestIndex(boolean, boolean)}.
+   *
+   * <p>This method will work in 2 cases:
+   *
+   * <ol>
+   *   <li>Local core needs to fetch an update from Zero store
+   *   <li>Local core did not exist (was created empty before calling this method) and is fetched
+   *       from Zero store
+   * </ol>
+   *
+   * @param waitForSearcher {@code true} if this call should wait until the index searcher is
+   *     created (so that any query after the return from this method sees the new pulled content)
+   *     or {@code false} if we request a new index searcher to be eventually created but do not
+   *     wait for it to be created (a query following the return from this call might see the old
+   *     core content).
+   * @throws CorruptedCoreException If the local core index cannot be opened after pulling it from
+   *     the Zero store.
+   * @throws Exception We currently do not treat differently errors such as Zero store temporarily
+   *     not available or network issues. We therefore consider all exceptions thrown by this method
+   *     as a sign that it is durably not possible to pull the core from the Zero Store. TODO This
+   *     has to be revisited at some point
+   */
+  void pullUpdateFromZero(
+      MetadataComparisonResult metadataComparisonResult, boolean waitForSearcher) throws Exception {
+    Instant startTime = Instant.now();
+    boolean isSuccessful = false;
+    FileTransferCounter counter = new FileTransferCounter();
+    try {
+      // if there is a conflict between local and Zero store contents we will move the core to a new
+      // index directory
+      boolean coreSwitchedToNewIndexDir = false;
+      // Create temp directory (within the core local folder).
+      // If we are moving index to a new directory because of conflict then this will be that new
+      // directory.
+      // Even if we are not moving to a newer directory we will first download files from the Zero
+      // store into this temp directory.
+      // Then we will move files from temp directory to index directory. This is to avoid leaving
+      // a download half done in case of failure as well as to limit the time during which
+      // we close then reopen the index writer to take into account the new files. In theory
+      // nothing should be changing the local directory as we pull files from the Zero store,
+      // but let's be defensive (we're checking further down that local dir hasn't changed
+      // in the meantime).
+      String tempIndexDirName = "index.pull." + System.nanoTime();
+      String tempIndexDirPath = solrCore.getDataDir() + tempIndexDirName;
+      Directory tempIndexDir = getDirectory(solrCore, tempIndexDirPath);
+      try {
+        String indexDirPath = solrCore.getIndexDir();
+        Collection<ZeroFile.WithLocal> filesToDownload;
+        if (metadataComparisonResult.isLocalConflictingWithZero()) {
+          // This is an optimization to not download everything from Zero store if possible
+          Directory indexDir = getDirectory(solrCore, indexDirPath);
+          try {
+            filesToDownload =
+                initializeNewIndexDirWithLocallyAvailableFiles(
+                    indexDir, tempIndexDir, metadataComparisonResult.getFilesToPull());
+          } finally {
+            solrCore.getDirectoryFactory().release(indexDir);
+          }
+        } else {
+          filesToDownload = metadataComparisonResult.getFilesToPull();
+        }
+
+        pullZeroFiles(tempIndexDir, filesToDownload, counter);
+
+        Directory indexDir = getDirectory(solrCore, indexDirPath);
+        try {
+          if (!metadataComparisonResult.isLocalConflictingWithZero()) {
+            // TODO should we call solrCore.closeSearcher() here? IndexFetcher.fetchLatestIndex()
+            //  does call it.
+            // Close the index writer to stop changes to this core
+            solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(solrCore, true);
+          }
+
+          Exception thrownException = null;
+          try {
+            // Make sure Solr core directory content hasn't changed since we decided what we want
+            // to pull from the Zero store
+            if (!metadataComparisonResult.isSameDirectoryContent(indexDir)) {
+              // Maybe return something less aggressive than throwing an exception? TBD once we
+              // end up calling this method :)
+              throw new Exception(
+                  "Local Directory content "
+                      + indexDirPath
+                      + " has changed since core pull from Zero store started. Aborting pull.");
+            }
+
+            if (metadataComparisonResult.isLocalConflictingWithZero()) {
+              // point index to the new directory.
+              solrCore.modifyIndexProps(tempIndexDirName);
+              coreSwitchedToNewIndexDir = true;
+            } else {
+              moveFilesFromTempToIndexDir(tempIndexDir, indexDir, metadataComparisonResult);
+            }
+          } catch (Exception e) {
+            // Do not mask the exception in the finally block below.
+            thrownException = e;
+            throw e;
+          } finally {
+            try {
+              openIndexWriter(
+                  metadataComparisonResult.isLocalConflictingWithZero(), coreSwitchedToNewIndexDir);
+            } catch (Exception e) {
+              // We are not able to open an index writer on files we just wrote locally.
+              // Consider the index is corrupted and wipe it, so it will be pulled again.
+              wipeCorruptedIndex(indexDir);
+              wipeCorruptedIndex(tempIndexDir);
+              throw new CorruptedCoreException(
+                  "Core " + getCoreName() + " is corrupted locally",
+                  thrownException == null ? e : thrownException);
+            }
+          }
+        } finally {
+          try {
+            if (coreSwitchedToNewIndexDir) {
+              solrCore.getDirectoryFactory().doneWithDirectory(indexDir);
+              solrCore.getDirectoryFactory().remove(indexDir);
+            }
+          } catch (Exception e) {
+            log.warn("Cannot remove previous index directory {}", indexDir, e);
+          } finally {
+            solrCore.getDirectoryFactory().release(indexDir);
+          }
+        }
+      } finally {
+        try {
+          if (!coreSwitchedToNewIndexDir) {
+            solrCore.getDirectoryFactory().doneWithDirectory(tempIndexDir);
+            solrCore.getDirectoryFactory().remove(tempIndexDir);
+          }
+        } catch (Exception e) {
+          log.warn("Cannot remove temp directory {}", tempIndexDirPath, e);
+        } finally {
+          solrCore.getDirectoryFactory().release(tempIndexDir);
+        }
+      }
+      try {
+        openSearcher(waitForSearcher);
+      } catch (Exception e) {
+        // We are not able to open an index searcher on files we just wrote locally.
+        // Consider the index is corrupted and wipe it, so it will be pulled again.
+        Directory indexDir = getDirectory(solrCore, solrCore.getIndexDir());
+        try {
+          wipeCorruptedIndex(indexDir);
+        } finally {
+          solrCore.getDirectoryFactory().release(indexDir);
+        }
+        throw new CorruptedCoreException("Core " + getCoreName() + " is corrupted locally", e);
+      }
+      isSuccessful = true;
+    } finally {
+      if (log.isInfoEnabled()) {
+        log.info(
+            "PULL runTime={} bytesTransferred={} expectedFilesAffected={} actualFilesAffected={} isSuccessful={} "
+                + "localGeneration={} zeroGeneration={}",
+            startTime.until(Instant.now(), ChronoUnit.MILLIS),
+            counter.getBytesTransferred(),
+            counter.getExpectedFilesTransferred(),
+            counter.getActualFilesTransferred(),
+            isSuccessful,
+            metadataComparisonResult.getLocalGeneration(),
+            metadataComparisonResult.getDistantGeneration());
+      }
+    }
+  }
+
+  public static Directory getDirectory(SolrCore core, String path) throws IOException {
+    return core.getDirectoryFactory()
+        .get(path, DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+  }
+
+  @VisibleForTesting
+  protected void pullZeroFiles(
+      Directory tempIndexDir,
+      Collection<ZeroFile.WithLocal> filesToDownload,
+      FileTransferCounter counter)
+      throws InterruptedException, ExecutionException {
+    counter.setExpectedFilesTransferred(filesToDownload.size());
+
+    CompletableFuture<?>[] futures = new CompletableFuture<?>[filesToDownload.size()];
+    int i = 0;
+    for (ZeroFile.WithLocal bf : filesToDownload) {
+      futures[i++] = zeroStoreClient.pullFileAsync(tempIndexDir, bf);
+
+      counter.incrementActualFilesTransferred();
+      counter.incrementBytesTransferred(bf.getFileSize());
+    }
+
+    CompletableFuture.allOf(futures).get();
+  }
+
+  private Collection<ZeroFile.WithLocal> initializeNewIndexDirWithLocallyAvailableFiles(
+      Directory indexDir, Directory newIndexDir, Collection<ZeroFile.WithLocal> filesToPull) {
+    Collection<ZeroFile.WithLocal> filesToDownload = new HashSet<>();
+    for (ZeroFile.WithLocal zeroFile : filesToPull) {
+      assert zeroFile.isChecksumPresent();
+      try {
+        try (final IndexInput indexInput =
+            indexDir.openInput(zeroFile.getSolrFileName(), IOContext.READONCE)) {
+          if (indexInput.length() == zeroFile.getFileSize()
+              && CodecUtil.retrieveChecksum(indexInput) == zeroFile.getChecksum()) {
+            copyFileToDirectory(indexDir, zeroFile.getSolrFileName(), newIndexDir);
+          } else {
+            filesToDownload.add(zeroFile);
+          }
+        }
+      } catch (Exception ex) {
+        // Either file does not exist locally or copy failed, we will download from Zero store
+        filesToDownload.add(zeroFile);
+      }
+    }
+    return filesToDownload;
+  }
+
+  protected ZeroMetadataVersion getShardMetadataVersion() {
+    return metadataController.readMetadataValue(getCollectionName(), getShardName());
+  }
+
+  @VisibleForTesting
+  protected void openIndexWriter(boolean createNewIndexDir, boolean coreSwitchedToNewIndexDir)
+      throws Exception {
+    if (!createNewIndexDir) {
+      // The closed index writer must be opened back
+      // (in the finally block as stated in the javadoc of SolrCoreState.closeIndexWriter).
+      solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(solrCore);
+    } else if (coreSwitchedToNewIndexDir) {
+      solrCore.getUpdateHandler().newIndexWriter(true);
+    }
+  }
+
+  void openSearcher(boolean waitForSearcher) throws Exception {
+    if (waitForSearcher) {
+      // Open and register a new searcher, we don't need it but we wait for it to be open.
+      @SuppressWarnings("unchecked")
+      final Future<Void>[] waitSearcher = (Future<Void>[]) Array.newInstance(Future.class, 1);
+      solrCore.getSearcher(true, false, waitSearcher, true);
+      if (waitSearcher[0] == null) {
+        throw new Exception(
+            "Can't wait for index searcher to be created. Future queries might misbehave for core="
+                + solrCore.getName());
+      } else {
+        waitSearcher[0].get();
+      }
+    } else {
+      // Open and register a new searcher, but don't wait and we don't need it either.
+      solrCore.getSearcher(true, false, null, true);
+    }
+  }
+
+  /** Wipe the provided index directory. */
+  private void wipeCorruptedIndex(Directory dir) {
+    try {
+      // Delete the files individually instead of calling Directory.remove().
+      // The latter is not done immediately and may result in the files not being
+      // removed before the next pull attempt.
+      for (String file : dir.listAll()) {
+        dir.deleteFile(file);
+      }
+    } catch (Exception e) {
+      log.error("Cannot wipe the corrupted index in directory {}", dir, e);
+    }
+  }
+
+  /** Copies {@code fileName} from {@code fromDir} to {@code toDir} */
+  private void copyFileToDirectory(Directory fromDir, String fileName, Directory toDir)
+      throws IOException {
+    // TODO: Consider optimizing with org.apache.lucene.store.HardlinkCopyDirectoryWrapper
+    toDir.copyFrom(fromDir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+  }
+
+  /** Moves {@code fileName} from {@code fromDir} to {@code toDir} */
+  private void moveFileToDirectory(Directory fromDir, String fileName, Directory toDir)
+      throws IOException {
+    // We don't need to keep the original files so we move them over.
+    // TODO: Consider optimizing with org.apache.lucene.store.HardlinkCopyDirectoryWrapper
+    solrCore
+        .getDirectoryFactory()
+        .move(fromDir, toDir, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+  }
+
+  private void moveFilesFromTempToIndexDir(
+      Directory tmpIndexDir, Directory dir, MetadataComparisonResult metadataComparisonResult)
+      throws IOException {
+    if (metadataComparisonResult.getFilesToPull().isEmpty()) {
+      return;
+    }
+
+    // Copy all files into the Solr directory
+    // Move the segments_N file last once all other are ok.
+    String segmentsN = null;
+    for (ZeroFile.WithLocal bf : metadataComparisonResult.getFilesToPull()) {
+      if (metadataController.isSegmentsNFilename(bf)) {
+        assert segmentsN == null;
+        segmentsN = bf.getSolrFileName();
+      } else {
+        // Copy all non segments_N files
+        moveFileToDirectory(tmpIndexDir, bf.getSolrFileName(), dir);
+      }
+    }
+    assert segmentsN != null;
+    // Copy segments_N file. From this point on the local core might be accessed and is up to date
+    // with Zero store content
+    moveFileToDirectory(tmpIndexDir, segmentsN, dir);
+  }
+
+  /**
+   * @return timestamp in millis when the core's searcher was opened, NOT wall-clock time but nano
+   *     time, so that it can be compared against the last run timestamp of this job.
+   */
+  private Instant getOpenTimeForSearcher() throws IOException {
+    return solrCore == null
+        ? Instant.EPOCH
+        : Instant.ofEpochMilli(
+            TimeUnit.MILLISECONDS.convert(
+                solrCore.withSearcher(SolrIndexSearcher::getOpenTimeStamp).getTime(),
+                TimeUnit.NANOSECONDS));
+  }
+
+  public String getCoreName() {
+    return solrCore.getName();
+  }
+
+  public String getShardName() {
+    return solrCore.getCoreDescriptor().getCloudDescriptor().getShardId();
+  }
+
+  public String getCollectionName() {
+    return solrCore.getCoreDescriptor().getCollectionName();
+  }
+
+  private void throwZeroException(Exception ex, String format, Object... args)
+      throws ZeroException {
+    throw new ZeroException(
+        String.format(Locale.ROOT, format, args) + ": " + getTaskDescription(), ex);
+  }
+
+  private void throwZeroException(String format, Object... args) throws ZeroException {
+    throw new ZeroException(String.format(Locale.ROOT, format, args) + ": " + getTaskDescription());
+  }
+
+  public SolrCore getCore() {
+    return solrCore;
+  }
+
+  /** Needed for the {@link CorePuller} to be used in a {@link DeduplicatingList}. */
+  static class CorePullerMerger implements DeduplicatingList.Merger<String, CorePuller> {
+    /**
+     * Given two tasks (that have not yet started executing!) that target the same shard (and would
+     * basically do the same things were they both executed), returns a merged task that can replace
+     * both and that retains the oldest enqueue time and the smallest number of attempts, so we
+     * don't "lose" retries because of the merge yet we correctly report that tasks might have been
+     * waiting for execution for a long while.
+     *
+     * @return a merged {@link CorePuller} that can replace the two tasks passed as parameters.
+     */
+    @Override
+    public CorePuller merge(CorePuller puller1, CorePuller puller2) {
+      // We allow more opportunities to try as the core is changed again by Solr...
+      int mergedAttempts =
+          Math.min(puller1.getCurrentPullAttempt(), puller2.getCurrentPullAttempt());
+      int mergedMaxAttempts = Math.max(puller1.getMaxPullAttempts(), puller2.getMaxPullAttempts());
+
+      // ...and base the delay computation on the time of last attempt.
+      Instant mergedLastAttemptsTime =
+          puller1.getLastPullAttemptTime().isBefore(puller2.getLastPullAttemptTime())
+              ? puller2.getLastPullAttemptTime()
+              : puller1.getLastPullAttemptTime();
+
+      Instant minQueueTime =
+          puller1.getQueuedTime().isBefore(puller2.getQueuedTime())
+              ? puller1.getQueuedTime()
+              : puller2.getQueuedTime();
+      // We merge the tasks.
+      return puller1.newUpdatedPuller(
+          minQueueTime, mergedAttempts, mergedMaxAttempts, mergedLastAttemptsTime);
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CorePullerBlockingQueue.java b/solr/core/src/java/org/apache/solr/zero/process/CorePullerBlockingQueue.java
new file mode 100644
index 00000000000..db19e6acb8e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CorePullerBlockingQueue.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.process;
+
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.zero.util.DeduplicatingList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A wrapper class for a {@link DeduplicatingList} so it appears to be a {@link BlockingQueue} of
+ * {@link Runnable} and can be used by the {@link ZeroStoreManager#asyncCorePullExecutor} {@link
+ * ThreadPoolExecutor}.
+ *
+ * <p>This is a bare-bones implementation. Only methods {@link #take()}, {@link #isEmpty()} and
+ * {@link #drainTo(Collection)} are implemented. This class is only meant to serve the single very
+ * specific use case mentioned above.
+ */
+public class CorePullerBlockingQueue implements BlockingQueue<Runnable> {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final DeduplicatingList<String, CorePuller> pullTaskQueue;
+  private final ZeroStoreManager storeManager;
+
+  public CorePullerBlockingQueue(
+      DeduplicatingList<String, CorePuller> pullTaskQueue, ZeroStoreManager storeManager) {
+    this.pullTaskQueue = pullTaskQueue;
+    this.storeManager = storeManager;
+  }
+
+  /**
+   * Wrapper of {@link CorePuller} instantiated each time an object is {@link #take}-en from the
+   * {@link CorePullerBlockingQueue} so that the executor that consumes that queue ({@link
+   * ZeroStoreManager#asyncCorePullExecutor}) get a {@link Runnable} as expected.
+   */
+  static class RunnableAsyncCorePull implements Runnable {
+    private final CorePuller corePuller;
+    private final boolean isleaderPulling;
+    private final Instant enqueueTime = Instant.now();
+
+    public RunnableAsyncCorePull(CorePuller corePuller, boolean isleaderPulling) {
+      this.corePuller = corePuller;
+      this.isleaderPulling = isleaderPulling;
+    }
+
+    @Override
+    public void run() {
+      MDCLoggingContext.setCore(corePuller.getCore());
+      try {
+        CorePullStatus pullStatus = corePuller.lockAndPullCore(isleaderPulling, 0);
+        // Tests might count down on a latch to observe the end of the pull
+        corePuller.notifyPullEnd(pullStatus);
+        if (pullStatus.isSuccess()) {
+          if (log.isInfoEnabled()) {
+            log.info(
+                "Async Zero store pull finished successfully {} ms after being enqueued",
+                enqueueTime.until(Instant.now(), ChronoUnit.MILLIS));
+          }
+        } else {
+          // Just a log, no retries, the pull has failed. Future pulls might succeed
+          log.error(
+              "Async Zero store pull failed with status {} {} ms after being enqueued",
+              pullStatus,
+              enqueueTime.until(Instant.now(), ChronoUnit.MILLIS));
+        }
+      } finally {
+        MDCLoggingContext.clear();
+      }
+    }
+  }
+
+  @Override
+  public Runnable take() throws InterruptedException {
+    CorePuller cp = pullTaskQueue.removeFirst();
+    return new RunnableAsyncCorePull(cp, storeManager.isLeader(cp.getCore()));
+  }
+
+  @Override
+  public boolean isEmpty() {
+    return pullTaskQueue.size() == 0;
+  }
+
+  @Override
+  public int drainTo(Collection<? super Runnable> c) {
+    int count = 0;
+    while (!isEmpty()) {
+      try {
+        c.add(take());
+        count++;
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+      }
+    }
+    return count;
+  }
+
+  /*
+   * CorePullerBlockingQueue methods below this point throw UnsupportedOperationException
+   */
+
+  @Override
+  public boolean add(Runnable runnable) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean offer(Runnable runnable) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Runnable remove() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Runnable poll() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Runnable element() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Runnable peek() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void put(Runnable runnable) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean offer(Runnable runnable, long timeout, TimeUnit unit) throws InterruptedException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Runnable poll(long timeout, TimeUnit unit) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int remainingCapacity() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean remove(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean containsAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean addAll(Collection<? extends Runnable> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean removeAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean retainAll(Collection<?> c) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void clear() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int size() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public boolean contains(Object o) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Iterator<Runnable> iterator() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Object[] toArray() {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public <T> T[] toArray(T[] a) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public int drainTo(Collection<? super Runnable> c, int maxElements) {
+    throw new UnsupportedOperationException();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CorePusher.java b/solr/core/src/java/org/apache/solr/zero/process/CorePusher.java
new file mode 100644
index 00000000000..fa5017d16cb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CorePusher.java
@@ -0,0 +1,501 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+import org.apache.lucene.index.IndexCommit;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.client.solrj.cloud.BadVersionException;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.apache.solr.zero.exception.ZeroLockException;
+import org.apache.solr.zero.metadata.LocalCoreMetadata;
+import org.apache.solr.zero.metadata.MetadataCacheManager;
+import org.apache.solr.zero.metadata.MetadataCacheManager.MetadataCacheEntry;
+import org.apache.solr.zero.metadata.MetadataCacheManager.ZeroCoreStage;
+import org.apache.solr.zero.metadata.MetadataComparisonResult;
+import org.apache.solr.zero.metadata.ZeroMetadataController;
+import org.apache.solr.zero.metadata.ZeroMetadataVersion;
+import org.apache.solr.zero.metadata.ZeroStoreShardMetadata;
+import org.apache.solr.zero.util.FileTransferCounter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class executes synchronous pushes of core updates to the Zero store.
+ *
+ * <p>Pushes will be triggered at the end of an indexing batch when a shard's index data has changed
+ * locally and needs to be persisted in the Zero store, or when a core is created post split or
+ * restore.
+ *
+ * <p>Pushes to the Zero store do not happen during replica creation (including when part of
+ * collection creation) but only after the first indexing batch.
+ *
+ * <p>TODO This is most certainly an issue when a collection is created but not indexed right away,
+ * or when a collection is created with multiple shards and some of them are not indexed (possibly
+ * never indexed).
+ */
+public class CorePusher {
+
+  /**
+   * There should be no contention when acquiring these locks so a small timeout should be
+   * reasonable
+   */
+  private static final int LOCK_ACQUIRE_TIMEOUT = 5;
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  ZeroStoreClient zeroStoreClient;
+  DeleteProcessor deleteProcessor;
+  MetadataCacheManager metadataCacheManager;
+  ZeroMetadataController metadataController;
+  SolrCore solrCore;
+
+  public CorePusher(
+      SolrCore solrCore,
+      ZeroStoreClient zeroStoreClient,
+      DeleteProcessor deleteProcessor,
+      MetadataCacheManager metadataCacheManager,
+      ZeroMetadataController metadataController) {
+    this.zeroStoreClient = zeroStoreClient;
+    this.deleteProcessor = deleteProcessor;
+    this.metadataCacheManager = metadataCacheManager;
+    this.metadataController = metadataController;
+    this.solrCore = solrCore;
+  }
+
+  /**
+   * Pushes the local core updates to the Zero store and logs whether the push succeeded or failed.
+   * This is the end to end push method, comparing the local and distant states of the core (local
+   * core vs "distant" shard stored in Zero store), writing all required files to the Zero store and
+   * finally updating the metadata in ZooKeeper to refer to the new shard.metadata file.
+   */
+  @SuppressWarnings("try")
+  public CorePusherExecutionInfo endToEndPushCoreToZeroStore() throws SolrException {
+    try {
+      String collectionName = getCollectionName();
+      String shardName = getShardName();
+
+      MDCLoggingContext.setCore(solrCore);
+      if (log.isInfoEnabled()) {
+        log.info("Initiating push {}", getTaskDescription());
+      }
+
+      Instant startTime = Instant.now();
+      try (AutoCloseable ignore =
+          metadataCacheManager
+              .getOrCreateCoreMetadata(solrCore.getName())
+              .getZeroAccessLocks()
+              .acquirePushLock(LOCK_ACQUIRE_TIMEOUT)) {
+        long lockAcquisitionTime = startTime.until(Instant.now(), ChronoUnit.MILLIS);
+        try {
+          metadataCacheManager.recordState(solrCore, ZeroCoreStage.PUSH_STARTED);
+
+          IndexCommit latestCommit = solrCore.getDeletionPolicy().getLatestCommit();
+          if (latestCommit == null) {
+            throw new SolrException(
+                ErrorCode.SERVER_ERROR, "Core " + getCoreName() + " has no available commit point");
+          }
+
+          MetadataCacheEntry coreMetadata =
+              metadataCacheManager.getOrCreateCoreMetadata(solrCore.getName());
+          long zeroGeneration = coreMetadata.getZeroShardMetadata().getGeneration();
+          if (latestCommit.getGeneration() == zeroGeneration) {
+            // Everything up to latest commit point has already been pushed
+            // This can happen if another indexing batch comes in and acquires the push lock first
+            // and ends up pushing segments produced by this indexing batch.
+            // TODO could it be another node pushed a different segment of same generation?
+            if (log.isInfoEnabled())
+              log.info(
+                  "Nothing to push, pushLockTime={}: {}",
+                  lockAcquisitionTime,
+                  getTaskDescription());
+            long totalTimeMs = startTime.until(Instant.now(), ChronoUnit.MILLIS);
+            return CorePusherExecutionInfo.localAlreadyUpToDate(
+                zeroGeneration, lockAcquisitionTime, totalTimeMs);
+          }
+
+          // Compute the differences (if any) between the local shard index data and shard index
+          // data on the Zero store. The names of new files to push to the Zero store will include
+          // a suffix that is generated here and passed to the comparison method. Even though not
+          // required, all files pushed together will have the same suffix (but different file
+          // names). This helps debug when trying to understand when a file appeared.
+          // Reserving the commit point, so it can be saved while pushing files to the Zero store.
+          // We don't need to compute a directory hash for the push scenario as we only need to
+          // verify local index does not change during pull.
+          LocalCoreMetadata localCoreMetadata = new LocalCoreMetadata(solrCore);
+          localCoreMetadata.readMetadata(true, false);
+          MetadataComparisonResult metadataComparisonResult =
+              metadataController.diffMetadataForPush(
+                  localCoreMetadata,
+                  coreMetadata.getZeroShardMetadata(),
+                  ZeroStoreShardMetadata.generateMetadataSuffix());
+          if (metadataComparisonResult.getFilesToPush().isEmpty()) {
+            log.warn(
+                "Why there is nothing to push even when there is a newer commit point since last push pushLockTime={}: {}",
+                lockAcquisitionTime,
+                getTaskDescription());
+            long totalTimeMs = startTime.until(Instant.now(), ChronoUnit.MILLIS);
+            return CorePusherExecutionInfo.oddEmptyDiffCommitPoint(
+                zeroGeneration,
+                Collections.emptySet(),
+                metadataComparisonResult.getFilesToDelete(),
+                lockAcquisitionTime,
+                totalTimeMs);
+          }
+
+          // begin the push process
+          Instant pushStartTime = Instant.now();
+
+          ZeroStoreShardMetadata zeroStoreShardMetadata =
+              pushFilesToZeroStore(metadataComparisonResult);
+          metadataCacheManager.recordState(
+              solrCore, MetadataCacheManager.ZeroCoreStage.FILE_PUSHED);
+          long pushTimeMs = pushStartTime.until(Instant.now(), ChronoUnit.MILLIS);
+          // at this point we've pushed the new metadata file with the newMetadataSuffix and now
+          // need to write to zookeeper
+          Instant metadataUpdateStartTime = Instant.now();
+          ZeroMetadataVersion newShardMetadataVersion = null;
+          try {
+            newShardMetadataVersion =
+                metadataController.updateMetadataValueWithVersion(
+                    collectionName,
+                    shardName,
+                    metadataComparisonResult.getMetadataSuffix(),
+                    coreMetadata.getMetadataVersion().getVersion());
+          } catch (Exception ex) {
+            // conditional update of zookeeper failed, reset "cache likely up to date"
+            // regardless of the exception type.
+            // We observe that we can hit Keeper$ConnectionLoss while the suffix is actually updated
+            // That will make sure before processing next indexing batch, we sync with zookeeper and
+            // pull from the Zero store.
+            metadataCacheManager.updateCoreMetadata(solrCore.getName(), false);
+            throw ex;
+          }
+          metadataCacheManager.recordState(
+              solrCore, MetadataCacheManager.ZeroCoreStage.ZK_UPDATE_FINISHED);
+          long metadataUpdateTimeMs =
+              metadataUpdateStartTime.until(Instant.now(), ChronoUnit.MILLIS);
+
+          assert metadataComparisonResult
+              .getMetadataSuffix()
+              .equals(newShardMetadataVersion.getMetadataSuffix());
+          // after successful update to zookeeper, update cached core metadata with new info and
+          // consider cache likely up to date wrt the Zero store
+          metadataCacheManager.updateCoreMetadata(
+              solrCore.getName(), newShardMetadataVersion, zeroStoreShardMetadata, true);
+          metadataCacheManager.recordState(
+              solrCore, MetadataCacheManager.ZeroCoreStage.LOCAL_CACHE_UPDATE_FINISHED);
+          if (log.isInfoEnabled())
+            log.info(
+                "Successfully pushed to the Zero store, pushLockTime={}: {}",
+                lockAcquisitionTime,
+                getTaskDescription());
+          long totalTimeMs = startTime.until(Instant.now(), ChronoUnit.MILLIS);
+          return new CorePusherExecutionInfo(
+              true,
+              zeroGeneration,
+              metadataComparisonResult.getMetadataSuffix(),
+              metadataComparisonResult.getFilesToPush(),
+              metadataComparisonResult.getFilesToDelete(),
+              lockAcquisitionTime,
+              pushTimeMs,
+              metadataUpdateTimeMs,
+              totalTimeMs);
+        } finally {
+          metadataCacheManager.recordState(solrCore, ZeroCoreStage.PUSH_FINISHED);
+        }
+      }
+      // TODO - make error handling a little nicer?
+    } catch (InterruptedException e) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR, "CorePusher was interrupted while pushing to the Zero store", e);
+    } catch (SolrException e) {
+      Throwable t = e.getCause();
+      if (t instanceof BadVersionException) {
+        throw new SolrException(
+            ErrorCode.SERVER_ERROR,
+            "CorePusher failed to push because the node "
+                + "version doesn't match, requestedVersion="
+                + ((BadVersionException) t).getRequested(),
+            t);
+      }
+      throw e;
+    } catch (Exception ex) {
+      // wrap every thrown exception in a solr exception
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Error trying to push to the Zero store " + getTaskDescription(),
+          ex);
+    } finally {
+      MDCLoggingContext.clear();
+    }
+  }
+
+  /**
+   * This method calls the {@link #endToEndPushCoreToZeroStore()} after having acquired the indexing
+   * lock. It verifies that this is the first push of that shard to the Zero store.
+   *
+   * <p>Acquiring the indexing lock is slightly vain... This code works because it is called from a
+   * restore or post shard split, there are no competing cores elsewhere in the cluster that could
+   * push content to the Zero store while this core is trying to do its push.
+   *
+   * <p>TODO need to clarify the lifecycle of the MetadataCacheManager vs ZK stored metadataSuffix
+   */
+  @SuppressWarnings("try")
+  public void initialCorePushToZeroStore() throws InterruptedException {
+    String collectionName = getCollectionName();
+    String shardName = getShardName();
+
+    // we'll do a timed attempt to acquire the read lock, but at this point in time the sub shard
+    // should not be active and indexing so we shouldn't wait long
+    try (ZeroAccessLocks.NoThrowAutoCloseable ignore =
+        metadataCacheManager
+            .getOrCreateCoreMetadata(solrCore.getName())
+            .getZeroAccessLocks()
+            .acquireIndexingLock(LOCK_ACQUIRE_TIMEOUT)) {
+      ZeroMetadataVersion shardMetadataVersion =
+          metadataController.readMetadataValue(collectionName, shardName);
+      // TODO if split failed and retried, will it use a different core name? Or fail forever?
+      if (!metadataController.hasDefaultNodeSuffix(shardMetadataVersion)) {
+        throw new SolrException(
+            SolrException.ErrorCode.SERVER_ERROR,
+            "New shard has zk information that is not default. "
+                + "The suffix value is "
+                + shardMetadataVersion.getMetadataSuffix()
+                + getTaskDescription());
+      }
+      // sync local cache with zk's default information i.e. equivalent of no-op pull
+      // this syncing is necessary for the zk conditional update to succeed at the end of core
+      // push
+      try (ZeroAccessLocks.NoThrowAutoCloseable ignore2 =
+          metadataCacheManager
+              .getOrCreateCoreMetadata(solrCore.getName())
+              .getZeroAccessLocks()
+              .acquirePushLock(LOCK_ACQUIRE_TIMEOUT)) {
+        // Push yet to happen so metadata cache not assumed up to date
+        metadataCacheManager.updateCoreMetadata(
+            solrCore.getName(), shardMetadataVersion, new ZeroStoreShardMetadata(), false);
+        endToEndPushCoreToZeroStore();
+      }
+    } catch (ZeroLockException e) {
+
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Push lock: " + e.getMessage() + " " + getTaskDescription());
+    }
+  }
+
+  /**
+   * Writes to the Zero store all the files that should be written to it, including the contents of
+   * {@link ZeroStoreShardMetadata} as shard.metadata (with a suffix). This method does not update
+   * ZooKeeper with the new metadataSuffix. Until this happens, the writes to Zero store are not
+   * taken into account.
+   */
+  @VisibleForTesting
+  protected ZeroStoreShardMetadata pushFilesToZeroStore(
+      MetadataComparisonResult metadataComparisonResult) throws Exception {
+
+    String collectionName = getCollectionName();
+    String shardName = getShardName();
+
+    MetadataCacheEntry coreMetadata =
+        metadataCacheManager.getOrCreateCoreMetadata(solrCore.getName());
+    ZeroStoreShardMetadata shardMetadata = coreMetadata.getZeroShardMetadata();
+    Instant startTime = Instant.now();
+
+    FileTransferCounter counter = new FileTransferCounter();
+    boolean isSuccessful = false;
+
+    try {
+      MDCLoggingContext.setCore(solrCore);
+
+      // Creating the new ZeroStoreShardMetadata as a modified clone of the existing one
+      ZeroStoreShardMetadata newShardMetadata =
+          new ZeroStoreShardMetadata(
+              shardMetadata.getZeroFiles(),
+              shardMetadata.getZeroFilesToDelete(),
+              metadataComparisonResult.getLocalGeneration());
+
+      // Submit for delete only Zero store files previously marked for deletion, not the ones that
+      // the current update marks for deletion. That's why the enqueue for delete is done before
+      // adding the just deleted files to newShardMetadata. If the current update fails (in the
+      // ZooKeeper update phase for example), the just deleted files remain required.
+      enqueueForHardDelete(newShardMetadata);
+
+      // Files that are part of the current commit point on the Zero store but no longer needed
+      // after this push succeeds are not deleted (see above why) but moved from the set of files
+      // being part of the shard to the set of files that need to be deleted later.
+      for (ZeroFile.WithLocal bf : metadataComparisonResult.getFilesToDelete()) {
+        newShardMetadata.removeFile(bf);
+        ZeroFile.ToDelete zftd = new ZeroFile.ToDelete(bf, Instant.now());
+        newShardMetadata.addFileToDelete(zftd);
+      }
+
+      // add the old shard.metadata file to delete
+      if (!metadataController.hasDefaultNodeSuffix(coreMetadata)) {
+        ZeroFile.ToDelete zftd =
+            metadataController.newShardMetadataZeroFileToDelete(
+                collectionName, shardName, coreMetadata.getMetadataVersion().getMetadataSuffix());
+        newShardMetadata.addFileToDelete(zftd);
+      }
+
+      // When we build the LocalCoreMetadata we requested to reserve the commit point for some short
+      // duration. Assumption is it took less than this duration to get here (no blocking
+      // operations). Now we actually save the commit point for the (potentially long) time it takes
+      // to push all files to the Zero store.
+      solrCore.getDeletionPolicy().saveCommitPoint(metadataComparisonResult.getLocalGeneration());
+
+      Directory coreIndexDir = CorePuller.getDirectory(solrCore, solrCore.getIndexDir());
+      try {
+        // parallel pushes of segment files and the new shard.metadata file into the Zero store
+        CompletableFuture<?>[] futures =
+            new CompletableFuture<?>[metadataComparisonResult.getFilesToPush().size() + 1];
+
+        int i = 0;
+        // Push all segment files
+        for (ZeroFile.WithLocal zf : metadataComparisonResult.getFilesToPush()) {
+          // Sanity check that we're talking about the same file (just sanity, Solr doesn't update
+          // files so should never be different)
+          assert zf.getFileSize() == coreIndexDir.fileLength(zf.getSolrFileName());
+
+          futures[i++] = zeroStoreClient.pushFileAsync(coreIndexDir, zf);
+          newShardMetadata.addFile(zf);
+          counter.incrementActualFilesTransferred();
+          counter.incrementBytesTransferred(zf.getFileSize());
+        }
+        // push the new shard.metadata in parallel with other files
+        // As opposed to segment files, for shard.metadata not capturing file size and checksum.
+        // This is because there is nothing to compare them with, as opposed to normal segment files
+        // for which this information is captured... in the shard.metadata file :-O
+        futures[i] =
+            zeroStoreClient.pushShardMetadataAsync(
+                metadataController.newShardMetadataZeroFile(
+                    getCollectionName(),
+                    getShardName(),
+                    metadataComparisonResult.getMetadataSuffix()),
+                newShardMetadata);
+        counter.incrementActualFilesTransferred();
+        String shardMetadataFileContent = newShardMetadata.toJson();
+        counter.incrementBytesTransferred(shardMetadataFileContent.length());
+
+        CompletableFuture.allOf(futures).get();
+      } finally {
+        solrCore.getDirectoryFactory().release(coreIndexDir);
+        solrCore
+            .getDeletionPolicy()
+            .releaseCommitPoint(metadataComparisonResult.getLocalGeneration());
+      }
+
+      isSuccessful = true;
+      return newShardMetadata;
+    } finally {
+      MDCLoggingContext.clear();
+
+      counter.setExpectedFilesTransferred(metadataComparisonResult.getFilesToPush().size());
+
+      if (log.isInfoEnabled()) {
+        log.info(
+            "PUSH runTime={} bytesTransferred={} expectedFilesAffected={} actualFilesAffected={} isSuccessful={} "
+                + "localGeneration={} zeroGeneration={}",
+            startTime.until(Instant.now(), ChronoUnit.MILLIS),
+            counter.getBytesTransferred(),
+            counter.getExpectedFilesTransferred(),
+            counter.getActualFilesTransferred(),
+            isSuccessful,
+            metadataComparisonResult.getLocalGeneration(),
+            coreMetadata.getZeroShardMetadata().getGeneration());
+      }
+    }
+  }
+
+  @VisibleForTesting
+  public void enqueueForHardDelete(ZeroStoreShardMetadata shardMetadata) {
+    Set<ZeroFile.ToDelete> filesToDelete =
+        shardMetadata.getZeroFilesToDelete().stream()
+            .filter(this::okForHardDelete)
+            .collect(Collectors.toSet());
+    if (enqueueForDelete(filesToDelete)) {
+      shardMetadata.removeFilesFromDeleted(filesToDelete);
+    }
+  }
+
+  /**
+   * @return true if the files were enqueued for deletion successfully
+   */
+  @VisibleForTesting
+  protected boolean enqueueForDelete(Set<ZeroFile.ToDelete> zeroFilesToDelete) {
+    if (zeroFilesToDelete == null || zeroFilesToDelete.isEmpty()) {
+      return false;
+    }
+    Set<ZeroFile> zeroFiles = new HashSet<>(zeroFilesToDelete);
+
+    if (zeroFiles.isEmpty()) return true;
+    try {
+      deleteProcessor.deleteFiles(zeroFiles, true);
+      return true;
+    } catch (Exception ex) {
+      return false;
+    }
+  }
+
+  private String getTaskDescription() {
+    return String.format(
+        Locale.ROOT,
+        "(col=%s,shard=%s,name=%s,thread=%s)",
+        getCollectionName(),
+        getShardName(),
+        getCoreName(),
+        Thread.currentThread().getName());
+  }
+
+  /**
+   * Returns true if a deleted Zero store file (i.e. a file marked for delete but not deleted yet)
+   * can be hard deleted now.
+   */
+  @VisibleForTesting
+  protected boolean okForHardDelete(ZeroFile.ToDelete file) {
+    // For now we only check how long ago the file was marked for delete.
+    return file.getDeletedAt().until(Instant.now(), ChronoUnit.MILLIS)
+        >= deleteProcessor.getDeleteDelayMs();
+  }
+
+  protected String getCoreName() {
+    return solrCore.getName();
+  }
+
+  protected String getShardName() {
+    return solrCore.getCoreDescriptor().getCloudDescriptor().getShardId();
+  }
+
+  protected String getCollectionName() {
+    return solrCore.getCoreDescriptor().getCollectionName();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/CorePusherExecutionInfo.java b/solr/core/src/java/org/apache/solr/zero/process/CorePusherExecutionInfo.java
new file mode 100644
index 00000000000..ffc3a478a5e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/CorePusherExecutionInfo.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.solr.zero.client.ZeroFile;
+
+public class CorePusherExecutionInfo {
+
+  private final boolean hasPushed;
+  private final long zeroGeneration;
+  private final String metadataSuffix;
+  private final Collection<ZeroFile.WithLocal> filesToDelete;
+  private final Collection<ZeroFile.WithLocal> filesToPush;
+  private final long pushLockWaitTimeMs;
+  private final long actualPushTimeMs;
+  private final long metadataUpdateTimeMs;
+  private final long totalTimeMs;
+
+  public CorePusherExecutionInfo(
+      boolean hasPushed,
+      long zeroGeneration,
+      String metadataSuffix,
+      Collection<ZeroFile.WithLocal> filesToPush,
+      Collection<ZeroFile.WithLocal> filesToDelete,
+      long pushLockWaitTimeMs,
+      long actualPushTimeMs,
+      long metadataUpdateTimeMs,
+      long totalTimeMs) {
+    this.hasPushed = hasPushed;
+    this.zeroGeneration = zeroGeneration;
+    this.metadataSuffix = metadataSuffix;
+    this.filesToPush = filesToPush;
+    this.filesToDelete = filesToDelete;
+    this.pushLockWaitTimeMs = pushLockWaitTimeMs;
+    this.actualPushTimeMs = actualPushTimeMs;
+    this.metadataUpdateTimeMs = metadataUpdateTimeMs;
+    this.totalTimeMs = totalTimeMs;
+  }
+
+  public static CorePusherExecutionInfo localAlreadyUpToDate(
+      long zeroGeneration, long pushLockWaitTimeMs, long totalTimeMs) {
+    return new CorePusherExecutionInfo(
+        false,
+        zeroGeneration,
+        null,
+        new ArrayList<>(),
+        new ArrayList<>(),
+        pushLockWaitTimeMs,
+        0,
+        0,
+        totalTimeMs);
+  }
+
+  public static CorePusherExecutionInfo oddEmptyDiffCommitPoint(
+      long zeroGeneration,
+      Collection<ZeroFile.WithLocal> filesToPush,
+      Collection<ZeroFile.WithLocal> filesToDelete,
+      long pushLockWaitTimeMs,
+      long totalTimeMs) {
+    return new CorePusherExecutionInfo(
+        false,
+        zeroGeneration,
+        null,
+        filesToPush,
+        filesToDelete,
+        pushLockWaitTimeMs,
+        0,
+        0,
+        totalTimeMs);
+  }
+
+  public boolean hasPushed() {
+    return hasPushed;
+  }
+
+  public long getZeroGeneration() {
+    return zeroGeneration;
+  }
+
+  public String getMetadataSuffix() {
+    return metadataSuffix;
+  }
+
+  public long getPushLockWaitTimeMs() {
+    return pushLockWaitTimeMs;
+  }
+
+  public long getActualPushTimeMs() {
+    return actualPushTimeMs;
+  }
+
+  public long getMetadataUpdateTimeMs() {
+    return metadataUpdateTimeMs;
+  }
+
+  public long getTotalTimeMs() {
+    return totalTimeMs;
+  }
+
+  public int getNumFilesPushed() {
+    return filesToPush.size();
+  }
+
+  public long getSizeBytesPushed() {
+    return filesToPush.stream().mapToLong(ZeroFile.WithLocal::getFileSize).sum();
+  }
+
+  public int getNumFilesToDelete() {
+    return filesToDelete.size();
+  }
+
+  @Override
+  public String toString() {
+    return "ExecutionInfo [hasPushed="
+        + hasPushed
+        + ", zeroGeneration="
+        + zeroGeneration
+        + ", metadataSuffix="
+        + metadataSuffix
+        + ", filesToPush="
+        + filesToPush
+        + ", filesToDelete="
+        + filesToDelete
+        + ", pushLockWaitTimeMs="
+        + pushLockWaitTimeMs
+        + ", actualPushTimeMs="
+        + actualPushTimeMs
+        + ", metadataUpdateTimeMs="
+        + metadataUpdateTimeMs
+        + ", totalTimeMs="
+        + totalTimeMs
+        + "]";
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/DeleteProcessor.java b/solr/core/src/java/org/apache/solr/zero/process/DeleteProcessor.java
new file mode 100644
index 00000000000..b31000abce7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/DeleteProcessor.java
@@ -0,0 +1,196 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.Closeable;
+import java.lang.invoke.MethodHandles;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.ExecutorUtil.MDCAwareThreadPoolExecutor;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A generic deletion processor used for deleting files from Zero store. Each processor manages its
+ * own task bounded thread pool for processing {@link DeleterTask} asynchronously. Processors
+ * support retrying tasks if necessary but retry decisions are left to the individual task
+ * implementations.
+ *
+ * <p>Instances of {@link DeleteProcessor} are managed by the {@link ZeroStoreManager}.
+ */
+public class DeleteProcessor implements Closeable {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final String name;
+  private final int almostMaxQueueSize;
+
+  /**
+   * Note we sleep() after each failed attempt, so multiply this value by {@link #fixedRetryDelay}
+   * to find out how long we'll retry (at least) if Zero store access fails for some reason ("at
+   * least" because we re-enqueue at the tail of the queue, so there might be additional processing
+   * delay if the queue is not empty and is processed before the enqueued retry is processed).
+   */
+  private final int maxDeleteAttempts;
+
+  private final long fixedRetryDelay;
+  private final ZeroStoreClient zeroStoreClient;
+  private final BlockingQueue<Runnable> deleteQueue;
+  private final MDCAwareThreadPoolExecutor deleteExecutor;
+  private final long deleteDelayMs;
+
+  /**
+   * @param name identifying the processor
+   * @param zeroStoreClient the zeroStoreClient to use in this processor
+   * @param almostMaxQueueSize the target max queue size
+   * @param numDeleterThreads the number of threads to configure in the underlying thread pool
+   * @param defaultMaxDeleteAttempts maximum number of attempts to retry any task enqueued in this
+   *     processor
+   * @param fixedRetryDelay fixed time delay in ms between retry attempts
+   */
+  public DeleteProcessor(
+      String name,
+      ZeroStoreClient zeroStoreClient,
+      int almostMaxQueueSize,
+      int numDeleterThreads,
+      int defaultMaxDeleteAttempts,
+      long fixedRetryDelay,
+      long deleteDelayMs) {
+    this.name = name;
+    this.almostMaxQueueSize = almostMaxQueueSize;
+    this.maxDeleteAttempts = defaultMaxDeleteAttempts;
+    this.fixedRetryDelay = fixedRetryDelay;
+    this.deleteDelayMs = deleteDelayMs;
+    NamedThreadFactory threadFactory = new NamedThreadFactory(name);
+
+    // Note this queue MUST NOT BE BOUNDED, or we risk deadlocks given that DeleterTask's
+    // re-enqueue themselves upon failure
+    deleteQueue = new LinkedBlockingDeque<>();
+
+    deleteExecutor =
+        new MDCAwareThreadPoolExecutor(
+            numDeleterThreads, numDeleterThreads, 0L, TimeUnit.SECONDS, deleteQueue, threadFactory);
+    this.zeroStoreClient = zeroStoreClient;
+  }
+
+  /**
+   * Enqueues the given set of files for deletion from Zero store as an async task.
+   *
+   * @param zeroFiles list of files to delete from Zero store
+   * @param allowRetry flag indicating if the task should be retried if it fails
+   */
+  public CompletableFuture<DeleterTask.Result> deleteFiles(
+      Set<ZeroFile> zeroFiles, boolean allowRetry) {
+    String collectionName = zeroFiles.iterator().next().getCollectionName();
+    DeleterTask task =
+        new FilesDeletionTask(
+            zeroStoreClient, collectionName, zeroFiles, allowRetry, maxDeleteAttempts);
+    return enqueue(task, false);
+  }
+
+  /**
+   * Enqueues a task to delete all files belonging to the given collection from Zero store as an
+   * async task.
+   *
+   * @param collectionName the name of the collection to be deleted from Zero store
+   * @param allowRetry flag indicating if the task should be retried if it fails
+   */
+  public CompletableFuture<DeleterTask.Result> deleteCollection(
+      String collectionName, boolean allowRetry) {
+    DeleterTask task =
+        new CollectionDeletionTask(zeroStoreClient, collectionName, allowRetry, maxDeleteAttempts);
+    return enqueue(task, false);
+  }
+
+  /**
+   * Enqueues a task to delete all files belonging to the given collection and shard as an async
+   * task.
+   *
+   * @param collectionName targeted collection
+   * @param shardName shard to delete
+   * @param allowRetry flag indicating if the task should be retried if it fails
+   */
+  public CompletableFuture<DeleterTask.Result> deleteShard(
+      String collectionName, String shardName, boolean allowRetry) {
+    DeleterTask task =
+        new ShardDeletionTask(
+            zeroStoreClient, collectionName, shardName, allowRetry, maxDeleteAttempts);
+    return enqueue(task, false);
+  }
+
+  /**
+   * Enqueues a task to be processed by a thread in the {@link DeleteProcessor#deleteExecutor}
+   * thread pool. The callback is handled by the same execution thread and will re-enqueue a task
+   * that has failed and should be retried. Tasks that are enqueued via the retry mechanism are not
+   * bound by the same size constraints as newly minted tasks are.
+   *
+   * @return CompletableFuture to allow calling threads the capability to block on the computation
+   *     results as needed, retrieved suppressed exceptions in retry, etc
+   */
+  @VisibleForTesting
+  protected CompletableFuture<DeleterTask.Result> enqueue(DeleterTask task, boolean isRetry) {
+    if (!isRetry && deleteQueue.size() > almostMaxQueueSize) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Unable to enqueue deletion task: " + task.toString());
+    }
+
+    return CompletableFuture.supplyAsync(task::call, deleteExecutor)
+        .thenCompose(
+            result -> {
+              // the callback will execute on the same thread as the executing task
+              if (!result.isSuccess() && result.shouldRetry()) {
+                try {
+                  // Some delay before retry... (could move this delay to before trying to delete a
+                  // file that previously failed to be deleted, that way if the queue is busy, and
+                  // it took time to retry, we don't add a delay on top of that. On the other hand,
+                  // an exception here could be an issue with the Zero store itself and nothing
+                  // specific to the file at hand, so slowing all delete attempts for all files
+                  // might make sense.
+                  Thread.sleep(fixedRetryDelay);
+                  return enqueue(result.getTask(), result.shouldRetry());
+                } catch (Exception ex) {
+                  log.error(
+                      "Could not re-enqueue failed deleter task that should have been enqueued!",
+                      ex);
+                }
+              }
+              return CompletableFuture.completedFuture(result);
+            });
+  }
+
+  @Override
+  public void close() {
+    deleteExecutor.shutdown();
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public long getDeleteDelayMs() {
+    return deleteDelayMs;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/DeleterTask.java b/solr/core/src/java/org/apache/solr/zero/process/DeleterTask.java
new file mode 100644
index 00000000000..9929f906adb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/DeleterTask.java
@@ -0,0 +1,181 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Generic deletion task for files located on the Zero store */
+public abstract class DeleterTask implements Callable<DeleterTask.Result> {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  protected final ZeroStoreClient zeroStoreClient;
+
+  private final Set<ZeroFile> zeroFiles;
+  private final AtomicInteger attempt;
+  private final Instant queuedTime;
+  private final int maxAttempts;
+  private final boolean allowRetry;
+  private Throwable err;
+
+  public DeleterTask(
+      ZeroStoreClient zeroStoreClient,
+      Set<ZeroFile> zeroFiles,
+      boolean allowRetry,
+      int maxAttempts) {
+    this.zeroStoreClient = zeroStoreClient;
+    this.zeroFiles = zeroFiles;
+    this.attempt = new AtomicInteger(0);
+    this.queuedTime = Instant.now();
+    this.allowRetry = allowRetry;
+    this.maxAttempts = maxAttempts;
+  }
+
+  public abstract String getBasePath();
+
+  /** Return a String representing the action performed by the DeleterTask for logging purposes */
+  public abstract String getActionName();
+
+  public abstract void setMDCContext();
+
+  @Override
+  public DeleterTask.Result call() {
+    List<ZeroFile> filesDeleted = new ArrayList<>();
+    final Instant startTime = Instant.now();
+    boolean isSuccess = true;
+    boolean shouldRetry = false;
+    try {
+      filesDeleted.addAll(doDelete());
+      attempt.incrementAndGet();
+      return new DeleterTask.Result(this, filesDeleted, isSuccess, shouldRetry, err);
+    } catch (Exception ex) {
+      if (err == null) {
+        err = ex;
+      } else {
+        err.addSuppressed(ex);
+      }
+      int attempts = attempt.incrementAndGet();
+      isSuccess = false;
+      log.warn("{} DeleterTask failed on attempt={}", this, attempts, ex);
+      if (allowRetry) {
+        if (attempts < maxAttempts) {
+          shouldRetry = true;
+        } else {
+          log.warn("{} reached {} attempt limit. This task won't be retried.", this, maxAttempts);
+        }
+      }
+    } finally {
+      Instant now = Instant.now();
+      long runTime = startTime.until(now, ChronoUnit.MILLIS);
+      long startLatency = this.queuedTime.until(now, ChronoUnit.MILLIS);
+      if (log.isInfoEnabled()) {
+        log.info(
+            "{} path={} runTime={} startLatency={} isSuccess={}",
+            this,
+            getBasePath(),
+            runTime,
+            startLatency,
+            isSuccess);
+      }
+    }
+    return new DeleterTask.Result(this, filesDeleted, isSuccess, shouldRetry, err);
+  }
+
+  public Collection<ZeroFile> doDelete() throws Exception {
+    zeroStoreClient.deleteZeroFiles(zeroFiles);
+    return zeroFiles;
+  }
+
+  /** Represents the result of a deletion task */
+  public static class Result {
+    private final DeleterTask task;
+    private final Collection<ZeroFile> filesDeleted;
+    private boolean isSuccess;
+    private final boolean shouldRetry;
+    private final Throwable err;
+
+    public Result(
+        DeleterTask task,
+        Collection<ZeroFile> filesDeleted,
+        boolean isSuccess,
+        boolean shouldRetry,
+        Throwable errs) {
+      this.task = task;
+      this.filesDeleted = filesDeleted;
+      this.isSuccess = isSuccess;
+      this.shouldRetry = shouldRetry;
+      this.err = errs;
+    }
+
+    public boolean isSuccess() {
+      return isSuccess;
+    }
+
+    public void updateSuccess(boolean s) {
+      isSuccess = s;
+    }
+
+    public boolean shouldRetry() {
+      return shouldRetry;
+    }
+
+    public DeleterTask getTask() {
+      return task;
+    }
+
+    /**
+     * @return the files that are being deleted. Note if the task wasn't successful there is no
+     *     guarantee all of these files were in fact deleted from the Zero store
+     */
+    public Collection<ZeroFile> getFilesDeleted() {
+      return filesDeleted;
+    }
+
+    public Throwable getError() {
+      return err;
+    }
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        Locale.ROOT,
+        "action=%s totalFilesSpecified=%d allowRetry=%b queuedTime=%s attemptsTried=%d",
+        getActionName(),
+        zeroFiles.size(),
+        allowRetry,
+        queuedTime.toString(),
+        attempt.get());
+  }
+
+  public int getAttempts() {
+    return attempt.get();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/FilesDeletionTask.java b/solr/core/src/java/org/apache/solr/zero/process/FilesDeletionTask.java
new file mode 100644
index 00000000000..fcacd5883cb
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/FilesDeletionTask.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import java.util.Locale;
+import java.util.Set;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.zero.client.ZeroFile;
+import org.apache.solr.zero.client.ZeroStoreClient;
+
+public class FilesDeletionTask extends DeleterTask {
+
+  protected final String collectionName;
+
+  public FilesDeletionTask(
+      ZeroStoreClient zeroStoreClient,
+      String collectionName,
+      Set<ZeroFile> zeroFiles,
+      boolean allowRetry,
+      int maxRetryAttempt) {
+    super(zeroStoreClient, zeroFiles, allowRetry, maxRetryAttempt);
+    this.collectionName = collectionName;
+  }
+
+  @Override
+  public String getActionName() {
+    return "DELETE_FILES";
+  }
+
+  @Override
+  public void setMDCContext() {
+    MDCLoggingContext.setCollection(collectionName);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(
+        Locale.ROOT, "FilesDeletionTask %s collection=%s", super.toString(), collectionName);
+  }
+
+  @Override
+  public String getBasePath() {
+    return zeroStoreClient.getCollectionURI(collectionName).toString();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/ShardDeletionTask.java b/solr/core/src/java/org/apache/solr/zero/process/ShardDeletionTask.java
new file mode 100644
index 00000000000..f30068bde21
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/ShardDeletionTask.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.NoSuchFileException;
+import java.util.Locale;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** A file deletion task that deletes all files from Zero store under given path */
+public class ShardDeletionTask extends FilesDeletionTask {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  protected final String shardName;
+
+  public ShardDeletionTask(
+      ZeroStoreClient zeroStoreClient,
+      String collectionName,
+      String shardName,
+      boolean allowRetry,
+      int maxRetryAttempt) {
+    super(
+        zeroStoreClient,
+        collectionName,
+        zeroStoreClient.listShardZeroFiles(collectionName, shardName),
+        allowRetry,
+        maxRetryAttempt);
+    this.shardName = shardName;
+  }
+
+  @Override
+  public String getActionName() {
+    return "DELETE_SHARD_FILES";
+  }
+
+  @Override
+  public void setMDCContext() {
+    super.setMDCContext();
+    MDCLoggingContext.setShard(shardName);
+  }
+
+  @Override
+  public String toString() {
+    return String.format(Locale.ROOT, "ShardDeletionTask %s shard=%s", super.toString(), shardName);
+  }
+
+  @Override
+  public String getBasePath() {
+    return zeroStoreClient.getShardURI(collectionName, shardName).toString();
+  }
+
+  @Override
+  public DeleterTask.Result call() {
+    DeleterTask.Result result = super.call();
+    if (result.isSuccess()) {
+      try {
+        zeroStoreClient.deleteShardDirectory(collectionName, shardName);
+      } catch (NoSuchFileException ex) {
+        if (log.isWarnEnabled())
+          log.warn(
+              "Could not delete Zero store directory for collection={} shard={} as it does not exist",
+              collectionName,
+              shardName);
+      } catch (IOException ex) {
+        result.updateSuccess(false);
+        if (log.isWarnEnabled())
+          log.warn(
+              "Could not delete Zero store directory for collection={} shard={} after all files have been deleted",
+              collectionName,
+              shardName);
+      }
+    }
+    return result;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/ZeroAccessLocks.java b/solr/core/src/java/org/apache/solr/zero/process/ZeroAccessLocks.java
new file mode 100644
index 00000000000..9472edfc0c1
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/ZeroAccessLocks.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.invoke.MethodHandles;
+import java.time.Instant;
+import java.time.temporal.ChronoUnit;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.solr.zero.exception.ZeroLockException;
+import org.apache.solr.zero.metadata.ZeroMetadataVersion;
+import org.apache.solr.zero.metadata.ZeroStoreShardMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Locking logic (and locks) for read and write access to the Zero store */
+public class ZeroAccessLocks {
+  /**
+   * Should only be created once at initialization time, subsequent updates should reuse same lock
+   * instance. see {@link
+   * org.apache.solr.zero.metadata.MetadataCacheManager.MetadataCacheEntry#updatedOf(ZeroMetadataVersion,
+   * ZeroStoreShardMetadata, boolean)} and other overload.
+   *
+   * <p>We don't need fair ordering policy for this lock, which normally has lower throughput. We
+   * rely on cacheLikelyUpToDate and isLeader at query time so that queries may not contend on this
+   * lock and let the steady state indexing do its job without contention.
+   *
+   * <p>On indexing side we rely on read lock and we can have multiple readers just fine. Write lock
+   * is only needed in a fail over scenario(leader changed) where we need to pull from Zero store
+   * but that is only needed to be done by one thread. Therefore, we acquire write lock with a
+   * timeout and check for that condition after the timeout. Therefore, no concern of starvation
+   * there either.
+   */
+  private final ReentrantReadWriteLock indexingAndPullLock;
+
+  /**
+   * Should only be created once at initialization time, subsequent updates should reuse same lock
+   * instance. see {@link
+   * org.apache.solr.zero.metadata.MetadataCacheManager.MetadataCacheEntry#updatedOf(ZeroMetadataVersion,
+   * ZeroStoreShardMetadata, boolean)} and other overload.
+   *
+   * <p>Use a fair ordering policy for this lock, to ensure that the longest waiting thread gets to
+   * acquire the lock first. This will help prevent threads that may starve waiting to acquire the
+   * lock to push, even though another thread may have already completed the work on their behalf.
+   * See {@link CorePusher#endToEndPushCoreToZeroStore()} for details.
+   */
+  private final ReentrantLock pushLock;
+
+  private final CloseableLock closableIndexingLock;
+  private final CloseableLock closablePullLock;
+  private final CloseableLock closablePushLock;
+
+  public ZeroAccessLocks() {
+    this(new ReentrantReadWriteLock(), new ReentrantLock(true));
+  }
+
+  @VisibleForTesting
+  public ZeroAccessLocks(ReentrantReadWriteLock indexingAndPullLock, ReentrantLock pushLock) {
+    this.indexingAndPullLock = indexingAndPullLock;
+    this.pushLock = pushLock;
+
+    closableIndexingLock = new CloseableLock(indexingAndPullLock.readLock());
+    closablePullLock = new CloseableLock(indexingAndPullLock.writeLock());
+    closablePushLock = new CloseableLock(pushLock);
+  }
+
+  /**
+   * This lock must be acquired before starting indexing (or any other update) on a core. It
+   * disallows pulls from occurring during indexing, because otherwise the subsequent Zero store
+   * update could end up overwriting a previous update and leading to data loss. This lock does
+   * allow multiple indexing batches to proceed concurrently on a core. This lock must be held
+   * during the whole indexing duration, including when the core is pushed to the Zero store at the
+   * end of indexing (pushing requires first calling {@link #acquirePushLock(long)})
+   */
+  public NoThrowAutoCloseable acquireIndexingLock(long timeoutSecond)
+      throws ZeroLockException, InterruptedException {
+    return closableIndexingLock.tryLockAsResource(timeoutSecond, TimeUnit.SECONDS);
+  }
+
+  /**
+   * This lock protects pulls from each other (only one pull at any given time for efficiency
+   * reasons given only one can succeed), and prevents indexing from happening concurrently with
+   * pulls (this could lead to data loss). This lock must be held for the whole duration of a core
+   * pull from the Zero store.
+   */
+  public NoThrowAutoCloseable acquirePullLock(long timeoutSecond)
+      throws ZeroLockException, InterruptedException {
+    return closablePullLock.tryLockAsResource(timeoutSecond, TimeUnit.SECONDS);
+  }
+
+  /**
+   * This lock is required for pushing the core to Zero store. Before acquiring this lock, {@link
+   * #acquireIndexingLock(long)} must have been called (both are needed for pushing). Although
+   * multiple indexing batches can index in parallel (multiple threads can concurrently call {@link
+   * #acquireIndexingLock(long)}, only one push can be done in parallel to the Zero store, and only
+   * one thread at a time will succeed calling this method (for efficiency reasons, because if
+   * multiple pushes happen in parallel only one will succeed).
+   */
+  public NoThrowAutoCloseable acquirePushLock(long timeoutSecond)
+      throws ZeroLockException, InterruptedException {
+    return closablePushLock.tryLockAsResource(timeoutSecond, TimeUnit.SECONDS);
+  }
+
+  /**
+   * To update the cached core metadata, need either a pull write lock or push lock along with pull
+   * read lock
+   */
+  public boolean canUpdateCoreMetadata() {
+    return indexingAndPullLock.getWriteHoldCount() > 0
+        || (pushLock.getHoldCount() > 0 && indexingAndPullLock.getReadHoldCount() > 0);
+  }
+
+  public int getIndexingHoldCount() {
+    // The indexing lock is the read component. See closableIndexingLock assignment
+    return indexingAndPullLock.getReadHoldCount();
+  }
+
+  public int getPullHoldCount() {
+    // The pull lock is the write component. See closablePullLock assignment
+    return indexingAndPullLock.getWriteHoldCount();
+  }
+
+  @Override
+  public String toString() {
+    return "pullLockHoldCount="
+        + getPullHoldCount()
+        + " indexingLockHoldCount="
+        + getIndexingHoldCount()
+        + " pushLockHoldCount="
+        + pushLock.getHoldCount();
+  }
+
+  public interface NoThrowAutoCloseable extends AutoCloseable {
+    @Override
+    void close();
+  }
+
+  public static class CloseableLock {
+    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+    private final ZeroAccessLocks.NoThrowAutoCloseable unlocker = CloseableLock.this::unlock;
+
+    private final Lock lock;
+
+    public CloseableLock(Lock lock) {
+      assert (lock != null);
+      this.lock = lock;
+    }
+
+    public ZeroAccessLocks.NoThrowAutoCloseable tryLockAsResource(long timeout, TimeUnit unit)
+        throws InterruptedException, ZeroLockException {
+      Instant startTime = Instant.now();
+      boolean success = lock.tryLock(timeout, unit);
+      if (!success) {
+        if (log.isWarnEnabled()) {
+          log.warn(
+              "Acquiring lock timed out after: {} ",
+              startTime.until(Instant.now(), ChronoUnit.MILLIS));
+        }
+        throw new ZeroLockException("Couldn't acquire lock within %d ms ", timeout);
+      } else {
+        if (log.isDebugEnabled()) {
+          log.debug("Lock acquired in {} ms", startTime.until(Instant.now(), ChronoUnit.MILLIS));
+        }
+        return unlocker;
+      }
+    }
+
+    public void unlock() {
+      lock.unlock();
+      log.info("Lock released successfully");
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/ZeroCoreIndexingBatchProcessor.java b/solr/core/src/java/org/apache/solr/zero/process/ZeroCoreIndexingBatchProcessor.java
new file mode 100644
index 00000000000..427356572e8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/ZeroCoreIndexingBatchProcessor.java
@@ -0,0 +1,337 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.io.Closeable;
+import java.lang.invoke.MethodHandles;
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.logging.MDCLoggingContext;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.zero.exception.ZeroLockException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class implements the features required for a ZERO core (replica) to do indexing. It brings
+ * it up to date (if needed) before indexing starts and pushes the updated core after a successful
+ * commit.
+ */
+public class ZeroCoreIndexingBatchProcessor implements Closeable {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Time indexing thread needs to wait to try acquiring pull lock before checking if someone else
+   * has already done the pull.
+   */
+  public static final int SECONDS_TO_WAIT_PULL_LOCK = 5;
+
+  /**
+   * If an indexing thread is waiting on the indexing lock, then it has not performed local indexing
+   * yet. If it is waiting, that means another thread is currently pulling index files for the core.
+   * Other threads can be indexing concurrently so the longest wait time is proportional to the time
+   * it takes to hydrate (load) a full core from Zero store. This can take time on saturated
+   * clusters.
+   */
+  public static final int SECONDS_TO_WAIT_INDEXING_LOCK = 120;
+
+  private final SolrCore core;
+  private final String collectionName;
+  private final String shardName;
+  private IndexingBatchState state;
+  private final ZeroStoreManager zeroStoreManager;
+  private final ZeroAccessLocks locks;
+
+  private ZeroAccessLocks.NoThrowAutoCloseable coreIndexingLock;
+  private final SolrQueryResponse response;
+
+  public ZeroCoreIndexingBatchProcessor(
+      SolrCore core, ClusterState clusterState, SolrQueryResponse response) {
+    this.core = core;
+    this.response = response;
+    CloudDescriptor cloudDescriptor = core.getCoreDescriptor().getCloudDescriptor();
+    collectionName = cloudDescriptor.getCollectionName();
+    shardName = cloudDescriptor.getShardId();
+
+    CoreContainer coreContainer = core.getCoreContainer();
+    zeroStoreManager = coreContainer.getZeroStoreManager();
+    locks = zeroStoreManager.getZeroAccessLocks(core.getName());
+
+    DocCollection collection = clusterState.getCollection(collectionName);
+    if (!collection.isZeroIndex()) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR, collectionName + " is not a Zero collection.");
+    }
+
+    Slice shard = collection.getSlicesMap().get(shardName);
+    if (shard == null) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          "Indexing batch received for an unknown shard,"
+              + " collection="
+              + collectionName
+              + " shard="
+              + shardName
+              + " core="
+              + core.getName());
+    }
+
+    if (!Slice.State.ACTIVE.equals(shard.getState())) {
+      // This happens when we buffer updates for a sub shard.
+      // ZERO replica should eventually stop supporting buffered updates and then this should
+      // become a real exception
+      log.warn(
+          "Processing an indexing batch for a non-active shard, collection={} shard={} core={}",
+          collectionName,
+          shardName,
+          core.getName());
+
+      // TODO it would have been better to have instead the code below, but then ZERO
+      //  replica split tests fail. Something to look into.
+      // In ZeroStoreUpdateProcessor.rejectIfConstructing() an indexing batch would be rejected
+      // if a shard splits, so seeing activity here might be due to post split actions?
+
+      //      // ZERO replica do not support buffered updates (not until transaction log is updated
+      //      // to go to Zero store, as the Solr nodes are considered stateless)
+      //      throw new SolrException(
+      //          ErrorCode.SERVER_ERROR,
+      //          "Processing an indexing batch for a non-active shard, collection="
+      //              + collectionName
+      //              + " shard="
+      //              + shardName
+      //              + " core="
+      //              + core.getName());
+    }
+
+    state = IndexingBatchState.NOT_STARTED;
+  }
+
+  /**
+   * Should be called whenever a document is about to be added/deleted from the ZERO core. If it is
+   * the first doc of the core, this method will mark the start of an indexing batch and bring a
+   * stale ZERO core upto date by pulling from the Zero store.
+   */
+  public void addOrDeleteGoingToBeIndexedLocally() {
+    // Following logic is built on the assumption that one particular instance of this processor
+    // will solely be consumed by a single thread. And all the documents of indexing batch will be
+    // processed by this one instance.
+    try {
+      MDCLoggingContext.setCore(core);
+
+      if (IndexingBatchState.NOT_STARTED.equals(state)) {
+        startIndexingBatch();
+      } else if (IndexingBatchState.STARTED.equals(state)) {
+        // do nothing, we only use this method to start an indexing batch once
+      } else if (IndexingBatchState.COMMITTED.equals(state)) {
+        throw new SolrException(
+            ErrorCode.SERVER_ERROR,
+            "Why are we adding/deleting a doc through an already committed indexing batch?"
+                + " collection="
+                + collectionName
+                + " shard="
+                + shardName
+                + " core="
+                + core.getName());
+      } else {
+        throwUnknownStateError();
+      }
+    } finally {
+      MDCLoggingContext.clear();
+    }
+  }
+
+  @VisibleForTesting
+  protected void startIndexingBatch() {
+    // Following pull logic should only run once before the first add/delete of an indexing batch is
+    // processed by this processor
+    assert IndexingBatchState.NOT_STARTED.equals(state);
+
+    if (coreIndexingLock != null) {
+      String err = "Indexing thread already has an indexing lock but indexing has not yet started!";
+      if (locks.getIndexingHoldCount() != 0) {
+        err += " The current thread also holds a lock on it when it shouldn't!";
+      }
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          err
+              + " collection="
+              + collectionName
+              + " shard="
+              + shardName
+              + " core="
+              + core.getName());
+    }
+
+    try {
+      zeroStoreManager.setIndexingBatchReceived(core);
+      state = IndexingBatchState.STARTED;
+      CorePullStatus pullStatus = zeroStoreManager.pullCoreFromZeroStore(core);
+      response.addToLog("pull.done", pullStatus == CorePullStatus.SUCCESS);
+      coreIndexingLock = locks.acquireIndexingLock(getIndexingLockTimeout());
+      zeroStoreManager.setIndexingStarted(core);
+    } catch (ZeroLockException ex) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          "Indexing thread timed out trying to acquire the pull read lock in "
+              + getIndexingLockTimeout()
+              + " seconds"
+              + " collection="
+              + collectionName
+              + " shard="
+              + shardName
+              + " core="
+              + core.getName());
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          "Indexing thread interrupted while trying to acquire pull read lock."
+              + " collection="
+              + collectionName
+              + " shard="
+              + shardName
+              + " core="
+              + core.getName(),
+          ex);
+    }
+  }
+
+  /**
+   * Should be called after the ZERO core has successfully hard committed locally. This method will
+   * push the updated core to the Zero store. If there was no local add/delete of a document for
+   * this processor then the push will be skipped.
+   */
+  public void hardCommitCompletedLocally() {
+    finishIndexingBatch();
+  }
+
+  protected void finishIndexingBatch() {
+    String coreName = core.getName();
+    if (IndexingBatchState.NOT_STARTED.equals(state)) {
+      // This is an isolated commit since we've not seen any added/deleted doc
+      // ZERO replicas currently require (and force) hard commits after each batch, isolated commits
+      // are therefore ignored
+      state = IndexingBatchState.COMMITTED;
+      if (log.isInfoEnabled()) {
+        log.info(
+            "Isolated commit encountered for a ZERO replica, ignoring writing to Zero store."
+                + " collection={} shard={} core={}",
+            collectionName,
+            shardName,
+            coreName);
+      }
+    } else if (IndexingBatchState.STARTED.equals(state)) {
+      if (locks.getIndexingHoldCount() != 1) {
+        throw new SolrException(
+            ErrorCode.SERVER_ERROR,
+            "Indexing lock hold count is not 1 (is "
+                + locks.getIndexingHoldCount()
+                + ") collection="
+                + collectionName
+                + " shard="
+                + shardName
+                + " core="
+                + coreName);
+      }
+
+      zeroStoreManager.setIndexingFinished(core);
+      state = IndexingBatchState.COMMITTED;
+      CorePusherExecutionInfo pushExecutionInfo = zeroStoreManager.pushCoreToZeroStore(core);
+      addToLog(pushExecutionInfo);
+
+    } else if (IndexingBatchState.COMMITTED.equals(state)) {
+      throw new SolrException(
+          ErrorCode.SERVER_ERROR,
+          "Why are we committing an already committed indexing batch?"
+              + " collection="
+              + collectionName
+              + " shard="
+              + shardName
+              + " core="
+              + coreName);
+    } else {
+      throwUnknownStateError();
+    }
+  }
+
+  private void addToLog(CorePusherExecutionInfo pushExecutionInfo) {
+    response.addToLog("push.done", pushExecutionInfo.hasPushed());
+    response.addToLog("push.zeroGeneration", pushExecutionInfo.getZeroGeneration());
+    response.addToLog("push.metadataSuffix", pushExecutionInfo.getMetadataSuffix());
+    response.addToLog("push.lockWaiTimeMs", pushExecutionInfo.getPushLockWaitTimeMs());
+    response.addToLog("push.actualPushTimeMs", pushExecutionInfo.getActualPushTimeMs());
+    response.addToLog("push.metadataUpdateTimeMs", pushExecutionInfo.getMetadataUpdateTimeMs());
+    response.addToLog("push.totalTimeMs", pushExecutionInfo.getTotalTimeMs());
+    response.addToLog("push.numFiles", pushExecutionInfo.getNumFilesPushed());
+    response.addToLog("push.sizeBytes", pushExecutionInfo.getSizeBytesPushed());
+    response.addToLog("push.numFilesToDelete", pushExecutionInfo.getNumFilesToDelete());
+  }
+
+  private void throwUnknownStateError() {
+    throw new SolrException(
+        ErrorCode.SERVER_ERROR,
+        "Programmer's error, unknown IndexingBatchState"
+            + state
+            + " collection="
+            + collectionName
+            + " shard="
+            + shardName
+            + " core="
+            + core.getName());
+  }
+
+  @VisibleForTesting
+  protected int getIndexingLockTimeout() {
+    return SECONDS_TO_WAIT_INDEXING_LOCK;
+  }
+
+  @Override
+  public void close() {
+    if (!IndexingBatchState.NOT_STARTED.equals(state)) {
+      try {
+        zeroStoreManager.setIndexingFinished(core);
+      } catch (Exception ex) {
+        log.error("Error recording the finish of a ZERO core indexing batch", ex);
+      }
+    }
+    // TODO would be nice to decide in a more deterministic way if the lock has to be released
+    if (locks.getIndexingHoldCount() > 0) {
+      // release read lock
+      coreIndexingLock.close();
+      if (locks.getIndexingHoldCount() != 0) {
+        log.error(
+            "Indexing lock of a ZERO core was unlocked but this thread still holds it: {}",
+            locks.getIndexingHoldCount());
+      }
+    }
+  }
+
+  private enum IndexingBatchState {
+    NOT_STARTED,
+    STARTED,
+    COMMITTED
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/zero/process/ZeroStoreManager.java b/solr/core/src/java/org/apache/solr/zero/process/ZeroStoreManager.java
new file mode 100644
index 00000000000..264607f72f4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/zero/process/ZeroStoreManager.java
@@ -0,0 +1,692 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.zero.process;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.ZkController;
+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.params.CollectionAdminParams;
+import org.apache.solr.common.util.CollectionUtil;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.ZeroConfig;
+import org.apache.solr.zero.client.ZeroStoreClient;
+import org.apache.solr.zero.client.ZeroStoreClientFactory;
+import org.apache.solr.zero.exception.ZeroException;
+import org.apache.solr.zero.exception.ZeroLockException;
+import org.apache.solr.zero.metadata.MetadataCacheManager;
+import org.apache.solr.zero.metadata.ZeroMetadataController;
+import org.apache.solr.zero.metadata.ZeroMetadataVersion;
+import org.apache.solr.zero.util.DeduplicatingList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** This is the main entry point to the ZERO implementation from existing SolrCloud classes. */
+public class ZeroStoreManager {
+
+  /**
+   * If more than this number of DISTINCT cores need to be enqueued for async pull, the enqueue
+   * might block. We likely do not want the enqueue to block (currently enqueue is done from {@link
+   * org.apache.solr.servlet.HttpSolrCall#init()} so it would block a Jetty thread), so keep this
+   * number high enough. They day is far though before a single SolrCloud node is able to support
+   * that number of cores. That constant is likely more than an order of magnitude too big.
+   */
+  private static final int CORE_TO_PULL_LIST_MAX_SIZE = 100000;
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private final CoreContainer coreContainer;
+  private ZeroMetadataController zeroMetadataController;
+  private MetadataCacheManager metadataCacheManager;
+  private ZeroStoreClient zeroStoreClient;
+
+  /** A generic delete processor that is used only to delete individual files */
+  private DeleteProcessor corePusherFileDeleteProcessor;
+
+  private final DeduplicatingList<String, CorePuller> corePullTaskQueue;
+
+  /**
+   * This executor feeds off the {@link #corePullTaskQueue} {@link DeduplicatingList}, disguising as
+   * a bare bones {@link BlockingQueue} by using {@link CorePullerBlockingQueue}. This executor is
+   * used in a very specific way: nothing is ever submitted to it, it only consumes the queue, with
+   * entries added directly into the queue. This imposes constraints on the worker threads of the
+   * executor.
+   *
+   * <p>The reason is that we want to maintain the queue of cores to pull as a {@link
+   * DeduplicatingList} of {@link CorePuller} in order to effectively be able to deduplicate
+   * inserts. A {@link ThreadPoolExecutor} only accepts queues of {@link Runnable}, so we can't
+   * easily do sumbits (unless we submitted {@link CorePuller} instances already wrapped within a
+   * {@link Runnable}, but then the deduplication code would become ugly).
+   */
+  private final ExecutorUtil.MDCAwareThreadPoolExecutor asyncCorePullExecutor;
+
+  private DeleteProcessor overseerDeleteProcessor;
+  private CorePullerFactory corePullerFactory;
+  private final ZeroConfig config;
+
+  public ZeroStoreManager(CoreContainer coreContainer, ZeroConfig config) {
+    this.coreContainer = coreContainer;
+
+    zeroMetadataController =
+        new ZeroMetadataController(coreContainer.getZkController().getSolrCloudManager());
+    metadataCacheManager = new MetadataCacheManager(config, coreContainer);
+
+    // The default CorePuller creation below passes null for the pullEndNotification which
+    // means no pull end notifications. A different factory is installed by tests when a
+    // notification is needed
+    corePullerFactory =
+        (solrCore,
+            zeroStoreClient,
+            metadataCacheManager,
+            metadataController,
+            maxFailedCorePullAttempts) ->
+            new CorePuller(
+                solrCore,
+                zeroStoreClient,
+                metadataCacheManager,
+                metadataController,
+                maxFailedCorePullAttempts,
+                null);
+
+    this.zeroStoreClient =
+        ZeroStoreClientFactory.newInstance(
+            coreContainer.getNodeConfig(), coreContainer.getMetricManager());
+
+    corePusherFileDeleteProcessor =
+        new DeleteProcessor(
+            "PusherFileDeleter",
+            zeroStoreClient,
+            config.getAlmostMaxDeleterQueueSize(),
+            config.getDeleterThreadPoolSize(),
+            config.getMaxDeleteAttempts(),
+            config.getDeleteSleepMsFailedAttempt(),
+            config.getDeleteDelayMs());
+
+    // Non-Overseer nodes will initiate a delete processor but the underlying pool will sit idle
+    // until the node is elected and tasks are added. The overhead should be small.
+    overseerDeleteProcessor =
+        new DeleteProcessor(
+            "OverseerDeleteProcessor",
+            zeroStoreClient,
+            config.getAlmostMaxDeleterQueueSize(),
+            config.getDeleterThreadPoolSize(),
+            config.getMaxDeleteAttempts(),
+            config.getDeleteSleepMsFailedAttempt(),
+            config.getDeleteDelayMs());
+
+    this.config = config;
+
+    // Queue containing cores to pull from the Zero store
+    this.corePullTaskQueue = CorePuller.getDeduplicatingList(CORE_TO_PULL_LIST_MAX_SIZE);
+    // The executor executing the code pulling these cores. Actual files are pulled using the
+    // file pull executor defined in ZeroStoreClient.
+    //
+    // Tasks are added directly into the queue, not via the executor. We therefore set a fixed
+    // thread pool size (i.e. core size = max size) to not let threads terminate but have them
+    // wait for tasks to arrive in the queue.
+    // TODO All pushing and pulling executors likely belong in the same place
+    asyncCorePullExecutor =
+        new ExecutorUtil.MDCAwareThreadPoolExecutor(
+            config.getNumCorePullerThreads(),
+            config.getNumCorePullerThreads(),
+            0L,
+            TimeUnit.MILLISECONDS,
+            new CorePullerBlockingQueue(corePullTaskQueue, this),
+            new SolrNamedThreadFactory("asyncCorePullExecutor"));
+    // We also need to pre-start all threads because nothing will start them for us (there are no
+    // direct submits to this ThreadPoolExecutor).
+    asyncCorePullExecutor.prestartAllCoreThreads();
+
+    if (log.isInfoEnabled()) {
+      log.info("asyncCorePullExecutor initialized");
+    }
+  }
+
+  public ZeroStoreClient getZeroStoreClient() {
+    return zeroStoreClient;
+  }
+
+  /** For testing purposes only. Current Zero store client is closed before replacing it. */
+  @VisibleForTesting
+  public void replaceZeroStoreClient(ZeroStoreClient zeroStoreClient) {
+    if (this.zeroStoreClient != null) {
+      this.zeroStoreClient.shutdown();
+    }
+    this.zeroStoreClient = zeroStoreClient;
+  }
+
+  @VisibleForTesting
+  public void replaceDeleteProcessors(
+      DeleteProcessor deleteProcessor, DeleteProcessor overseerDeleteProcessor) {
+    if (this.corePusherFileDeleteProcessor != null) {
+      this.corePusherFileDeleteProcessor.close();
+    }
+    this.corePusherFileDeleteProcessor = deleteProcessor;
+
+    if (this.overseerDeleteProcessor != null) {
+      this.overseerDeleteProcessor.close();
+    }
+    this.overseerDeleteProcessor = overseerDeleteProcessor;
+  }
+
+  public ZeroMetadataController getZeroMetadataController() {
+    return zeroMetadataController;
+  }
+
+  public MetadataCacheManager getMetadataCacheManager() {
+    return metadataCacheManager;
+  }
+
+  public CoreContainer getCoreContainer() {
+    return coreContainer;
+  }
+
+  public ZeroConfig getConfig() {
+    return config;
+  }
+
+  /**
+   * Check and evict any existing entry for Zero store metadata in the MetadataCacheManager. Zero
+   * store metadata should be treated the way CoreDescriptors in SolrCores are, and removed from
+   * this cache when the descriptors are removed from SolrCores.
+   */
+  public void evictCoreZeroMetadata(CoreDescriptor cd) {
+    if (cd != null
+        && cd.getCloudDescriptor().getReplicaType().equals(Replica.Type.ZERO)
+        && metadataCacheManager.removeCoreMetadata(cd.getName())) {
+      String collectionName = cd.getCollectionName();
+      String shardId = cd.getCloudDescriptor().getShardId();
+      if (log.isInfoEnabled()) {
+        log.info(
+            "Evicted core {} for collection {} and shard {} from ZERO core concurrency cache",
+            cd.getName(),
+            collectionName,
+            shardId);
+      }
+    }
+  }
+
+  public CorePuller newCorePuller(SolrCore solrCore) {
+    return corePullerFactory.createPuller(
+        solrCore,
+        zeroStoreClient,
+        metadataCacheManager,
+        zeroMetadataController,
+        config.getMaxFailedCorePullAttempts());
+  }
+
+  public CorePusher newCorePusher(SolrCore solrCore) {
+    return new CorePusher(
+        solrCore,
+        zeroStoreClient,
+        corePusherFileDeleteProcessor,
+        metadataCacheManager,
+        zeroMetadataController);
+  }
+
+  /**
+   * This method is called at the end of an indexing batch. It pushes new content of the local core
+   * present in the Zero store. When called after the first indexing batch for a core, it pushes all
+   * files of the core (i.e. this could be the first write of that shard to the Zero store).
+   */
+  public CorePusherExecutionInfo pushCoreToZeroStore(SolrCore core) {
+    CorePusher corePusher = newCorePusher(core);
+    return corePusher.endToEndPushCoreToZeroStore();
+  }
+
+  public void shutdown() {
+    stopBackgroundCorePulling();
+
+    if (corePusherFileDeleteProcessor != null) {
+      corePusherFileDeleteProcessor.close();
+      if (log.isInfoEnabled()) {
+        log.info("DeleteProcessor {} has shutdown", corePusherFileDeleteProcessor.getName());
+      }
+    }
+    if (overseerDeleteProcessor != null) {
+      overseerDeleteProcessor.close();
+      if (log.isInfoEnabled()) {
+        log.info("Overseer DeleteProcessor {} has shutdown", overseerDeleteProcessor.getName());
+      }
+    }
+
+    if (zeroStoreClient != null) {
+      zeroStoreClient.shutdown();
+      zeroStoreClient = null;
+    }
+  }
+
+  @VisibleForTesting
+  void replaceMetadataCacheManager(MetadataCacheManager metadataCacheManager) {
+    this.metadataCacheManager = metadataCacheManager;
+  }
+
+  @VisibleForTesting
+  void replaceCorePullerFactory(CorePullerFactory corePullerFactory) {
+    this.corePullerFactory = corePullerFactory;
+  }
+
+  @VisibleForTesting
+  void replaceZeroShardMetadataController(ZeroMetadataController metadataController) {
+    this.zeroMetadataController = metadataController;
+  }
+
+  /**
+   * This is used for optimizations and skipping pull in some cases. Unclear if the cost of the
+   * optimization (running this method) is not higher than the potential savings.
+   */
+  public boolean isLeader(SolrCore core) {
+    try {
+      if (!coreContainer.isZooKeeperAware()) {
+        // not solr cloud
+        return false;
+      }
+
+      CoreDescriptor coreDescriptor = coreContainer.getCoreDescriptor(core.getName());
+      if (coreDescriptor == null) {
+        // core descriptor does not exist
+        return false;
+      }
+
+      CloudDescriptor cd = coreDescriptor.getCloudDescriptor();
+      if (cd == null || cd.getReplicaType() != Replica.Type.ZERO) {
+        // not a ZERO replica
+        return false;
+      }
+
+      ZkController zkController = coreContainer.getZkController();
+      Replica leaderReplica =
+          zkController.getZkStateReader().getLeaderRetry(cd.getCollectionName(), cd.getShardId());
+      // not a leader replica
+      return leaderReplica != null && cd.getCoreNodeName().equals(leaderReplica.getName());
+    } catch (Exception ex) {
+      log.warn(
+          String.format(
+              Locale.ROOT,
+              "Could not establish if current replica is leader for the given core, collection=%s shard=%s core=%s",
+              core.getCoreDescriptor().getCollectionName(),
+              core.getCoreDescriptor().getCloudDescriptor().getShardId(),
+              core.getName()),
+          ex);
+      // we will proceed further as we are not a leader
+    }
+    return false;
+  }
+
+  @VisibleForTesting
+  @FunctionalInterface
+  public interface CorePullerFactory {
+    CorePuller createPuller(
+        SolrCore solrCore,
+        ZeroStoreClient zeroStoreClient,
+        MetadataCacheManager metadataCacheManager,
+        ZeroMetadataController metadataController,
+        int maxFailedCorePullAttempts);
+  }
+
+  /**
+   * Returns the list of core properties that are needed to create a core corresponding to provided
+   * {@code replica} of the {@code collection}. These cores are ZERO cores found in ZooKeeper that
+   * do not have a local presence on the disk at startup. See {@link
+   * ZeroStoreManager#discoverAdditionalCoreDescriptorsForZeroReplicas}
+   */
+  public Map<String, String> getZeroCoreProperties(DocCollection collection, Replica replica) {
+    // "numShards" is a property that is found in core descriptors. But it is only set on the
+    // cores created at collection creation time. It is not part of cores created by addition of
+    // replicas/shards or shard splits. Once set, it is not even kept in sync with latest number
+    // of shards. Therefore, we do not put it in any of missing cores we create.
+
+    Map<String, String> params = new HashMap<>();
+    params.put(CoreDescriptor.CORE_COLLECTION, collection.getName());
+    params.put(CoreDescriptor.CORE_NODE_NAME, replica.getName());
+    params.put(
+        CoreDescriptor.CORE_SHARD,
+        collection.getShardId(replica.getNodeName(), replica.getCoreName()));
+    params.put(CloudDescriptor.REPLICA_TYPE, Replica.Type.ZERO.name());
+    params.put(CollectionAdminParams.COLL_CONF, collection.getConfigName());
+    return params;
+  }
+
+  /** True if the core has ever synced with the Zero store, otherwise, false. */
+  public boolean hasCoreEverSyncedWithZeroStore(SolrCore core) {
+    MetadataCacheManager.MetadataCacheEntry coreMetadata =
+        metadataCacheManager.getOrCreateCoreMetadata(core.getName());
+    // If the cached value has moved beyond the initialization value then we must have pushed
+    // (performed indexing as a leader) or pulled (including the no-op pull that happens when the
+    // shard is at ZeroShardMetadataController.METADATA_NODE_DEFAULT_VALUE )
+    if (!metadataCacheManager.hasInitialCacheSuffixValue(coreMetadata)) return true;
+
+    // If local cache doesn't know about any updates, check in ZK
+    ZeroMetadataVersion shardMetadataVersion =
+        zeroMetadataController.readMetadataValue(getCollectionName(core), getShardName(core));
+    return zeroMetadataController.hasDefaultNodeSuffix(shardMetadataVersion);
+  }
+
+  /** If the core has never synced with the Zero store then fail the query. */
+  public void ensureZeroCoreFreshness(SolrCore core) {
+    CloudDescriptor cloudDescriptor = core.getCoreDescriptor().getCloudDescriptor();
+    if (cloudDescriptor != null && cloudDescriptor.getReplicaType() == Replica.Type.ZERO) {
+      boolean hasCoreEverSynced = hasCoreEverSyncedWithZeroStore(core);
+      if (!hasCoreEverSynced) {
+        // The message in this exception is verified by some tests. Change here, change there...
+        throw new SolrException(
+            SolrException.ErrorCode.SERVICE_UNAVAILABLE,
+            core.getName()
+                + " is not fresh enough because it has never synced with the Zero store.");
+      }
+    }
+  }
+
+  public CorePullStatus pullCoreFromZeroStore(SolrCore core) throws SolrException {
+    try {
+      CorePuller corePuller = newCorePuller(core);
+      return corePuller.pullCoreWithRetries(
+          isLeader(core),
+          ZeroCoreIndexingBatchProcessor.SECONDS_TO_WAIT_PULL_LOCK,
+          getConfig().getCorePullRetryDelay());
+    } catch (ZeroException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e.getMessage(), e);
+    }
+  }
+
+  public void setIndexingBatchReceived(SolrCore core) {
+    getMetadataCacheManager()
+        .recordState(core, MetadataCacheManager.ZeroCoreStage.INDEXING_BATCH_RECEIVED);
+  }
+
+  public void setIndexingStarted(SolrCore core) {
+    getMetadataCacheManager()
+        .recordState(core, MetadataCacheManager.ZeroCoreStage.LOCAL_INDEXING_STARTED);
+  }
+
+  public void setIndexingFinished(SolrCore core) {
+    getMetadataCacheManager()
+        .recordState(core, MetadataCacheManager.ZeroCoreStage.INDEXING_BATCH_FINISHED);
+  }
+
+  public ZeroAccessLocks getZeroAccessLocks(String coreName) {
+    return getMetadataCacheManager().getOrCreateCoreMetadata(coreName).getZeroAccessLocks();
+  }
+
+  /**
+   * Acquires the indexing lock that must be held during the split process using the {@link
+   * ZeroCoreIndexingBatchProcessor#SECONDS_TO_WAIT_PULL_LOCK} timeout.
+   *
+   * <p>For indexing, the same lock is acquired in {@link
+   * ZeroCoreIndexingBatchProcessor#startIndexingBatch} using the actual indexing timeout.
+   */
+  public AutoCloseable acquireIndexingLockForSplit(SolrCore core) throws SolrException {
+    try {
+      // TODO Maybe use ZeroCoreIndexingBatchProcessor.SECONDS_TO_WAIT_INDEXING_LOCK instead?
+      return metadataCacheManager
+          .getOrCreateCoreMetadata(core.getName())
+          .getZeroAccessLocks()
+          .acquireIndexingLock(ZeroCoreIndexingBatchProcessor.SECONDS_TO_WAIT_PULL_LOCK);
+    } catch (ZeroLockException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          String.format(
+              Locale.ROOT,
+              "Failed to acquire indexing lock within %s seconds. "
+                  + "collection=%s shard=%s core=%s",
+              ZeroCoreIndexingBatchProcessor.SECONDS_TO_WAIT_PULL_LOCK,
+              getCollectionName(core),
+              getShardName(core),
+              core.getName()));
+    } catch (InterruptedException e) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          String.format(
+              Locale.ROOT,
+              "Thread interrupted while trying to acquire pull read lock."
+                  + " collection=%s shard=%s core=%s",
+              getCollectionName(core),
+              getShardName(core),
+              core.getName()));
+    }
+  }
+
+  public void initialCorePushToZeroStore(SolrCore core) throws InterruptedException {
+    CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
+    String collectionName = cloudDesc.getCollectionName();
+    String shardName = cloudDesc.getShardId();
+    String coreName = core.getName();
+
+    if (log.isInfoEnabled()) {
+      log.info(
+          "Attempting to push to Zero store for collection={} shard={} core={}",
+          collectionName,
+          shardName,
+          coreName);
+    }
+
+    CorePusher pusher = newCorePusher(core);
+
+    pusher.initialCorePushToZeroStore();
+
+    if (log.isInfoEnabled()) {
+      log.info(
+          "Successfully pushed to Zero store for collection={} shard={} core={}",
+          collectionName,
+          shardName,
+          coreName);
+    }
+  }
+
+  /**
+   * Enqueues the core (wrapped in a {@link CorePuller}) into the deduplicating list to be
+   * eventually pulled from Zero store.
+   *
+   * <p>Eventually, the {@link #asyncCorePullExecutor} will call {@link
+   * CorePuller#lockAndPullCore(boolean, long)}. The actual execution will happen in {@link
+   * CorePullerBlockingQueue.RunnableAsyncCorePull#run()}, built in {@link
+   * CorePullerBlockingQueue#take()}.
+   *
+   * <p>In case of pull errors (even obviously transient ones) there are no retries. Given the
+   * enqueue is triggered by queries, a subsequent query will trigger a new pull request. Note that
+   * upon completion, the {@link CorePuller#pullEndNotification} is called by calling {@link
+   * CorePuller#notifyPullEnd(CorePullStatus)}.
+   *
+   * <p>TODO more work needed because we shouldn't try to pull too often
+   */
+  public void enqueueCorePullFromZeroStore(SolrCore core) throws Exception {
+    CorePuller puller = newCorePuller(core);
+    corePullTaskQueue.addDeduplicated(puller, false);
+  }
+
+  @VisibleForTesting
+  void stopBackgroundCorePulling() {
+    // Stopping the executor will stop the async pulls.
+    // Tests do not need to restart the pulls, we're ok.
+    ExecutorUtil.shutdownAndAwaitTermination(asyncCorePullExecutor);
+  }
+
+  public void deleteShard(String collectionName, String shardName) throws SolrException {
+    zeroMetadataController.cleanUpMetadataNodes(collectionName, shardName);
+
+    CompletableFuture<DeleterTask.Result> deleteFuture =
+        overseerDeleteProcessor.deleteShard(collectionName, shardName, false);
+
+    DeleterTask.Result result = null;
+    Throwable t = null;
+    try {
+      // TODO: Find a reasonable timeout value
+      result = deleteFuture.get(60, TimeUnit.SECONDS);
+    } catch (Exception ex) {
+      t = ex;
+    }
+    if (t != null || !result.isSuccess()) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Could not complete deleting shard"
+              + shardName
+              + " from Zero store belonging to collection "
+              + collectionName
+              + ". Files belonging to this shard may be orphaned.",
+          t);
+    }
+  }
+
+  public void deleteCollection(String collectionName) throws SolrException {
+    // deletes all files belonging to this collection
+    CompletableFuture<DeleterTask.Result> deleteFuture =
+        overseerDeleteProcessor.deleteCollection(collectionName, false);
+
+    DeleterTask.Result result = null;
+    Throwable t = null;
+    try {
+      // TODO: Find a reasonable timeout value
+      result = deleteFuture.get(60, TimeUnit.SECONDS);
+    } catch (Exception ex) {
+      t = ex;
+    }
+    if (t != null || !result.isSuccess()) {
+      throw new SolrException(
+          SolrException.ErrorCode.SERVER_ERROR,
+          "Could not complete deleting collection "
+              + collectionName
+              + " from Zero store, files belonging to this collection"
+              + " may be orphaned.",
+          t);
+    }
... 10929 lines suppressed ...