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:45 UTC

(solr) branch jira/solr-17125-zero-replicas created (now a713415c0f8)

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

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


      at a713415c0f8 Initial code drop for ZERO replicas. SIP-20 and SOLR-17125

This branch includes the following new commits:

     new a713415c0f8 Initial code drop for ZERO replicas. SIP-20 and SOLR-17125

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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

Posted by il...@apache.org.
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 ...