You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2019/10/30 20:45:05 UTC

[lucene-solr] branch jira/SOLR-13101 updated (0e66bec -> 7c9063b)

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

yonik pushed a change to branch jira/SOLR-13101
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


    from 0e66bec  SOLR-13101: disable SharedStorageSplitTest.testLiveSplit
     new c7ad68f  @W-6475448 Allow CorePullerFeeder to be provisioned with a callback in tests and… (#386)
     new ce12e73  @W-6635251 Refactor async pull code to avoid static initialized data structures from colliding in unit tests (#390)
     new d63c032  @W-6587409 Make sure newly pulled blob contents are considered source… (#389)
     new 648d01e  Adding S3 dependencies back (#396)
     new 2b70913  @W-6650036 Fix pull data object mergers for dedup lists and add tests (#391)
     new d0c13cf  @W-6684857 Refactor HttpSolrCall to call missing pull correctly and refactor asy… (#398)
     new bf100b6  @W-6684857 Don't call shutdown multiple times (#399)
     new 5358453  @W-6684857 indentation cleanup, precommit error fixes, remove junit use in non-test classes (#400)
     new 581f468  @W-6587412 Concurrent indexing, pull and pushes (#403)
     new 1bc8d34  @W-6587412 (#406)
     new 7c9063b  @W-6782433 Shared replica skip recovery (#409)

The 11 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.


Summary of changes:
 .../multicore/core1/core.properties => Jenkinsfile |   0
 lucene/ivy-versions.properties                     |   1 +
 solr/core/ivy.xml                                  |   6 +-
 .../org/apache/solr/cloud/ElectionContext.java     | 157 ++---
 .../solr/handler/admin/RequestApplyUpdatesOp.java  |  35 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java | 154 ++---
 .../store/blob/DeletedCorruptCoreException.java    |   6 +-
 .../solr/store/blob/PullInProgressException.java   |   6 +-
 .../store/blob/client/BlobClientException.java     |  20 +-
 .../solr/store/blob/client/BlobCoreMetadata.java   | 472 ++++++++-------
 .../store/blob/client/BlobCoreMetadataBuilder.java | 156 ++---
 .../solr/store/blob/client/BlobException.java      |  18 +-
 .../store/blob/client/BlobServiceException.java    |  18 +-
 .../solr/store/blob/client/CoreStorageClient.java  | 248 ++++----
 .../apache/solr/store/blob/client/ToFromJson.java  |  32 +-
 .../solr/store/blob/metadata/BlobCoreSyncer.java   | 665 ++++++++++-----------
 .../solr/store/blob/metadata/CorePushPull.java     | 247 ++++----
 .../solr/store/blob/metadata/PushPullData.java     |  61 +-
 .../store/blob/metadata/ServerSideMetadata.java    | 421 ++++++++-----
 .../blob/metadata/SharedStoreResolutionUtil.java   | 106 +++-
 .../solr/store/blob/process/BlobDeleteManager.java |   5 +
 .../solr/store/blob/process/BlobProcessUtil.java   |  24 +-
 .../solr/store/blob/process/CorePullTask.java      | 200 ++++---
 .../solr/store/blob/process/CorePullTracker.java   |  44 +-
 .../solr/store/blob/process/CorePullerFeeder.java  |  92 ++-
 .../solr/store/blob/process/CorePullerThread.java  | 117 +++-
 .../apache/solr/store/blob/process/CorePusher.java | 187 ++++--
 .../solr/store/blob/process/CoreSyncFeeder.java    | 202 +++----
 .../solr/store/blob/process/CoreSyncStatus.java    |   2 -
 .../solr/store/blob/process/CoreUpdateTracker.java |  20 -
 .../solr/store/blob/util/BlobStoreUtils.java       | 122 ++--
 .../solr/store/blob/util/DeduplicatingList.java    | 294 ++++-----
 .../shared/SharedCoreConcurrencyController.java    | 351 +++++++++++
 .../solr/store/shared/SharedStoreManager.java      |  32 +-
 .../metadata/SharedShardMetadataController.java    | 117 ++--
 .../processor/DistributedZkUpdateProcessor.java    | 149 ++++-
 .../solr/store/blob/SharedStorageSplitTest.java    | 152 ++---
 .../store/blob/client/CoreStorageClientTest.java   |   2 +-
 .../store/blob/metadata/BlobCoreMetadataTest.java  |  13 +-
 .../solr/store/blob/metadata/CorePushPullTest.java | 139 ++++-
 .../blob/metadata/DeleteBlobStrategyTest.java      |  20 +-
 .../metadata/SharedStoreResolutionUtilTest.java    | 218 +++++--
 .../solr/store/blob/metadata/ToFromJsonTest.java   |  17 +-
 .../blob/process/PullMergeDeduplicationTest.java   | 223 +++++++
 .../solr/store/blob/util/BlobStoreUtilsTest.java   |  29 +-
 .../store/shared/SharedCoreConcurrencyTest.java    | 488 +++++++++++++++
 .../shared/SimpleSharedStoreEndToEndPullTest.java  |  65 +-
 .../shared/SimpleSharedStoreEndToEndPushTest.java  |   9 +-
 .../SimpleSharedStoreMissingCorePullTest.java      |  51 +-
 .../store/shared/SolrCloudSharedStoreTestCase.java |  27 +-
 .../SharedShardMetadataControllerTest.java         | 162 +----
 .../DistributedZkUpdateProcessorTest.java          |   3 +-
 52 files changed, 4101 insertions(+), 2304 deletions(-)
 copy solr/solrj/src/test-files/solrj/solr/multicore/core1/core.properties => Jenkinsfile (100%)
 create mode 100644 solr/core/src/java/org/apache/solr/store/shared/SharedCoreConcurrencyController.java
 create mode 100644 solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
 create mode 100644 solr/core/src/test/org/apache/solr/store/shared/SharedCoreConcurrencyTest.java


[lucene-solr] 03/11: @W-6587409 Make sure newly pulled blob contents are considered source… (#389)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d63c03224df89015d41721ea686de212791714ca
Author: Bilal Waheed <mw...@salesforce.com>
AuthorDate: Mon Sep 30 17:21:34 2019 -0400

    @W-6587409 Make sure newly pulled blob contents are considered source… (#389)
    
    @W-6587409 Make sure newly pulled blob contents are considered source of truth
    
    -When local index contents conflict with contents to be pulled from blob we move core to new index dir when pulling blob contents
    Two cases:
    1. local index is at higher generation number than blob's generation number
    2. same index file exist in both places with different size/checksum
---
 .../solr/store/blob/client/BlobCoreMetadata.java   |  84 +++++----
 .../store/blob/client/BlobCoreMetadataBuilder.java |  16 +-
 .../solr/store/blob/metadata/CorePushPull.java     | 185 +++++++++++++------
 .../store/blob/metadata/ServerSideMetadata.java    | 118 +++++++++---
 .../blob/metadata/SharedStoreResolutionUtil.java   |  94 +++++++---
 .../store/blob/client/CoreStorageClientTest.java   |   2 +-
 .../store/blob/metadata/BlobCoreMetadataTest.java  |  13 +-
 .../solr/store/blob/metadata/CorePushPullTest.java | 107 ++++++++++-
 .../blob/metadata/DeleteBlobStrategyTest.java      |  20 +-
 .../metadata/SharedStoreResolutionUtilTest.java    | 203 ++++++++++++++++-----
 .../solr/store/blob/metadata/ToFromJsonTest.java   |  17 +-
 11 files changed, 645 insertions(+), 214 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
index b234550..8bf2f2a 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
@@ -2,6 +2,7 @@ package org.apache.solr.store.blob.client;
 
 import java.util.Arrays;
 import java.util.HashSet;
+import java.util.Objects;
 import java.util.Set;
 import java.util.UUID;
 
@@ -20,6 +21,14 @@ public class BlobCoreMetadata {
     private final String sharedBlobName;
 
     /**
+     * Generation number of index represented by this metadata.
+     * This generation number is only meant to identify a scenario where local index generation number is higher than
+     * what we have in blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
+     * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+     */
+    private final long generation;
+
+    /**
      * Unique identifier of this metadata, that changes on every update to the metadata (except generating a new corrupt metadata
      * through {@link #getCorruptOf}).
      */
@@ -57,16 +66,17 @@ public class BlobCoreMetadata {
      * It always builds non "isCorrupt" and non "isDeleted" metadata. 
      * The only way to build an instance of "isCorrupt" metadata is to use {@link #getCorruptOf} and for "isDeleted" use {@link #getDeletedOf()}
      */
-    BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete) {
-        this(sharedBlobName, blobFiles, blobFilesToDelete, UUID.randomUUID().toString(), false,
+    BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation) {
+        this(sharedBlobName, blobFiles, blobFilesToDelete, generation, UUID.randomUUID().toString(), false,
                 false);
     }
 
-    private BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, 
+    private BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation,
         String uniqueIdentifier, boolean isCorrupt, boolean isDeleted) {
         this.sharedBlobName = sharedBlobName;
         this.blobFiles = blobFiles;
         this.blobFilesToDelete = blobFilesToDelete;
+        this.generation = generation;
         this.uniqueIdentifier = uniqueIdentifier;
         this.isCorrupt = isCorrupt;
         this.isDeleted = isDeleted;
@@ -78,7 +88,7 @@ public class BlobCoreMetadata {
      */
     public BlobCoreMetadata getCorruptOf() {
         assert !isCorrupt;
-        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, uniqueIdentifier, true, isDeleted);
+        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, true, isDeleted);
     }
 
     /**
@@ -88,7 +98,7 @@ public class BlobCoreMetadata {
      */
     public BlobCoreMetadata getDeletedOf() {
         assert !isDeleted;
-        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, uniqueIdentifier, isCorrupt, true);
+        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, isCorrupt, true);
     }
 
     /**
@@ -123,6 +133,10 @@ public class BlobCoreMetadata {
         return uniqueIdentifier;
     }
 
+    public long getGeneration() {
+        return this.generation;
+    }
+
     public BlobFile[] getBlobFiles() {
         return blobFiles;
     }
@@ -138,6 +152,7 @@ public class BlobCoreMetadata {
 
         BlobCoreMetadata that = (BlobCoreMetadata) o;
 
+        if (this.generation != that.generation) return false;
         if (this.isCorrupt != that.isCorrupt) return false;
         if (this.isDeleted != that.isDeleted) return false;
         if (!this.uniqueIdentifier.equals(that.uniqueIdentifier)) return false;
@@ -156,19 +171,17 @@ public class BlobCoreMetadata {
 
     @Override
     public int hashCode() {
-        int result = sharedBlobName.hashCode();
-        result = 31 * result + uniqueIdentifier.hashCode();
-        // The array of files is not ordered so need to compare as a set
-        result = 31 * result + new HashSet<>(Arrays.asList(this.blobFiles)).hashCode();
-        result = 31 * result + new HashSet<>(Arrays.asList(this.blobFilesToDelete)).hashCode();
-        result = 31 * result + (isCorrupt ? 1 : 0);
-        result = 31 * result + (isDeleted ? 1 : 0);
-        return result;
+        return Objects.hash(sharedBlobName, uniqueIdentifier, generation,
+            // The array of files is not ordered so need to compare as a set
+            new HashSet<>(Arrays.asList(this.blobFiles)).hashCode(),
+            new HashSet<>(Arrays.asList(this.blobFilesToDelete)).hashCode(),
+            isCorrupt, isDeleted);
     }
 
     @Override
     public String toString() {
-        return "sharedBlobName=" + sharedBlobName + " isCorrupt=" + isCorrupt + " uniqueIdentifier=" + uniqueIdentifier;
+        return "sharedBlobName=" + sharedBlobName +  " generation=" + generation 
+            + " isCorrupt=" + isCorrupt + " uniqueIdentifier=" + uniqueIdentifier;
     }
 
     /**
@@ -187,14 +200,18 @@ public class BlobCoreMetadata {
          */
         private final String blobName;
 
-        // TODO add some checksum here to verify blob files are not corrupt
-
         private final long fileSize;
 
-        public BlobFile(String solrFileName, String blobName, long fileSize) {
+        /**
+         * Lucene generated checksum of the file. It is used in addition to file size to compare local and blob files.
+         */
+        private final long checksum;
+
+        public BlobFile(String solrFileName, String blobName, long fileSize, long checksum) {
             this.solrFileName = solrFileName;
             this.blobName = blobName;
             this.fileSize = fileSize;
+            this.checksum = checksum;
         }
 
         @Override
@@ -202,19 +219,17 @@ public class BlobCoreMetadata {
             if (this == o) return true;
             if (o == null || getClass() != o.getClass()) return false;
 
-            BlobFile blobFile = (BlobFile) o;
+            BlobFile other = (BlobFile) o;
 
-            if (fileSize != blobFile.fileSize) return false;
-            if (!solrFileName.equals(blobFile.solrFileName)) return false;
-            return blobName.equals(blobFile.blobName);
+            return Objects.equals(solrFileName, other.solrFileName) &&
+                Objects.equals(blobName, other.blobName) &&
+                Objects.equals(checksum, other.checksum) &&
+                Objects.equals(fileSize, other.fileSize);
         }
 
         @Override
         public int hashCode() {
-            int result = solrFileName.hashCode();
-            result = 31 * result + blobName.hashCode();
-            result = 31 * result + (int) (fileSize ^ (fileSize >>> 32));
-            return result;
+            return Objects.hash(solrFileName, blobName, fileSize, checksum);
         }
 
         public String getSolrFileName() {
@@ -228,7 +243,10 @@ public class BlobCoreMetadata {
         public long getFileSize() {
             return this.fileSize;
         }
-        
+
+        public long getChecksum() {
+            return this.checksum;
+        }
     }
 
     /**
@@ -247,14 +265,14 @@ public class BlobCoreMetadata {
       // likelyhood of a really really slow update by another server causing a race is low).
       private final long deletedAt;
 
-      public BlobFileToDelete(String solrFileName, String blobName, long fileSize, long deletedAt) {
-        super(solrFileName, blobName, fileSize);
+      public BlobFileToDelete(String solrFileName, String blobName, long fileSize, long checksum, long deletedAt) {
+        super(solrFileName, blobName, fileSize, checksum);
 
         this.deletedAt = deletedAt;
       }
 
       public BlobFileToDelete(BlobFile bf, long deletedAt) {
-        super(bf.solrFileName, bf.blobName, bf.fileSize);
+        super(bf.solrFileName, bf.blobName, bf.fileSize, bf.checksum);
 
         this.deletedAt = deletedAt;
       }
@@ -265,16 +283,14 @@ public class BlobCoreMetadata {
         if (o == null || getClass() != o.getClass()) return false;
         if (!super.equals(o)) return false;
 
-        BlobFileToDelete that = (BlobFileToDelete) o;
+        BlobFileToDelete other = (BlobFileToDelete) o;
 
-        return deletedAt == that.deletedAt;
+        return deletedAt == other.deletedAt;
       }
 
       @Override
       public int hashCode() {
-        int result = super.hashCode();
-        result = 31 * result + (int) (deletedAt ^ (deletedAt >>> 32));
-        return result;
+          return Objects.hash(super.hashCode(), deletedAt);
       }
 
       public long getDeletedAt() {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
index a8c2676..bc43464 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
@@ -6,22 +6,30 @@ import java.util.*;
  * Builder for {@link BlobCoreMetadata}.
  */
 public class BlobCoreMetadataBuilder {
+    /**
+     * Generation number in metadata of cores not existing on the Blob Store.
+     */
+    public static final long UNDEFINED_VALUE = -1L;
 
     final private String sharedBlobName;
+    final private long generation;
     final private Set<BlobCoreMetadata.BlobFile> blobFiles;
     final private Set<BlobCoreMetadata.BlobFileToDelete> blobFilesToDelete;
 
-    public BlobCoreMetadataBuilder(String sharedBlobName) {
+    public BlobCoreMetadataBuilder(String sharedBlobName, long generation) {
         this.sharedBlobName = sharedBlobName;
+        this.generation= generation;
         this.blobFiles = new HashSet<>();
         this.blobFilesToDelete = new HashSet<>();
     }
 
     /**
      * Builder used for "cloning" then modifying an existing instance of {@link BlobCoreMetadata}.
+     * The new generation has to be passed in because it is final and can't be set later.
      */
-    public BlobCoreMetadataBuilder(BlobCoreMetadata bcm) {
+    public BlobCoreMetadataBuilder(BlobCoreMetadata bcm, long generation) {
         this.sharedBlobName = bcm.getSharedBlobName();
+        this.generation = generation;
         this.blobFiles = new HashSet<>(Arrays.asList(bcm.getBlobFiles()));
         this.blobFilesToDelete = new HashSet<>(Arrays.asList(bcm.getBlobFilesToDelete()));
     }
@@ -34,7 +42,7 @@ public class BlobCoreMetadataBuilder {
      * Builds a {@link BlobCoreMetadata} for a non existing core of a given name.
      */
     static public BlobCoreMetadata buildEmptyCoreMetadata(String sharedBlobName) {
-        return (new BlobCoreMetadataBuilder(sharedBlobName)).build();
+        return (new BlobCoreMetadataBuilder(sharedBlobName, UNDEFINED_VALUE)).build();
     }
 
     /**
@@ -90,6 +98,6 @@ public class BlobCoreMetadataBuilder {
         BlobCoreMetadata.BlobFile[] blobFilesArray = this.blobFiles.toArray(new BlobCoreMetadata.BlobFile[this.blobFiles.size()]);
         BlobCoreMetadata.BlobFileToDelete[] blobFilesToDeleteArray = this.blobFilesToDelete.toArray(new BlobCoreMetadata.BlobFileToDelete[this.blobFilesToDelete.size()]);
 
-        return new BlobCoreMetadata(this.sharedBlobName, blobFilesArray, blobFilesToDeleteArray);
+        return new BlobCoreMetadata(this.sharedBlobName, blobFilesArray, blobFilesToDeleteArray, generation);
     }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
index 8245651..a96c5a5 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
@@ -13,6 +13,7 @@ import java.util.concurrent.Future;
 import java.util.stream.Collectors;
 
 import org.apache.commons.io.IOUtils;
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -110,7 +111,7 @@ public class CorePushPull {
 
         try {
           // Creating the new BlobCoreMetadata as a modified clone of the existing one
-          BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder(blobMetadata);
+          BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder(blobMetadata, solrServerMetadata.getGeneration());
 
           // First copy index files over to a temp directory and then push to blob store from there. 
           // This is to avoid holding a lock over index directory involving network operation.
@@ -132,14 +133,14 @@ public class CorePushPull {
           // that can potentially be stopped in the middle because of a concurrent merge deleting the segment files being pushed. 
 
           // create a temp directory (within the core local folder).
-          String tempIndexDirName = solrCore.getDataDir() + "index.push." + System.nanoTime();
-          Directory tempIndexDir = solrCore.getDirectoryFactory().get(tempIndexDirName, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
+          String tempIndexDirPath = solrCore.getDataDir() + "index.push." + System.nanoTime();
+          Directory tempIndexDir = solrCore.getDirectoryFactory().get(tempIndexDirPath, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
           try {
             Directory indexDir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
             try {
               // copy index files to the temp directory
               for (CoreFileData cfd : resolvedMetadataResult.getFilesToPush()) {
-                copyFileToDirectory(indexDir, cfd.fileName, tempIndexDir);
+                copyFileToDirectory(indexDir, cfd.getFileName(), tempIndexDir);
               }
             } finally {
               solrCore.getDirectoryFactory().release(indexDir);
@@ -176,7 +177,8 @@ public class CorePushPull {
               
               String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(pushPullData.getLastReadMetadataSuffix());
               String coreMetadataPath = blobMetadata.getSharedBlobName() + "/" + blobCoreMetadataName;
-              BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete("", coreMetadataPath, bcmSize, System.currentTimeMillis());
+              // so far checksum is not used for metadata file
+              BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete("", coreMetadataPath, bcmSize, BlobCoreMetadataBuilder.UNDEFINED_VALUE, System.currentTimeMillis());
               bcmBuilder.addFileToDelete(bftd);
             }
             
@@ -184,13 +186,21 @@ public class CorePushPull {
             // But this is untrue/totally false/misleading. SnapPuller has File all over.
             for (CoreFileData cfd : resolvedMetadataResult.getFilesToPush()) {
               // Sanity check that we're talking about the same file (just sanity, Solr doesn't update files so should never be different)
-              assert cfd.fileSize == tempIndexDir.fileLength(cfd.fileName);
+              assert cfd.getFileSize() == tempIndexDir.fileLength(cfd.getFileName());
 
-              String blobPath = pushFileToBlobStore(coreStorageClient, tempIndexDir, cfd.fileName, cfd.fileSize);
-              bcmBuilder.addFile(new BlobCoreMetadata.BlobFile(cfd.fileName, blobPath, cfd.fileSize));
+              String blobPath = pushFileToBlobStore(coreStorageClient, tempIndexDir, cfd.getFileName(), cfd.getFileSize());
+              bcmBuilder.addFile(new BlobCoreMetadata.BlobFile(cfd.getFileName(), blobPath, cfd.getFileSize(), cfd.getChecksum()));
             }
           } finally {
-            removeTempDirectory(solrCore, tempIndexDirName, tempIndexDir);
+            try {
+              // Remove temp directory
+              solrCore.getDirectoryFactory().doneWithDirectory(tempIndexDir);
+              solrCore.getDirectoryFactory().remove(tempIndexDir);
+            } catch (Exception e) {
+              log.warn("Cannot remove temp directory " + tempIndexDirPath, e);
+            } finally {
+              solrCore.getDirectoryFactory().release(tempIndexDir);
+            }
           }
           
           // delete what we need
@@ -206,7 +216,7 @@ public class CorePushPull {
         }
       } finally {
         long filesAffected = resolvedMetadataResult.getFilesToPush().size();
-        long bytesTransferred = resolvedMetadataResult.getFilesToPush().stream().mapToLong(cfd -> cfd.fileSize).sum();
+        long bytesTransferred = resolvedMetadataResult.getFilesToPush().stream().mapToLong(cfd -> cfd.getFileSize()).sum();
         
         // todo correctness stuff
         logBlobAction("PUSH", filesAffected, bytesTransferred, requestQueuedTimeMs, attempt, startTimeMs);
@@ -226,7 +236,6 @@ public class CorePushPull {
 
     /**
      * We're doing here what replication does in {@link org.apache.solr.handler.IndexFetcher#fetchLatestIndex}.<p>
-     *     TODO: check changes in Solr.7's IndexFetcher. Core reloading needed?
      *
      * This method will work in 2 cases:
      * <ol>
@@ -255,58 +264,72 @@ public class CorePushPull {
           if (solrCore == null) {
             throw new Exception("Can't find core " + pushPullData.getCoreName());
           }
-
+          // if there is a conflict between local and blob contents we will move the core to a new index directory
+          final boolean createNewIndexDir = resolvedMetadataResult.isLocalConflictingWithBlob();
+          boolean coreSwitchedToNewIndexDir = false;
           try {
-            // We're here because we identified no conflicts in downloading from the Blob store the files missing locally.
-            // In order to make sure there still are no conflicts (local Solr server on which we run might have updated the
-            // core since we checked or might do so as we download files shortly), we'll download the needed files to a temp
-            // dir and before moving them to the core directory, we will check the directory hasn't changed locally.
-
             // Create temp directory (within the core local folder).
-            String tmpIdxDirName = solrCore.getDataDir() + "index.pull." + System.nanoTime();
-            Directory tmpIndexDir = solrCore.getDirectoryFactory().get(tmpIdxDirName, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
+            // 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 blob store into this temp directory.
+            // Then we will take a lock over index directory and move files from temp directory to index directory. This is to avoid
+            // involving a network operation within an index directory lock.
+            String tempIndexDirName = "index.pull." + System.nanoTime();
+            String tempIndexDirPath = solrCore.getDataDir() + tempIndexDirName;
+            Directory tempIndexDir = solrCore.getDirectoryFactory().get(tempIndexDirPath, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
             try {
-              downloadFilesFromBlob(tmpIndexDir, resolvedMetadataResult.getFilesToPull());
+              String indexDirPath = solrCore.getIndexDir();
+              Collection<BlobFile> filesToDownload;
+              if (createNewIndexDir) {
+                // This is an optimization to not download everything from blob if possible
+                // This made sense for some rolling start scenario in TLOG replicas and makes here too
+                // https://issues.apache.org/jira/browse/SOLR-11920
+                // https://issues.apache.org/jira/browse/SOLR-11815
+                // TODO: We might want to skip this optimization when healing a locally corrupt core
+                Directory indexDir = solrCore.getDirectoryFactory().get(indexDirPath, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
+                try {
+                  filesToDownload = initializeNewIndexDirWithLocallyAvailableFiles(indexDir, tempIndexDir);
+                } finally {
+                  solrCore.getDirectoryFactory().release(indexDir);
+                }
+              } else {
+                filesToDownload = resolvedMetadataResult.getFilesToPull();
+              }
+              downloadFilesFromBlob(tempIndexDir, filesToDownload);
 
-              String indexDir = solrCore.getIndexDir();
-              Directory dir = solrCore.getDirectoryFactory().get(indexDir, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
-              
+              Directory indexDir = solrCore.getDirectoryFactory().get(indexDirPath, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
               try {
-                // Close the index writer to stop changes to this core
-                solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(solrCore, true);
+                if (!createNewIndexDir) {
+                  // Close the index writer to stop changes to this core
+                  solrCore.getUpdateHandler().getSolrCoreState().closeIndexWriter(solrCore, true);
+                }
 
                 boolean thrownException = false;
                 try {
                   // Make sure Solr core directory content hasn't changed since we decided what we want to pull from Blob
-                  if (!solrServerMetadata.isSameDirectoryContent(dir)) {
+                  if (!solrServerMetadata.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 " + indexDir + " has changed since Blob pull started. Aborting pull.");
+                    throw new Exception("Local Directory content " + indexDirPath + " has changed since Blob pull started. Aborting pull.");
                   }
 
-                  // Copy all files into the Solr directory (there are no naming conflicts since we're doing an Action.PULL)
-                  // Move the segments_N file last once all other are ok.
-                  String segmentsN = null;
-                  for (BlobCoreMetadata.BlobFile bf : resolvedMetadataResult.getFilesToPull()) {
-                    if (SharedStoreResolutionUtil.isSegmentsNFilename(bf)) {
-                      assert segmentsN == null;
-                      segmentsN = bf.getSolrFileName();
-                    } else {
-                      // Copy all non segments_N files
-                      moveFileToDirectory(solrCore, tmpIndexDir, bf.getSolrFileName(), dir);
-                    }
+                  if (createNewIndexDir) {
+                    // point index to the new directory
+                    coreSwitchedToNewIndexDir = solrCore.modifyIndexProps(tempIndexDirName);
+                  } else {
+                    moveFilesFromTempToIndexDir(solrCore, tempIndexDir, indexDir);
                   }
-                  assert segmentsN != null;
-                  // Copy segments_N file. From this point on the local core might be accessed and is up to date with Blob content
-                  moveFileToDirectory(solrCore, tmpIndexDir, segmentsN, dir);
                 } catch (Exception e) {
                   // Used in the finally below to not mask an exception thrown from the try block above
                   thrownException = true;
                   throw e;
                 } finally {
                   try {
-                    // The closed index writer must be opened back (in the finally bloc)
                     // TODO this has been observed to throw org.apache.lucene.index.CorruptIndexException on certain types of corruptions in Blob Store. We need to handle this correctly (maybe we already do).
-                    solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(solrCore);
+                    if (!createNewIndexDir) {
+                      // The closed index writer must be opened back (in the finally bloc)
+                      solrCore.getUpdateHandler().getSolrCoreState().openIndexWriter(solrCore);
+                    } else if (coreSwitchedToNewIndexDir) {
+                      solrCore.getUpdateHandler().newIndexWriter(true);
+                    }
                   } catch (IOException ioe) {
                     // TODO corrupt core handling happened here
                     // CorruptCoreHandler.notifyBlobPullFailure(container, coreName, blobMetadata);
@@ -317,10 +340,28 @@ public class CorePushPull {
                   }
                 }
               } finally {
-                  solrCore.getDirectoryFactory().release(dir);
+                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 {
-                removeTempDirectory(solrCore, tmpIdxDirName, tmpIndexDir);
+              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 {
@@ -353,7 +394,44 @@ public class CorePushPull {
         }
     }
 
-    /**
+  private void moveFilesFromTempToIndexDir(SolrCore solrCore, Directory tmpIndexDir, Directory dir) throws IOException {
+    // Copy all files into the Solr directory
+    // Move the segments_N file last once all other are ok.
+    String segmentsN = null;
+    for (BlobFile bf : resolvedMetadataResult.getFilesToPull()) {
+      if (SharedStoreResolutionUtil.isSegmentsNFilename(bf)) {
+        assert segmentsN == null;
+        segmentsN = bf.getSolrFileName();
+      } else {
+        // Copy all non segments_N files
+        moveFileToDirectory(solrCore, 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 Blob content
+    moveFileToDirectory(solrCore, tmpIndexDir, segmentsN, dir);
+  }
+
+  private Collection<BlobFile> initializeNewIndexDirWithLocallyAvailableFiles(Directory indexDir, Directory newIndexDir) {
+    Collection<BlobFile> filesToDownload = new HashSet<>();
+      for (BlobFile blobFile : resolvedMetadataResult.getFilesToPull()) {
+        try (final IndexInput indexInput = indexDir.openInput(blobFile.getSolrFileName(), IOContext.READONCE)) {
+          long length = indexInput.length();
+          long checksum  = CodecUtil.retrieveChecksum(indexInput);
+          if (length == blobFile.getFileSize() && checksum == blobFile.getChecksum()) {
+            copyFileToDirectory(indexDir, blobFile.getSolrFileName(), newIndexDir);
+          } else {
+            filesToDownload.add(blobFile);
+          }
+        } catch (Exception ex){
+          // Either file does not exist locally or copy not succeeded, we will download from blob store
+          filesToDownload.add(blobFile);
+        }
+      }
+    return filesToDownload;
+  }
+
+  /**
      * Pushes a local file to blob store and returns a unique path to newly created blob  
      */
     @VisibleForTesting
@@ -366,13 +444,13 @@ public class CorePushPull {
         return blobPath;
     }
 
-    private void removeTempDirectory(SolrCore solrCore, String tempDirName, Directory tempDir) throws IOException {
+    private void removeTempDirectory(SolrCore solrCore, String tempDirPath, Directory tempDir) throws IOException {
         try {
             // Remove temp directory
             solrCore.getDirectoryFactory().doneWithDirectory(tempDir);
             solrCore.getDirectoryFactory().remove(tempDir);
         } catch (Exception e) {
-            log.warn("Cannot remove temp directory " + tempDirName, e);
+            log.warn("Cannot remove temp directory " + tempDirPath, e);
         } finally {
             solrCore.getDirectoryFactory().release(tempDir);
         }
@@ -390,11 +468,12 @@ public class CorePushPull {
       long now = System.currentTimeMillis();
       long runTime = now - startTimeMs;
       long startLatency = now - requestQueuedTimeMs;
- 
+
       String message = String.format("PushPullData=[%s] action=%s storageProvider=%s bucketRegion=%s bucketName=%s "
-        + "runTime=%s startLatency=%s bytesTransferred=%s attempt=%s filesAffected=%s",
-        pushPullData.toString(), action, coreStorageClient.getStorageProvider().name(), coreStorageClient.getBucketRegion(), 
-        coreStorageClient.getBucketName(), runTime, startLatency, bytesTransferred, attempt, filesAffected);
+              + "runTime=%s startLatency=%s bytesTransferred=%s attempt=%s filesAffected=%s localGenerationNumber=%s blobGenerationNumber=%s ",
+          pushPullData.toString(), action, coreStorageClient.getStorageProvider().name(), coreStorageClient.getBucketRegion(),
+          coreStorageClient.getBucketName(), runTime, startLatency, bytesTransferred, attempt, filesAffected,
+          solrServerMetadata.getGeneration(), blobMetadata.getGeneration());
       log.info(message);
     }
 
@@ -423,6 +502,7 @@ public class CorePushPull {
      * 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);
     }
 
@@ -431,6 +511,7 @@ public class CorePushPull {
      */
     private void moveFileToDirectory(SolrCore solrCore, 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);
     }
     
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
index 7e2d3e0..bd3562b 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
@@ -5,11 +5,16 @@ import java.io.IOException;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
+import java.util.List;
 import java.util.Objects;
 
 import org.apache.commons.codec.binary.Hex;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrCore;
@@ -34,7 +39,13 @@ public class ServerSideMetadata {
      * Files composing the core. They are are referenced from the core's current commit point's segments_N file
      * which is ALSO included in this collection.
      */
-    private final ImmutableCollection<CoreFileData> files;
+    private final ImmutableCollection<CoreFileData> latestCommitFiles;
+
+    /**
+     * Index files related to current and previous commit points(if any).
+     * These files do not matter when pushing contents to blob but they do matter if blob content being pulled conflicts with them.
+     */
+    private final ImmutableCollection<CoreFileData> allCommitsFiles;
 
     /**
      * Hash of the directory content used to make sure the content doesn't change as we proceed to pull new files from Blob
@@ -42,6 +53,13 @@ public class ServerSideMetadata {
      */
     private final 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 blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
+     * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+     */
+    private final long generation;
     private final SolrCore core;
     private final String coreName;
     private final CoreContainer container;
@@ -62,14 +80,17 @@ public class ServerSideMetadata {
         }
 
         try {
-            IndexCommit commit = core.getDeletionPolicy().getLatestCommit();
-            if (commit == null) {
+            IndexCommit latestCommit = core.getDeletionPolicy().getLatestCommit();
+            if (latestCommit == null) {
                 throw new BlobException("Core " + coreName + " has no available commit point");
             }
 
+            generation = latestCommit.getGeneration();
+
             // Work around possible bug returning same file multiple times by using a set here
             // See org.apache.solr.handler.ReplicationHandler.getFileList()
-            ImmutableCollection.Builder<CoreFileData> builder = new ImmutableSet.Builder<>();
+            ImmutableCollection.Builder<CoreFileData> latestCommitBuilder = new ImmutableSet.Builder<>();
+            ImmutableCollection.Builder<CoreFileData> allCommitsBuilder;
 
             Directory coreDir = core.getDirectoryFactory().get(core.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
             try {
@@ -78,20 +99,45 @@ public class ServerSideMetadata {
                 // with the download.
                 directoryHash = getSolrDirectoryHash(coreDir);
 
-                for (String fileName : commit.getFileNames()) {
-                    // Note we add here all segment related files as well as the commit point's segments_N file
-                    // Note commit points do not contain lock (write.lock) files.
-                    builder.add(new CoreFileData(fileName, coreDir.fileLength(fileName)));
+                buildCommitFiles(coreDir, latestCommit, latestCommitBuilder);
+
+                // A note on listCommits says that it does not guarantee consistent results if a commit is in progress.
+                // But in blob context we serialize commits and pulls by proper locking therefore we should be good here.
+                List<IndexCommit> allCommits = DirectoryReader.listCommits(coreDir);
+
+                // optimization:  normally we would only be dealing with one commit point. In that case just reuse latest commit files builder.
+                if (allCommits.size() > 1 ){
+                    allCommitsBuilder = new ImmutableSet.Builder<>();
+                    for (IndexCommit commit: allCommits) {
+                        buildCommitFiles(coreDir, commit, allCommitsBuilder);
+                    }
+                } else {
+                    // we should always have a commit point as verified in the beginning of this method.
+                    assert allCommits.size() == 1 && allCommits.get(0).equals(latestCommit);
+                    allCommitsBuilder = latestCommitBuilder;
                 }
             } finally {
                 core.getDirectoryFactory().release(coreDir);
             }
-            files = builder.build();
+            latestCommitFiles = latestCommitBuilder.build();
+            allCommitsFiles = allCommitsBuilder.build();
         } finally {
             core.close();
         }
     }
 
+    private void buildCommitFiles(Directory coreDir, IndexCommit commit, ImmutableCollection.Builder<CoreFileData> builder) throws IOException {
+        for (String fileName : commit.getFileNames()) {
+            // Note we add here all segment related files as well as the commit point's segments_N file
+            // Note commit points do not contain lock (write.lock) files.
+            try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
+                long length = indexInput.length();
+                long checksum = CodecUtil.retrieveChecksum(indexInput);
+                builder.add(new CoreFileData(fileName, length, checksum));
+            }
+        }
+    }
+
     public String getCoreName() {
         return this.coreName;
     }
@@ -100,12 +146,20 @@ public class ServerSideMetadata {
         return this.container;
     }
 
+    public long getGeneration() {
+        return this.generation;
+    }
+
     public String getDirectoryHash() {
         return this.directoryHash;
     }
 
-    public ImmutableCollection<CoreFileData> getFiles(){
-        return this.files;
+    public ImmutableCollection<CoreFileData> getLatestCommitFiles(){
+        return this.latestCommitFiles;
+    }
+
+    public ImmutableCollection<CoreFileData> getAllCommitsFiles() {
+        return this.allCommitsFiles;
     }
 
     /**
@@ -149,18 +203,28 @@ public class ServerSideMetadata {
         return hash;
     }
 
+    @Override
+    public String toString() {
+        return "collectionName=" + core.getCoreDescriptor().getCollectionName() +
+            " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() +
+            " coreName=" + core.getName() +
+            " generation=" + generation;
+    }
+
     /**
      * Information we capture per local core file (segments_N file *included*)
      */
     public static class CoreFileData {
         /** Local file name, no path */
-        public final String fileName;
+        private final String fileName;
         /** Size in bytes */
-        public final long fileSize;
+        private final long fileSize;
+        private final long checksum;
 
-        CoreFileData(String fileName, long fileSize) {
+        CoreFileData(String fileName, long fileSize, long checksum) {
             this.fileName = fileName;
             this.fileSize = fileSize;
+            this.checksum = checksum;
         }
 
         @Override
@@ -170,21 +234,27 @@ public class ServerSideMetadata {
 
             CoreFileData other = (CoreFileData) o;
 
-            return Objects.equals(fileSize, other.fileSize) && 
-                    Objects.equals(fileName, other.fileName);
+            return Objects.equals(fileName, other.fileName) &&
+                Objects.equals(fileSize, other.fileSize) &&
+                Objects.equals(checksum, other.checksum);
+        }
+
+        public String getFileName() {
+            return fileName;
+        }
+
+        public long getFileSize() {
+            return fileSize;
+        }
+
+        public long getChecksum() {
+            return checksum;
         }
 
         @Override
         public int hashCode() {
-            return Objects.hash(fileName, fileSize);
+            return Objects.hash(fileName, fileSize, checksum);
         }
-
     }
 
-    @Override
-    public String toString() {
-        return "collectionName=" + core.getCoreDescriptor().getCollectionName() +
-            " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() + 
-            " coreName=" + core.getName();
-    }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
index f96463d..2bc8054 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
@@ -1,5 +1,7 @@
 package org.apache.solr.store.blob.metadata;
 
+import java.lang.invoke.MethodHandles;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -9,13 +11,18 @@ import org.apache.solr.common.SolrException;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.BlobCoreMetadata.BlobFile;
 import org.apache.solr.store.blob.metadata.ServerSideMetadata.CoreFileData;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Utility class used to compare local {@link ServerSideMetadata} and remote 
  * {@link BlobCoreMetadata}, metadata of shard index data on the local solr node
  * and remote shared store (blob).
  */
-public class SharedStoreResolutionUtil {  
+public class SharedStoreResolutionUtil {
+
+  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   private static String SEGMENTS_N_PREFIX = "segments_";
 
   public static class SharedMetadataResolutionResult {
@@ -23,25 +30,29 @@ public class SharedStoreResolutionUtil {
     private final Collection<CoreFileData> filesToPush;
     // blob files needed to be pulled
     private final Collection<BlobFile> filesToPull;
+    // Whether the local index contents conflict with contents to be pulled from blob. If they do we will move the
+    // core to new index dir when pulling blob contents
+    // Two cases:
+    //  1. local index is at higher generation number than blob's generation number
+    //  2. same index file exist in both places with different size/checksum
+    private final boolean localConflictingWithBlob;
     
-    public SharedMetadataResolutionResult(Collection<CoreFileData> localFilesMissingOnBlob, 
-        Collection<BlobFile> blobFilesMissingLocally) {
-      if (localFilesMissingOnBlob == null) {
+    
+    public SharedMetadataResolutionResult(Collection<CoreFileData> filesToPush, 
+        Collection<BlobFile> filesToPull, boolean localConflictingWithBlob) {
+      if (filesToPush == null) {
         this.filesToPush = Collections.emptySet();
       } else {
-        this.filesToPush = localFilesMissingOnBlob;
+        this.filesToPush = filesToPush;
       }
       
-      if (blobFilesMissingLocally == null) {
+      if (filesToPull == null) {
         this.filesToPull = Collections.emptySet();
       } else {
-        this.filesToPull = blobFilesMissingLocally;
+        this.filesToPull = filesToPull;
       }
-    }
-    
-    public SharedMetadataResolutionResult(Map<String, CoreFileData> localFilesMissingOnBlob, 
-        Map<String, BlobFile> blobFilesMissingLocally) {
-      this(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values());
+
+      this.localConflictingWithBlob = localConflictingWithBlob;
     }
     
     public Collection<CoreFileData> getFilesToPush() {
@@ -51,6 +62,10 @@ public class SharedStoreResolutionUtil {
     public Collection<BlobFile> getFilesToPull() {
       return filesToPull;
     }
+    
+    public boolean isLocalConflictingWithBlob(){
+      return localConflictingWithBlob;
+    }
   }
   
   private SharedStoreResolutionUtil() {}
@@ -67,6 +82,7 @@ public class SharedStoreResolutionUtil {
   public static SharedMetadataResolutionResult resolveMetadata(ServerSideMetadata local, BlobCoreMetadata distant) {
     Map<String, CoreFileData> localFilesMissingOnBlob = new HashMap<>();
     Map<String, BlobFile> blobFilesMissingLocally = new HashMap<>();
+    Map<String, CoreFileData> allLocalFiles = new HashMap<>();
     
     if (local == null && distant == null) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Cannot resolve if both local and remote metadata is null"); 
@@ -74,8 +90,15 @@ public class SharedStoreResolutionUtil {
     
     if (local != null) {
       // Prepare local files for lookup by file name
-      for (CoreFileData cfd : local.getFiles()) {
-          localFilesMissingOnBlob.put(cfd.fileName, cfd);
+      
+      // for files to push only the current commit point matters
+      for (CoreFileData cfd : local.getLatestCommitFiles()) {
+          localFilesMissingOnBlob.put(cfd.getFileName(), cfd);
+      }
+
+      // for files to pull all the index files present locally matters
+      for (CoreFileData cfd : local.getAllCommitsFiles()) {
+        allLocalFiles.put(cfd.getFileName(), cfd);
       }
       // TODO we're not dealing here with local core on Solr server being corrupt. Not part of PoC at this stage but eventually need a solution
       // (fetch from Blob unless Blob corrupt as well...)
@@ -85,7 +108,7 @@ public class SharedStoreResolutionUtil {
         || distant.getBlobFiles().length == 0) {
       // The shard index data does not exist on the shared store. All we can do is push. 
       // We've computed localFilesMissingOnBlob above, and blobFilesMissingLocally is empty as it should be.
-      return new SharedMetadataResolutionResult(localFilesMissingOnBlob, blobFilesMissingLocally);
+      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), false);
     }
     
     // Verify we find one and only one segments_N file to download from Blob.
@@ -111,30 +134,43 @@ public class SharedStoreResolutionUtil {
     }
     
     if (local == null) {
-      // The shard index data does not exist on the shared store. All we can do is pull.  
+      // The shard index data does not exist locally. All we can do is pull.  
       // We've computed blobFilesMissingLocally and localFilesMissingOnBlob is empty as it should be.
-      return new SharedMetadataResolutionResult(localFilesMissingOnBlob, blobFilesMissingLocally);
+      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), false);
     }
-    
+
+    boolean localConflictingWithBlob = false;
     // Verify there are no inconsistencies between local index and blob index files
     for (BlobFile bf : distant.getBlobFiles()) {
       // We remove from map of local files those already present remotely since they don't have to be pushed.
-      CoreFileData cf = localFilesMissingOnBlob.remove(bf.getSolrFileName());
+      localFilesMissingOnBlob.remove(bf.getSolrFileName());
+      CoreFileData cf = allLocalFiles.get(bf.getSolrFileName());
       if (cf != null) {
-        // The blob file is present locally (by virtue of having been in localFilesMissingOnBlob initialized with
-        // all local files). Check if there is a conflict between local and distant (blob) versions of that file.
+        // The blob file is present locally. Check if there is a conflict between local and distant (blob) versions of that file.
         blobFilesMissingLocally.remove(bf.getSolrFileName());
-        // Later we could add checksum verification etc. here
-        if (cf.fileSize != bf.getFileSize()) {
-          // TODO - for now just log and propagate the error up
-          String errorMessage = "Size conflict for shared shard name: " + distant.getSharedBlobName() +". File " + 
-              bf.getSolrFileName() + " (Blob name " + bf.getBlobName() + ") local size " + 
-              cf.fileSize + " blob size " + bf.getFileSize() + " (core " + local.getCoreName() + ")";
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, errorMessage);
+        if (cf.getFileSize() != bf.getFileSize() || cf.getChecksum() != bf.getChecksum()) {
+          String message = String.format("Size/Checksum conflicts sharedShardName=%s coreName=%s fileName=%s blobName=%s" +
+                  " localSize=%s blobSize=%s localChecksum=%s blobCheckSum=%s",
+              distant.getSharedBlobName(), local.getCoreName(), bf.getSolrFileName(), bf.getBlobName(),
+              cf.getFileSize(), bf.getFileSize(), cf.getChecksum(), bf.getChecksum());
+          logger.info(message);
+          localConflictingWithBlob = true;
         }
       }
     }
-    return new SharedMetadataResolutionResult(localFilesMissingOnBlob, blobFilesMissingLocally);
+
+    if(!localConflictingWithBlob) {
+      // If local index generation number is higher than blob even than we will declare a conflict.
+      // Since in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+      localConflictingWithBlob = local.getGeneration() > distant.getGeneration();
+    }
+    // If there is a conflict we will switch index to a newer directory and pull all blob files.
+    // Later in the pipeline at the actual time of pull(CorePushPull#pullUpdateFromBlob) there is an optimization to make use of local index directory
+    // for already available files instead of downloading from blob. It was possible to design that into the contract of this 
+    // resolver to produce list of files to be pulled from blob and list of files to be pulled(read copied) from local index directory.
+    // But that would have unnecessarily convoluted the design of this resolver.
+    Collection<BlobFile> filesToPull = localConflictingWithBlob ? Arrays.asList(distant.getBlobFiles()) : blobFilesMissingLocally.values();
+    return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), filesToPull, localConflictingWithBlob);
   }
   
   /** Identify the segments_N file in Blob files. */
diff --git a/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java b/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
index 4e2fea2..1593f2d 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/client/CoreStorageClientTest.java
@@ -171,7 +171,7 @@ public class CoreStorageClientTest extends SolrTestCaseJ4 {
   @Test
   public void testPushPullCoreMetadata() throws Exception {
     blobClient.deleteCore(TEST_CORE_NAME_1);
-    BlobCoreMetadata pushedBcm = new BlobCoreMetadataBuilder(TEST_CORE_NAME_1).build();
+    BlobCoreMetadata pushedBcm = new BlobCoreMetadataBuilder(TEST_CORE_NAME_1, 19L).build();
     Assert.assertNull(blobClient.pullCoreMetadata(TEST_CORE_NAME_1, "core.metadata"));
     
     blobClient.pushCoreMetadata(TEST_CORE_NAME_1, "core.metadata", pushedBcm);
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/BlobCoreMetadataTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/BlobCoreMetadataTest.java
index f989c2d..9a3cd62 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/BlobCoreMetadataTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/BlobCoreMetadataTest.java
@@ -13,23 +13,30 @@ import org.apache.solr.store.blob.client.BlobCoreMetadataBuilder;
 public class BlobCoreMetadataTest extends Assert {
 
     final String SHARED_BLOB_NAME = "collectionName_shardNameTest";
+    final long GENERATION_NUMBER = 456;
+    final long FILE_SIZE = 123000;
+    final long CHECKSUM = 100;
 
     @Test
     public void buildCoreMetadataNoFiles() throws Exception {
-        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(SHARED_BLOB_NAME).build();
+        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(SHARED_BLOB_NAME, GENERATION_NUMBER).build();
 
         assertEquals("Blob metadata without any files should not have any files", 0, bcm.getBlobFiles().length);
         assertEquals("Blob metadata should have specified shared blob name", SHARED_BLOB_NAME, bcm.getSharedBlobName());
+        assertEquals("Blob metadata should have specified generation", GENERATION_NUMBER, bcm.getGeneration());
     }
 
     @Test
     public void buildCoreMetadataWithFile() throws Exception {
-        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(SHARED_BLOB_NAME)
-            .addFile(new BlobCoreMetadata.BlobFile("solrFilename", "blobFilename", 123000L)).build();
+        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(SHARED_BLOB_NAME, GENERATION_NUMBER)
+            .addFile(new BlobCoreMetadata.BlobFile("solrFilename", "blobFilename", FILE_SIZE, CHECKSUM)).build();
 
         assertEquals("Blob metadata should have specified shared blob name", SHARED_BLOB_NAME, bcm.getSharedBlobName());
+        assertEquals("Blob metadata should have specified generation", GENERATION_NUMBER, bcm.getGeneration());
         assertEquals("Blob metadata should have the correct number of added files", 1, bcm.getBlobFiles().length);
         assertEquals("Blob metadata file should have correct solr filename", "solrFilename", bcm.getBlobFiles()[0].getSolrFileName());
         assertEquals("Blob metadata file should have correct blob store filename", "blobFilename", bcm.getBlobFiles()[0].getBlobName());
+        assertEquals("Blob metadata file should have correct file size", FILE_SIZE, bcm.getBlobFiles()[0].getFileSize());
+        assertEquals("Blob metadata file should have correct checksum", CHECKSUM, bcm.getBlobFiles()[0].getChecksum());
     }
 }
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
index 71e187a..6121199 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
@@ -171,7 +171,95 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
     assertU(commit());
     assertQ(req("*:*"), "//*[@numFound='2']");
   }
-  
+
+  /**
+   * Tests that pull in the presence of higher local generation number is successful 
+   */
+  @Test
+  public void testLocalHigherGenerationConflictingPullSucceeds() throws Exception {
+    SolrCore core = h.getCore();
+
+    // add a doc that would be pushed to blob
+    assertU(adoc("id", "1"));
+    assertU(commit());
+
+    // the doc should be present
+    assertQ(req("*:*"),  xpathMatches("1"));
+
+    // do a push via CorePushPull, the returned BlobCoreMetadata is what we'd expect to find
+    // on the blob store
+    BlobCoreMetadata returnedBcm = doPush(core);
+
+    // add another doc but that would not be pushed to blob
+    assertU(adoc("id", "2"));
+    assertU(commit());
+
+    // the doc should be present
+    assertQ(req("*:*"),  xpathMatches("1", "2"));
+
+    long localGeneration = core.getDeletionPolicy().getLatestCommit().getGeneration();
+    assertTrue("Local generation is incorrectly not greater than blob generation", localGeneration > returnedBcm.getGeneration());
+
+    // now perform a pull, since blob being source of truth this pull should undo the addition of doc 2
+    SharedMetadataResolutionResult resResult = doPull(core, returnedBcm);
+
+    assertTrue("Pull is incorrectly not identified as conflicting", resResult.isLocalConflictingWithBlob());
+
+    // doc 1 should be present but not doc 2
+    assertQ(req("*:*"), xpathMatches("1"));
+    // for sanity index another doc
+    assertU(adoc("id", "3"));
+    assertU(commit());
+    assertQ(req("*:*"), xpathMatches("1", "3"));
+  }
+
+  /**
+   * Tests that pull in the presence of conflicting files is successful 
+   */
+  @Test
+  public void testConflictingFilesPullSucceeds() throws Exception {
+    SolrCore core = h.getCore();
+
+    // add a doc that would be pushed to blob
+    assertU(adoc("id", "1"));
+    assertU(commit());
+
+    // the doc should be present
+    assertQ(req("*:*"), xpathMatches("1"));
+
+    // do a push via CorePushPull, the returned BlobCoreMetadata is what we'd expect to find
+    // on the blob store
+    BlobCoreMetadata returnedBcm = doPush(core);
+
+    // Delete the core to clear the index data and then re-create it 
+    deleteCore();
+    initCore("solrconfig.xml", "schema-minimal.xml");
+    core = h.getCore();
+
+    // add a different doc, we will not push this to blob
+    assertU(adoc("id", "2"));
+    assertU(commit());
+
+    // the doc should be present
+    assertQ(req("*:*"), xpathMatches("2"));
+
+    // now blob and local should be at same generation number but different contents(conflicting files)
+    long localGeneration = core.getDeletionPolicy().getLatestCommit().getGeneration();
+    assertEquals("Local generation is not equal to blob generation", localGeneration, returnedBcm.getGeneration());
+
+    // now perform a pull
+    SharedMetadataResolutionResult resResult = doPull(core, returnedBcm);
+
+    assertTrue("Pull is not identified as conflicting", resResult.isLocalConflictingWithBlob());
+
+    // the doc should be present, and blob should prevail as source of truth i.e. we go back to doc 1
+    assertQ(req("*:*"), xpathMatches("1"));
+    // add another doc for sanity
+    assertU(adoc("id", "3"));
+    assertU(commit());
+    assertQ(req("*:*"), xpathMatches("1", "3"));
+  }
+
   private BlobCoreMetadata doPush(SolrCore core) throws Exception {
     // build the require metadata
     ServerSideMetadata solrServerMetadata = new ServerSideMetadata(core.getName(), h.getCoreContainer());
@@ -202,7 +290,7 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
     return pushPull.pushToBlobStore();
   }
   
-  private void doPull(SolrCore core, BlobCoreMetadata bcm) throws Exception {
+  private SharedMetadataResolutionResult doPull(SolrCore core, BlobCoreMetadata bcm) throws Exception {
     // build the require metadata
     ServerSideMetadata solrServerMetadata = new ServerSideMetadata(core.getName(), h.getCoreContainer());
     
@@ -220,5 +308,18 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
     
     CorePushPull pushPull = new CorePushPull(storageClient, deleteManager, ppd, resResult, solrServerMetadata, bcm);
     pushPull.pullUpdateFromBlob(true);
-  }  
+    return resResult;
+  }
+
+  private String[] xpathMatches(String... docIds) {
+    String[] tests = new String[docIds != null ? docIds.length + 1 : 1];
+    tests[0] = "*[count(//doc)=" + (tests.length-1) + "]";
+    if (docIds != null && docIds.length > 0) {
+      int i = 1;
+      for (String docId : docIds) {
+        tests[i++] = "//result/doc/str[@name='id'][.='" + docId + "']";
+      }
+    }
+    return tests;
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/DeleteBlobStrategyTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/DeleteBlobStrategyTest.java
index f4d9fb1d..ade1381 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/DeleteBlobStrategyTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/DeleteBlobStrategyTest.java
@@ -51,18 +51,18 @@ public class DeleteBlobStrategyTest {
     };
 
     // Now create a metadata builder with a set of files to delete and verify the right ones are deleted.
-    BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder("randomSharedName");
+    BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder("randomSharedName", 0L);
 
     // file 1 deleted at 5000 (should be removed)
-    BlobCoreMetadata.BlobFileToDelete blobFile1 = new BlobCoreMetadata.BlobFileToDelete("solrFile1", "BlobFile1", 1234L, 5000L);
+    BlobCoreMetadata.BlobFileToDelete blobFile1 = new BlobCoreMetadata.BlobFileToDelete("solrFile1", "BlobFile1", 1234L, 100L, 5000L);
     // file 2 deleted at 15000 (should NOT be removed)
-    BlobCoreMetadata.BlobFileToDelete blobFile2 = new BlobCoreMetadata.BlobFileToDelete("solrFile2", "BlobFile2", 1234L, 15000L);
+    BlobCoreMetadata.BlobFileToDelete blobFile2 = new BlobCoreMetadata.BlobFileToDelete("solrFile2", "BlobFile2", 1234L, 100L, 15000L);
     // file 3 deleted at 1000 (should be removed)
-    BlobCoreMetadata.BlobFileToDelete blobFile3 = new BlobCoreMetadata.BlobFileToDelete("solrFile3", "BlobFile3", 1234L, 1000L);
+    BlobCoreMetadata.BlobFileToDelete blobFile3 = new BlobCoreMetadata.BlobFileToDelete("solrFile3", "BlobFile3", 1234L, 100L, 1000L);
     // file 4 deleted at 1000000000 (should not be removed)
-    BlobCoreMetadata.BlobFileToDelete blobFile4 = new BlobCoreMetadata.BlobFileToDelete("solrFile4", "BlobFile4", 1234L, 1000000000L);
+    BlobCoreMetadata.BlobFileToDelete blobFile4 = new BlobCoreMetadata.BlobFileToDelete("solrFile4", "BlobFile4", 1234L, 100L, 1000000000L);
     // file 5 deleted at 1000000000 (should not be removed)
-    BlobCoreMetadata.BlobFileToDelete blobFile5 = new BlobCoreMetadata.BlobFileToDelete("solrFile5", "BlobFile5", 1234L, 1000000000L);
+    BlobCoreMetadata.BlobFileToDelete blobFile5 = new BlobCoreMetadata.BlobFileToDelete("solrFile5", "BlobFile5", 1234L, 100L, 1000000000L);
       
     bcmBuilder.addFileToDelete(blobFile1);
     bcmBuilder.addFileToDelete(blobFile2);
@@ -123,11 +123,11 @@ public class DeleteBlobStrategyTest {
     };
 
     // Now create a metadata builder with a set of files to delete
-    BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder("randomSharedName");
+    BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder("randomSharedName", 0L);
 
-    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile1", "BlobFile1", 1234L, 123456L));
-    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile2", "BlobFile2", 1234L, 234567L));
-    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile3", "BlobFile3", 1234L, 987654321L));
+    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile1", "BlobFile1", 1234L, 100L, 123456L));
+    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile2", "BlobFile2", 1234L, 100L, 234567L));
+    bcmBuilder.addFileToDelete(new BlobCoreMetadata.BlobFileToDelete("solrFile3", "BlobFile3", 1234L, 100L, 987654321L));
 
     // Call the delete code
     deleteButFailToEnqueue.enqueueForHardDelete(bcmBuilder);
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
index fcdd104..acc07ea 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
@@ -51,7 +51,7 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
    * Test that passing both local and blob as null to {@link SharedStoreResolutionUtil} throws exception.
    */
   @Test
-  public void testResolveMetadata() throws Exception {
+  public void testBothLocalAndBlobNullThrows() throws Exception {
     try {
       SharedStoreResolutionUtil.resolveMetadata(null, null);
       fail("SharedStoreResolutionUtil did not throw IllegalStateException");
@@ -67,12 +67,11 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   @Test
   public void testNoOpWhenBCMEmpty() throws Exception {
     String sharedShardName = "sharedShardName";
-    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .build();
+    final BlobCoreMetadata blobMetadata = BlobCoreMetadataBuilder.buildEmptyCoreMetadata(sharedShardName);
       
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Collections.emptySet());
+        SharedMetadataResolutionResult(Collections.emptySet(), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(null, blobMetadata);
@@ -87,14 +86,15 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   public void testPushNeededWhenBlobNull() throws Exception {
     String coreName = "localReplica";
     final long localFileSize = 10;
+    final long localFileChecksum = 100;
     // local core metadata
-    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName)
-      .addFile(new CoreFileData(getSolrSegmentFileName(1), localFileSize))
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 1L)
+      .addLatestCommitFile(new CoreFileData(getSolrSegmentFileName(1), localFileSize, localFileChecksum))
       .build();
       
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(serverMetadata.getFiles(), Collections.emptySet());
+        SharedMetadataResolutionResult(serverMetadata.getLatestCommitFiles(), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, null);
@@ -109,14 +109,15 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   public void testPullNeededWhenLocalNull() throws Exception {
     String sharedShardName = "sharedShardName";
     final long blobFileSize = 10;
+    final long blobFileChecksum = 100;
     // blob core metadata
-    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), blobFileSize))
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), blobFileSize, blobFileChecksum))
         .build();
 
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()));
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(null, blobMetadata);
@@ -130,10 +131,11 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   public void testSegmentNCorruptionThrowsError() throws Exception {
     String sharedShardName = "sharedShardName";
     final long blobFileSize = 10;
+    final long blobFileChecksum = 100;
     // blob core metadata with duplicate segment_n files
-    BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), blobFileSize))
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(2), blobFileSize))
+    BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), blobFileSize, blobFileChecksum))
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(2), blobFileSize, blobFileChecksum))
         .build();
     
     // do resolution
@@ -145,8 +147,8 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
     }
     
     // blob core metadata with missing segment_n file while we have other segment files
-    blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrFileName(1), getBlobFileName(1), blobFileSize))
+    blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrFileName(1), getBlobFileName(1), blobFileSize, blobFileChecksum))
         .build();
     // do resolution
     try {
@@ -160,25 +162,27 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   /**
    * Test resolve of both {@link BlobCoreMetadata} and {@link ServerSideMetadata} contains files to push only
    */
+  @Test
   public void testFilesToPushResolution() throws Exception {
     String coreName = "coreName";
     String sharedShardName = "sharedShardName";
     final long fileSize = 10;
+    final long checksum = 100;
+
+    CoreFileData expectedFileToPush = new CoreFileData(getSolrFileName(1), fileSize, checksum);
     
-    CoreFileData expectedFileToPush = new CoreFileData(getSolrFileName(1), fileSize);
-    
-    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName)
-        .addFile(new CoreFileData(getSolrSegmentFileName(1), fileSize))
-        .addFile(expectedFileToPush)
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 1L)
+        .addLatestCommitFile(new CoreFileData(getSolrSegmentFileName(1), fileSize, checksum))
+        .addLatestCommitFile(expectedFileToPush)
         .build();
     
-    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize))
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize, checksum))
         .build();
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Collections.emptySet());
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -188,25 +192,27 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
   /**
    * Test resolve of both {@link BlobCoreMetadata} and {@link ServerSideMetadata} contains files to pull only
    */
+  @Test
   public void testFilesToPullResolution() throws Exception {
     String coreName = "coreName";
     String sharedShardName = "sharedShardName";
     final long fileSize = 10;
+    final long checksum = 100;
+
+    BlobFile expectedFileToPull = new BlobFile(getSolrFileName(1),  getBlobFileName(1), fileSize, checksum);
     
-    BlobFile expectedFileToPull = new BlobFile(getSolrFileName(1),  getBlobFileName(1), fileSize);
-    
-    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName)
-        .addFile(new CoreFileData(getSolrSegmentFileName(1), fileSize))
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 1L)
+        .addLatestCommitFile(new CoreFileData(getSolrSegmentFileName(1), fileSize, checksum))
         .build();
     
-    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize))
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize, checksum))
         .addFile(expectedFileToPull)
         .build();
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(expectedFileToPull));
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(expectedFileToPull), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -217,36 +223,129 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
    * Test resolve of both {@link BlobCoreMetadata} and {@link ServerSideMetadata} contains both files to 
    * push and pull
    */
+  @Test
   public void testFilesToPushPullResolution() throws Exception {
     String coreName = "coreName";
     String sharedShardName = "sharedShardName";
     final long fileSize = 10;
+    final long checksum = 100;
+
+    CoreFileData expectedFileToPush = new CoreFileData(getSolrFileName(3), fileSize, checksum);
+    BlobFile expectedFileToPull = new BlobFile(getSolrFileName(2),  getBlobFileName(2), fileSize, checksum);
     
-    CoreFileData expectedFileToPush = new CoreFileData(getSolrFileName(1), fileSize);
-    BlobFile expectedFileToPull = new BlobFile(getSolrFileName(1),  getBlobFileName(1), fileSize);
-    
-    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName)
-        .addFile(new CoreFileData(getSolrSegmentFileName(1), fileSize))
-        .addFile(expectedFileToPush)
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 1L)
+        .addLatestCommitFile(new CoreFileData(getSolrSegmentFileName(1), fileSize, checksum))
+        .addLatestCommitFile(expectedFileToPush)
         .build();
     
-    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName)
-        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize))
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize, checksum))
         .addFile(expectedFileToPull)
         .build();
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(expectedFileToPull));
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(expectedFileToPull), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
     assertSharedStoreResolutionResult(expectedResult, actual);
   }
-  
+
+  /**
+   * Tests that local generation number being higher than blob resolves into a conflict 
+   */
+  @Test
+  public void testHigherLocalGenerationResolution() throws Exception {
+    String coreName = "coreName";
+    String sharedShardName = "sharedShardName";
+    final long fileSize = 10;
+    final long checksum = 100;
+
+    CoreFileData expectedFileToPush = new CoreFileData(getSolrSegmentFileName(2), fileSize, checksum);
+
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 2L)
+        .addLatestCommitFile(expectedFileToPush)
+        .addLatestCommitFile(new CoreFileData(getSolrFileName(1), fileSize, checksum))
+        .build();
+
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize, checksum))
+        .addFile(new BlobFile(getSolrFileName(1), getBlobFileName(1), fileSize, checksum))
+        .build();
+
+    // expected resolution
+    SharedMetadataResolutionResult expectedResult = new
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), true);
+
+    // do resolution
+    SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
+    assertSharedStoreResolutionResult(expectedResult, actual);
+  }
+
+  /**
+   * Tests that file belonging to latest commit point being present both locally and in blob but with different checksum
+   * resolves into a conflict 
+   */
+  @Test
+  public void testFileFromLatestCommitConflictsResolution() throws Exception {
+    String coreName = "coreName";
+    String sharedShardName = "sharedShardName";
+    final long fileSize = 10;
+
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 1L)
+        .addLatestCommitFile(new CoreFileData(getSolrSegmentFileName(1), fileSize, 99))
+        .build();
+
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 1L)
+        .addFile(new BlobFile(getSolrSegmentFileName(1), getBlobFileName(1), fileSize, 100))
+        .build();
+
+    // expected resolution
+    SharedMetadataResolutionResult expectedResult = new
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), true);
+
+    // do resolution
+    SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
+    assertSharedStoreResolutionResult(expectedResult, actual);
+  }
+
+  /**
+   * Tests that file belonging to previous commit point being present both locally and in blob but with different checksum
+   * resolves into a conflict 
+   */
+  @Test
+  public void testFileFromPreviousCommitConflictsResolution() throws Exception {
+    String coreName = "coreName";
+    String sharedShardName = "sharedShardName";
+    final long fileSize = 10;
+    final long checksum = 100;
+
+    CoreFileData expectedFileToPush = new CoreFileData(getSolrSegmentFileName(2), fileSize, checksum);
+
+    final ServerSideMetadata serverMetadata = new ServerSideCoreMetadataBuilder(coreName, 2L)
+        .addLatestCommitFile(expectedFileToPush)
+        .addAllCommitsFile(new CoreFileData(getSolrFileName(1), fileSize, 99))
+        .build();
+
+    final BlobCoreMetadata blobMetadata = new BlobCoreMetadataBuilder(sharedShardName, 3L)
+        .addFile(new BlobFile(getSolrSegmentFileName(3), getBlobFileName(3), fileSize, checksum))
+        .addFile(new BlobFile(getSolrFileName(1), getBlobFileName(1), fileSize, 100))
+        .build();
+
+    // expected resolution
+    SharedMetadataResolutionResult expectedResult = new
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), true);
+
+    // do resolution
+    SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
+    assertSharedStoreResolutionResult(expectedResult, actual);
+  }
+
   private static void assertSharedStoreResolutionResult(SharedMetadataResolutionResult expected, SharedMetadataResolutionResult actual) {
     assertCollections("filesToPull", expected.getFilesToPull(), actual.getFilesToPull());
     assertCollections("filesToPush", expected.getFilesToPush(), actual.getFilesToPush());
+    assertEquals("localConflictingWithBlob", expected.isLocalConflictingWithBlob(), actual.isLocalConflictingWithBlob());
   }
   
   /**
@@ -284,22 +383,34 @@ public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase
    */
   private static class ServerSideCoreMetadataBuilder {
     final private String coreName;
-    final private ImmutableSet.Builder<CoreFileData> files;
+    final private long generation;
+    final private ImmutableSet.Builder<CoreFileData> latestCommitFiles;
+    final private ImmutableSet.Builder<CoreFileData> allCommitsFiles;
 
-    ServerSideCoreMetadataBuilder(String coreName) {
+    ServerSideCoreMetadataBuilder(String coreName, Long generation) {
       this.coreName = coreName;
-      files = new ImmutableSet.Builder();
+      this.generation = generation;
+      latestCommitFiles = new ImmutableSet.Builder();
+      allCommitsFiles = new ImmutableSet.Builder();
+    }
+
+    ServerSideCoreMetadataBuilder addLatestCommitFile(CoreFileData file) {
+      this.latestCommitFiles.add(file);
+      addAllCommitsFile(file);
+      return this;
     }
 
-    ServerSideCoreMetadataBuilder addFile(CoreFileData file) {
-      this.files.add(file);
+    ServerSideCoreMetadataBuilder addAllCommitsFile(CoreFileData file) {
+      this.allCommitsFiles.add(file);
       return this;
     }
 
     public ServerSideMetadata build() {
       ServerSideMetadata serverMetadata = mock(ServerSideMetadata.class);
       when(serverMetadata.getCoreName()).thenReturn(coreName);
-      when(serverMetadata.getFiles()).thenReturn(files.build());
+      when(serverMetadata.getGeneration()).thenReturn(generation);
+      when(serverMetadata.getLatestCommitFiles()).thenReturn(latestCommitFiles.build());
+      when(serverMetadata.getAllCommitsFiles()).thenReturn(allCommitsFiles.build());
       return serverMetadata;
     }
   }
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/ToFromJsonTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/ToFromJsonTest.java
index 42775c1..6c1c359 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/ToFromJsonTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/ToFromJsonTest.java
@@ -35,28 +35,28 @@ public class ToFromJsonTest extends Assert {
 
     @Test
     public void jsonCoreMetadataNoFiles() throws Exception {
-        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(CORE_NAME).build();
+        BlobCoreMetadata bcm = new BlobCoreMetadataBuilder(CORE_NAME, 666L).build();
 
         verifyToJsonAndBack(bcm);
     }
 
     @Test
     public void jsonCoreMetadataFile() throws Exception {
-        BlobCoreMetadataBuilder bb = new BlobCoreMetadataBuilder(CORE_NAME);
+        BlobCoreMetadataBuilder bb = new BlobCoreMetadataBuilder(CORE_NAME, 777L);
 
-        BlobCoreMetadata bcm = bb.addFile(new BlobCoreMetadata.BlobFile("solrFilename", "blobFilename", 123000L)).build();
+        BlobCoreMetadata bcm = bb.addFile(new BlobCoreMetadata.BlobFile("solrFilename", "blobFilename", 123000L, 100L)).build();
 
         verifyToJsonAndBack(bcm);
     }
 
     @Test
     public void jsonCoreMetadataMultiFiles() throws Exception {
-        BlobCoreMetadataBuilder bb = new BlobCoreMetadataBuilder(CORE_NAME);
+        BlobCoreMetadataBuilder bb = new BlobCoreMetadataBuilder(CORE_NAME, 123L);
         Set<BlobCoreMetadata.BlobFile> files = new HashSet<>(Arrays.asList(
-                new BlobCoreMetadata.BlobFile("solrFilename11", "blobFilename11", 1234L),
-                new BlobCoreMetadata.BlobFile("solrFilename21", "blobFilename21", 2345L),
-                new BlobCoreMetadata.BlobFile("solrFilename31", "blobFilename31", 3456L),
-                new BlobCoreMetadata.BlobFile("solrFilename41", "blobFilename41", 4567L)
+                new BlobCoreMetadata.BlobFile("solrFilename11", "blobFilename11", 1234L, 100L),
+                new BlobCoreMetadata.BlobFile("solrFilename21", "blobFilename21", 2345L, 200L),
+                new BlobCoreMetadata.BlobFile("solrFilename31", "blobFilename31", 3456L, 200L),
+                new BlobCoreMetadata.BlobFile("solrFilename41", "blobFilename41", 4567L, 400L)
         ));
         for (BlobCoreMetadata.BlobFile f : files) {
             bb.addFile(f);
@@ -68,6 +68,7 @@ public class ToFromJsonTest extends Assert {
         verifyToJsonAndBack(bcm);
 
         assertEquals("blob core metadata should have core name specified to builder", CORE_NAME, bcm.getSharedBlobName());
+        assertEquals("blob core metadata should have generation specified to builder",123L, bcm.getGeneration());
 
         // Files are not necessarily in the same order
         assertEquals("blob core metadata should have file set specified to builder", files, new HashSet<>(Arrays.asList(bcm.getBlobFiles())));


[lucene-solr] 08/11: @W-6684857 indentation cleanup, precommit error fixes, remove junit use in non-test classes (#400)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 535845327c1f0a9b848a723bddcb4dbb5a22ee93
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Thu Oct 10 18:18:04 2019 -0400

    @W-6684857 indentation cleanup, precommit error fixes, remove junit use in non-test classes (#400)
    
    * Fix indentation in the remaining ported classes
    * Fix all precommit failures and remove junit from non-test classes
    * Remove MetadataResolverTest which is no longer relevant
---
 solr/core/ivy.xml                                  |   5 +-
 .../store/blob/DeletedCorruptCoreException.java    |   6 +-
 .../solr/store/blob/PullInProgressException.java   |   6 +-
 .../store/blob/client/BlobClientException.java     |  20 +-
 .../solr/store/blob/client/BlobCoreMetadata.java   | 490 +++++++--------
 .../store/blob/client/BlobCoreMetadataBuilder.java | 164 ++---
 .../solr/store/blob/client/BlobException.java      |  18 +-
 .../store/blob/client/BlobServiceException.java    |  18 +-
 .../solr/store/blob/client/CoreStorageClient.java  | 248 ++++----
 .../apache/solr/store/blob/client/ToFromJson.java  |  32 +-
 .../solr/store/blob/metadata/BlobCoreSyncer.java   | 662 ++++++++++-----------
 .../store/blob/metadata/ServerSideMetadata.java    | 406 ++++++-------
 .../blob/metadata/SharedStoreResolutionUtil.java   |   4 +-
 .../solr/store/blob/process/BlobDeleteManager.java |   5 +
 .../solr/store/blob/process/CoreSyncFeeder.java    | 189 +++---
 .../solr/store/blob/util/BlobStoreUtils.java       |   4 +-
 .../solr/store/blob/util/DeduplicatingList.java    | 294 ++++-----
 .../processor/DistributedZkUpdateProcessor.java    |   3 +-
 .../solr/store/blob/metadata/CorePushPullTest.java |   1 +
 .../metadata/SharedStoreResolutionUtilTest.java    |   8 +-
 .../solr/store/blob/util/BlobStoreUtilsTest.java   |   3 +-
 .../SharedShardMetadataControllerTest.java         |   3 +-
 .../DistributedZkUpdateProcessorTest.java          |   3 +-
 23 files changed, 1296 insertions(+), 1296 deletions(-)

diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index a09196c..eb6dc60 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -154,9 +154,6 @@
     <dependency org="software.amazon.ion" name="ion-java" rev="${/software.amazon.ion/ion-java}" conf="compile"/>
     <dependency org="com.google.code.gson" name="gson" rev="${/com.google/gson}" conf="compile"/>
 
-    <!-- Junit -->
-    <dependency org="junit" name="junit" rev="4.13-beta-1"/>
-
     <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
   </dependencies>
-</ivy-module>
+</ivy-module>
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/store/blob/DeletedCorruptCoreException.java b/solr/core/src/java/org/apache/solr/store/blob/DeletedCorruptCoreException.java
index a960c3f..ce7cf19 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/DeletedCorruptCoreException.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/DeletedCorruptCoreException.java
@@ -21,7 +21,7 @@ package org.apache.solr.store.blob;
  * another version from Blob Store.
  */
 public class DeletedCorruptCoreException extends Exception {
-    public DeletedCorruptCoreException(String message) {
-        super(message);
-    }
+  public DeletedCorruptCoreException(String message) {
+    super(message);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/PullInProgressException.java b/solr/core/src/java/org/apache/solr/store/blob/PullInProgressException.java
index 20460db..9181e60 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/PullInProgressException.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/PullInProgressException.java
@@ -21,7 +21,7 @@ package org.apache.solr.store.blob;
  * available so can't be used and request can't be handled.
  */
 public class PullInProgressException extends Exception {
-    public PullInProgressException(String message) {
-        super(message);
-    }
+  public PullInProgressException(String message) {
+    super(message);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobClientException.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobClientException.java
index 0f5387b..e5c1b3b 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobClientException.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobClientException.java
@@ -20,19 +20,19 @@ package org.apache.solr.store.blob.client;
  * 
  * Extension of BlobException that represents an error on the client-side when attempting to make
  * a request to the underlying blob store service. An exception of this type indicates that the
- * client was unable to successfully make the service call.  
+ * client was unable to successfully make the service call.
  */
 public class BlobClientException extends BlobException {
     
-    public BlobClientException(Throwable cause) {
-        super(cause);
-    }
+  public BlobClientException(Throwable cause) {
+    super(cause);
+  }
 
-    public BlobClientException(String message) {
-        super(message);
-    }
+  public BlobClientException(String message) {
+    super(message);
+  }
 
-    public BlobClientException(String message, Throwable cause) {
-        super(message, cause);
-    }
+  public BlobClientException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
index 8bf2f2a..dc36dc9 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadata.java
@@ -14,287 +14,287 @@ import java.util.UUID;
  */
 public class BlobCoreMetadata {
 
+  /**
+   * Name of the shard index data that is shared by all replicas belonging to that shard. This 
+   * name is to decouple the core name that Solr manages from the name of the core on blob store. 
+   */
+  private final String sharedBlobName;
+
+  /**
+   * Generation number of index represented by this metadata.
+   * This generation number is only meant to identify a scenario where local index generation number is higher than
+   * what we have in blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
+   * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+   */
+  private final long generation;
+
+  /**
+   * Unique identifier of this metadata, that changes on every update to the metadata (except generating a new corrupt metadata
+   * through {@link #getCorruptOf}).
+   */
+  private final String uniqueIdentifier;
+
+  /**
+   * Indicates that a Solr (search) server pulled this core and was then unable to open or use it. This flag is used as
+   * an indication to servers pushing blobs for that core into Blob Store to push a complete set of files if they have
+   * a locally working copy rather than just diffs (files missing on Blob Store).
+   */
+  private final boolean isCorrupt;
+
+  /**
+   * Indicates that this core has been deleted by the client. This flag is used as a marker to prevent other servers
+   * from pushing their version of this core to blob and to allow local copy cleanup.
+   */
+  private final boolean isDeleted;
+
+  /**
+   * The array of files that constitute the current commit point of the core (as known by the Blob store).
+   * This array is not ordered! There are no duplicate entries in it either (see how it's built in {@link BlobCoreMetadataBuilder}).
+   */
+  private final BlobFile[] blobFiles;
+
+  /**
+   * Files marked for delete but not yet removed from the Blob store. Each such file contains information indicating when
+   * it was marked for delete so we can actually remove the corresponding blob (and the entry from this array in the metadata)
+   * when it's safe to do so even if there are (unexpected) conflicting updates to the blob store by multiple solr servers...
+   * TODO: we might want to separate the metadata blob with the deletes as it's not required to always fetch the delete list when checking freshness of local core...
+   */
+  private final BlobFileToDelete[] blobFilesToDelete;
+
+  /**
+   * This is the constructor called by {@link BlobCoreMetadataBuilder}.
+   * It always builds non "isCorrupt" and non "isDeleted" metadata. 
+   * The only way to build an instance of "isCorrupt" metadata is to use {@link #getCorruptOf} and for "isDeleted" use {@link #getDeletedOf()}
+   */
+  BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation) {
+    this(sharedBlobName, blobFiles, blobFilesToDelete, generation, UUID.randomUUID().toString(), false,
+            false);
+  }
+
+  private BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation,
+    String uniqueIdentifier, boolean isCorrupt, boolean isDeleted) {
+    this.sharedBlobName = sharedBlobName;
+    this.blobFiles = blobFiles;
+    this.blobFilesToDelete = blobFilesToDelete;
+    this.generation = generation;
+    this.uniqueIdentifier = uniqueIdentifier;
+    this.isCorrupt = isCorrupt;
+    this.isDeleted = isDeleted;
+  }
+
+  /**
+   * Given a non corrupt {@link BlobCoreMetadata} instance, creates an equivalent one based on it but marked as corrupt.<p>
+   * The new instance keeps all the rest of the metadata unchanged, including the {@link #uniqueIdentifier}.
+   */
+  public BlobCoreMetadata getCorruptOf() {
+    assert !isCorrupt;
+    return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, true, isDeleted);
+  }
+
+  /**
+   * Given a {@link BlobCoreMetadata} instance, creates an equivalent one based on it but marked as deleted.
+   * <p>
+   * The new instance keeps all the rest of the metadata unchanged, including the {@link #uniqueIdentifier}.
+   */
+  public BlobCoreMetadata getDeletedOf() {
+    assert !isDeleted;
+    return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, isCorrupt, true);
+  }
+
+  /**
+   * Returns true if the Blob metadata was marked as deleted
+   */
+  public boolean getIsDeleted() {
+    return isDeleted;
+  }
+
+  /**
+   * Returns the core name corresponding to this metadata
+   */
+  public String getSharedBlobName() {
+    return sharedBlobName;
+  }
+
+  /**
+   * Returns true if the Blob metadata was marked as corrupt. In which case, the core should not be pulled from the Blob Store
+   * as it is useless.
+   */
+  public boolean getIsCorrupt() {
+    return isCorrupt;
+  }
+
+  /**
+   * Unique identifier of this blob core metadata. Allows quickly seeing that the core metadata has changed without comparing
+   * the whole content.<p>
+   * {@link #getCorruptOf()} is the only call allowing the creation of two instances of {@link BlobCoreMetadata} having
+   * the same unique identifier.
+   */
+  public String getUniqueIdentifier() {
+    return uniqueIdentifier;
+  }
+
+  public long getGeneration() {
+    return this.generation;
+  }
+
+  public BlobFile[] getBlobFiles() {
+    return blobFiles;
+  }
+
+  public BlobFileToDelete[] getBlobFilesToDelete() {
+    return blobFilesToDelete;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+  
+    BlobCoreMetadata that = (BlobCoreMetadata) o;
+  
+    if (this.generation != that.generation) return false;
+    if (this.isCorrupt != that.isCorrupt) return false;
+    if (this.isDeleted != that.isDeleted) return false;
+    if (!this.uniqueIdentifier.equals(that.uniqueIdentifier)) return false;
+    if (!this.sharedBlobName.equals(that.sharedBlobName)) return false;
+  
+    // blobFiles array is not ordered so not using Arrays.equals here but rather Set comparison (we also know all elements are distinct in the array)
+    Set<BlobFile> thisFiles = new HashSet<>(Arrays.asList(this.blobFiles));
+    Set<BlobFile> thatFiles = new HashSet<>(Arrays.asList(that.blobFiles));
+    if (!thisFiles.equals(thatFiles)) return false;
+  
+    // same for the conf files
+    Set<BlobFileToDelete> thisFilesToDelete = new HashSet<>(Arrays.asList(this.blobFilesToDelete));
+    Set<BlobFileToDelete> thatFilesToDelete = new HashSet<>(Arrays.asList(that.blobFilesToDelete));
+    return thisFilesToDelete.equals(thatFilesToDelete);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(sharedBlobName, uniqueIdentifier, generation,
+      // The array of files is not ordered so need to compare as a set
+      new HashSet<>(Arrays.asList(this.blobFiles)).hashCode(),
+      new HashSet<>(Arrays.asList(this.blobFilesToDelete)).hashCode(),
+      isCorrupt, isDeleted);
+  }
+
+  @Override
+  public String toString() {
+    return "sharedBlobName=" + sharedBlobName +  " generation=" + generation 
+        + " isCorrupt=" + isCorrupt + " uniqueIdentifier=" + uniqueIdentifier;
+  }
+
+  /**
+   * A file (or blob) stored in the blob store.
+   */
+  public static class BlobFile {
     /**
-     * Name of the shard index data that is shared by all replicas belonging to that shard. This 
-     * name is to decouple the core name that Solr manages from the name of the core on blob store. 
+     * 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 String sharedBlobName;
+    private final String solrFileName;
 
     /**
-     * Generation number of index represented by this metadata.
-     * This generation number is only meant to identify a scenario where local index generation number is higher than
-     * what we have in blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
-     * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+     * Name of the blob representing the file on the blob store. This will initially be an absolute path on the Blob
+     * server (for compatibility with {@link org.apache.solr.store.blob.client.LocalStorageClient}) but eventually might not include
+     * the core name if cores are organized into per core S3 buckets).
      */
-    private final long generation;
+    private final String blobName;
 
-    /**
-     * Unique identifier of this metadata, that changes on every update to the metadata (except generating a new corrupt metadata
-     * through {@link #getCorruptOf}).
-     */
-    private final String uniqueIdentifier;
-
-    /**
-     * Indicates that a Solr (search) server pulled this core and was then unable to open or use it. This flag is used as
-     * an indication to servers pushing blobs for that core into Blob Store to push a complete set of files if they have
-     * a locally working copy rather than just diffs (files missing on Blob Store).
-     */
-    private final boolean isCorrupt;
-
-    /**
-     * Indicates that this core has been deleted by the client. This flag is used as a marker to prevent other servers
-     * from pushing their version of this core to blob and to allow local copy cleanup.
-     */
-    private final boolean isDeleted;
-
-    /**
-     * The array of files that constitute the current commit point of the core (as known by the Blob store).
-     * This array is not ordered! There are no duplicate entries in it either (see how it's built in {@link BlobCoreMetadataBuilder}).
-     */
-    private final BlobFile[] blobFiles;
+    private final long fileSize;
 
     /**
-     * Files marked for delete but not yet removed from the Blob store. Each such file contains information indicating when
-     * it was marked for delete so we can actually remove the corresponding blob (and the entry from this array in the metadata)
-     * when it's safe to do so even if there are (unexpected) conflicting updates to the blob store by multiple solr servers...
-     * TODO: we might want to separate the metadata blob with the deletes as it's not required to always fetch the delete list when checking freshness of local core...
+     * Lucene generated checksum of the file. It is used in addition to file size to compare local and blob files.
      */
-    private final BlobFileToDelete[] blobFilesToDelete;
+    private final long checksum;
 
-    /**
-     * This is the constructor called by {@link BlobCoreMetadataBuilder}.
-     * It always builds non "isCorrupt" and non "isDeleted" metadata. 
-     * The only way to build an instance of "isCorrupt" metadata is to use {@link #getCorruptOf} and for "isDeleted" use {@link #getDeletedOf()}
-     */
-    BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation) {
-        this(sharedBlobName, blobFiles, blobFilesToDelete, generation, UUID.randomUUID().toString(), false,
-                false);
+    public BlobFile(String solrFileName, String blobName, long fileSize, long checksum) {
+      this.solrFileName = solrFileName;
+      this.blobName = blobName;
+      this.fileSize = fileSize;
+      this.checksum = checksum;
     }
 
-    private BlobCoreMetadata(String sharedBlobName, BlobFile[] blobFiles, BlobFileToDelete[] blobFilesToDelete, long generation,
-        String uniqueIdentifier, boolean isCorrupt, boolean isDeleted) {
-        this.sharedBlobName = sharedBlobName;
-        this.blobFiles = blobFiles;
-        this.blobFilesToDelete = blobFilesToDelete;
-        this.generation = generation;
-        this.uniqueIdentifier = uniqueIdentifier;
-        this.isCorrupt = isCorrupt;
-        this.isDeleted = isDeleted;
-    }
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
 
-    /**
-     * Given a non corrupt {@link BlobCoreMetadata} instance, creates an equivalent one based on it but marked as corrupt.<p>
-     * The new instance keeps all the rest of the metadata unchanged, including the {@link #uniqueIdentifier}.
-     */
-    public BlobCoreMetadata getCorruptOf() {
-        assert !isCorrupt;
-        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, true, isDeleted);
-    }
+      BlobFile other = (BlobFile) o;
 
-    /**
-     * Given a {@link BlobCoreMetadata} instance, creates an equivalent one based on it but marked as deleted.
-     * <p>
-     * The new instance keeps all the rest of the metadata unchanged, including the {@link #uniqueIdentifier}.
-     */
-    public BlobCoreMetadata getDeletedOf() {
-        assert !isDeleted;
-        return new BlobCoreMetadata(sharedBlobName, blobFiles, blobFilesToDelete, generation, uniqueIdentifier, isCorrupt, true);
+      return Objects.equals(solrFileName, other.solrFileName) &&
+          Objects.equals(blobName, other.blobName) &&
+          Objects.equals(checksum, other.checksum) &&
+          Objects.equals(fileSize, other.fileSize);
     }
 
-    /**
-     * Returns true if the Blob metadata was marked as deleted
-     */
-    public boolean getIsDeleted() {
-        return isDeleted;
+    @Override
+    public int hashCode() {
+      return Objects.hash(solrFileName, blobName, fileSize, checksum);
     }
 
-    /**
-     * Returns the core name corresponding to this metadata
-     */
-    public String getSharedBlobName() {
-        return sharedBlobName;
+    public String getSolrFileName() {
+      return this.solrFileName;
     }
 
-    /**
-     * Returns true if the Blob metadata was marked as corrupt. In which case, the core should not be pulled from the Blob Store
-     * as it is useless.
-     */
-    public boolean getIsCorrupt() {
-        return isCorrupt;
+    public String getBlobName() {
+      return this.blobName;
     }
 
-    /**
-     * Unique identifier of this blob core metadata. Allows quickly seeing that the core metadata has changed without comparing
-     * the whole content.<p>
-     * {@link #getCorruptOf()} is the only call allowing the creation of two instances of {@link BlobCoreMetadata} having
-     * the same unique identifier.
-     */
-    public String getUniqueIdentifier() {
-        return uniqueIdentifier;
+    public long getFileSize() {
+      return this.fileSize;
     }
 
-    public long getGeneration() {
-        return this.generation;
+    public long getChecksum() {
+      return this.checksum;
     }
-
-    public BlobFile[] getBlobFiles() {
-        return blobFiles;
+  }
+
+  /**
+   * A file (or blob) stored in the blob store that should be deleted (after a certain "delay" to make sure it's not used
+   * by a conflicting update to the core metadata on the Blob...).
+   */
+  public static class BlobFileToDelete extends BlobFile {
+
+    // TODO using the delete timestamp for now but likely need something else:
+    // deleted sequence number allows letting multiple sequence numbers to pass before deleting, whereas
+    // an old delete (as told by its timestamp) might still be the latest update to the core if not a lot of indexing
+    // activity, so hard to judge a delete is safe based on this. Possibly delete sequence and delete timestamp are
+    // the safest bet, covering both cases of very high indexing activity cores (we might want to wait until timestamp
+    // ages a bit given sequence number can increase quickly yet we could have a race with a server doing a slow update)
+    // as well as slowly updating cores (if delete date is a week ago, even if sequence number hasn't changed, the
+    // likelyhood of a really really slow update by another server causing a race is low).
+    private final long deletedAt;
+
+    public BlobFileToDelete(String solrFileName, String blobName, long fileSize, long checksum, long deletedAt) {
+      super(solrFileName, blobName, fileSize, checksum);
+
+      this.deletedAt = deletedAt;
     }
 
-    public BlobFileToDelete[] getBlobFilesToDelete() {
-        return blobFilesToDelete;
+    public BlobFileToDelete(BlobFile bf, long deletedAt) {
+      super(bf.solrFileName, bf.blobName, bf.fileSize, bf.checksum);
+
+      this.deletedAt = deletedAt;
     }
 
     @Override
     public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-
-        BlobCoreMetadata that = (BlobCoreMetadata) o;
-
-        if (this.generation != that.generation) return false;
-        if (this.isCorrupt != that.isCorrupt) return false;
-        if (this.isDeleted != that.isDeleted) return false;
-        if (!this.uniqueIdentifier.equals(that.uniqueIdentifier)) return false;
-        if (!this.sharedBlobName.equals(that.sharedBlobName)) return false;
-
-        // blobFiles array is not ordered so not using Arrays.equals here but rather Set comparison (we also know all elements are distinct in the array)
-        Set<BlobFile> thisFiles = new HashSet<>(Arrays.asList(this.blobFiles));
-        Set<BlobFile> thatFiles = new HashSet<>(Arrays.asList(that.blobFiles));
-        if (!thisFiles.equals(thatFiles)) return false;
-
-        // same for the conf files
-        Set<BlobFileToDelete> thisFilesToDelete = new HashSet<>(Arrays.asList(this.blobFilesToDelete));
-        Set<BlobFileToDelete> thatFilesToDelete = new HashSet<>(Arrays.asList(that.blobFilesToDelete));
-        return thisFilesToDelete.equals(thatFilesToDelete);
-    }
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+      if (!super.equals(o)) return false;
 
-    @Override
-    public int hashCode() {
-        return Objects.hash(sharedBlobName, uniqueIdentifier, generation,
-            // The array of files is not ordered so need to compare as a set
-            new HashSet<>(Arrays.asList(this.blobFiles)).hashCode(),
-            new HashSet<>(Arrays.asList(this.blobFilesToDelete)).hashCode(),
-            isCorrupt, isDeleted);
-    }
+      BlobFileToDelete other = (BlobFileToDelete) o;
 
-    @Override
-    public String toString() {
-        return "sharedBlobName=" + sharedBlobName +  " generation=" + generation 
-            + " isCorrupt=" + isCorrupt + " uniqueIdentifier=" + uniqueIdentifier;
+      return deletedAt == other.deletedAt;
     }
 
-    /**
-     * A file (or blob) stored in the blob store.
-     */
-    public static class BlobFile {
-        /**
-         * 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 String solrFileName;
-
-        /**
-         * Name of the blob representing the file on the blob store. This will initially be an absolute path on the Blob
-         * server (for compatibility with {@link org.apache.solr.store.blob.client.LocalStorageClient}) but eventually might not include
-         * the core name if cores are organized into per core S3 buckets).
-         */
-        private final String blobName;
-
-        private final long fileSize;
-
-        /**
-         * Lucene generated checksum of the file. It is used in addition to file size to compare local and blob files.
-         */
-        private final long checksum;
-
-        public BlobFile(String solrFileName, String blobName, long fileSize, long checksum) {
-            this.solrFileName = solrFileName;
-            this.blobName = blobName;
-            this.fileSize = fileSize;
-            this.checksum = checksum;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            BlobFile other = (BlobFile) o;
-
-            return Objects.equals(solrFileName, other.solrFileName) &&
-                Objects.equals(blobName, other.blobName) &&
-                Objects.equals(checksum, other.checksum) &&
-                Objects.equals(fileSize, other.fileSize);
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(solrFileName, blobName, fileSize, checksum);
-        }
-
-        public String getSolrFileName() {
-            return this.solrFileName;
-        }
-
-        public String getBlobName() {
-            return this.blobName;
-        }
-
-        public long getFileSize() {
-            return this.fileSize;
-        }
-
-        public long getChecksum() {
-            return this.checksum;
-        }
+    @Override
+    public int hashCode() {
+      return Objects.hash(super.hashCode(), deletedAt);
     }
 
-    /**
-     * A file (or blob) stored in the blob store that should be deleted (after a certain "delay" to make sure it's not used
-     * by a conflicting update to the core metadata on the Blob...).
-     */
-    public static class BlobFileToDelete extends BlobFile {
-
-      // TODO using the delete timestamp for now but likely need something else:
-      // deleted sequence number allows letting multiple sequence numbers to pass before deleting, whereas
-      // an old delete (as told by its timestamp) might still be the latest update to the core if not a lot of indexing
-      // activity, so hard to judge a delete is safe based on this. Possibly delete sequence and delete timestamp are
-      // the safest bet, covering both cases of very high indexing activity cores (we might want to wait until timestamp
-      // ages a bit given sequence number can increase quickly yet we could have a race with a server doing a slow update)
-      // as well as slowly updating cores (if delete date is a week ago, even if sequence number hasn't changed, the
-      // likelyhood of a really really slow update by another server causing a race is low).
-      private final long deletedAt;
-
-      public BlobFileToDelete(String solrFileName, String blobName, long fileSize, long checksum, long deletedAt) {
-        super(solrFileName, blobName, fileSize, checksum);
-
-        this.deletedAt = deletedAt;
-      }
-
-      public BlobFileToDelete(BlobFile bf, long deletedAt) {
-        super(bf.solrFileName, bf.blobName, bf.fileSize, bf.checksum);
-
-        this.deletedAt = deletedAt;
-      }
-
-      @Override
-      public boolean equals(Object o) {
-        if (this == o) return true;
-        if (o == null || getClass() != o.getClass()) return false;
-        if (!super.equals(o)) return false;
-
-        BlobFileToDelete other = (BlobFileToDelete) o;
-
-        return deletedAt == other.deletedAt;
-      }
-
-      @Override
-      public int hashCode() {
-          return Objects.hash(super.hashCode(), deletedAt);
-      }
-
-      public long getDeletedAt() {
-        return this.deletedAt;
-      }
+    public long getDeletedAt() {
+      return this.deletedAt;
     }
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
index bc43464..503cbf0 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobCoreMetadataBuilder.java
@@ -6,98 +6,98 @@ import java.util.*;
  * Builder for {@link BlobCoreMetadata}.
  */
 public class BlobCoreMetadataBuilder {
-    /**
-     * Generation number in metadata of cores not existing on the Blob Store.
-     */
-    public static final long UNDEFINED_VALUE = -1L;
+  /**
+   * Generation number in metadata of cores not existing on the Blob Store.
+   */
+  public static final long UNDEFINED_VALUE = -1L;
 
-    final private String sharedBlobName;
-    final private long generation;
-    final private Set<BlobCoreMetadata.BlobFile> blobFiles;
-    final private Set<BlobCoreMetadata.BlobFileToDelete> blobFilesToDelete;
+  final private String sharedBlobName;
+  final private long generation;
+  final private Set<BlobCoreMetadata.BlobFile> blobFiles;
+  final private Set<BlobCoreMetadata.BlobFileToDelete> blobFilesToDelete;
 
-    public BlobCoreMetadataBuilder(String sharedBlobName, long generation) {
-        this.sharedBlobName = sharedBlobName;
-        this.generation= generation;
-        this.blobFiles = new HashSet<>();
-        this.blobFilesToDelete = new HashSet<>();
-    }
+  public BlobCoreMetadataBuilder(String sharedBlobName, long generation) {
+    this.sharedBlobName = sharedBlobName;
+    this.generation= generation;
+    this.blobFiles = new HashSet<>();
+    this.blobFilesToDelete = new HashSet<>();
+  }
 
-    /**
-     * Builder used for "cloning" then modifying an existing instance of {@link BlobCoreMetadata}.
-     * The new generation has to be passed in because it is final and can't be set later.
-     */
-    public BlobCoreMetadataBuilder(BlobCoreMetadata bcm, long generation) {
-        this.sharedBlobName = bcm.getSharedBlobName();
-        this.generation = generation;
-        this.blobFiles = new HashSet<>(Arrays.asList(bcm.getBlobFiles()));
-        this.blobFilesToDelete = new HashSet<>(Arrays.asList(bcm.getBlobFilesToDelete()));
-    }
+  /**
+   * Builder used for "cloning" then modifying an existing instance of {@link BlobCoreMetadata}.
+   * The new generation has to be passed in because it is final and can't be set later.
+   */
+  public BlobCoreMetadataBuilder(BlobCoreMetadata bcm, long generation) {
+    this.sharedBlobName = bcm.getSharedBlobName();
+    this.generation = generation;
+    this.blobFiles = new HashSet<>(Arrays.asList(bcm.getBlobFiles()));
+    this.blobFilesToDelete = new HashSet<>(Arrays.asList(bcm.getBlobFilesToDelete()));
+  }
 
-    public String getSharedBlobName() {
-        return this.sharedBlobName;
-    }
+  public String getSharedBlobName() {
+    return this.sharedBlobName;
+  }
 
-    /**
-     * Builds a {@link BlobCoreMetadata} for a non existing core of a given name.
-     */
-    static public BlobCoreMetadata buildEmptyCoreMetadata(String sharedBlobName) {
-        return (new BlobCoreMetadataBuilder(sharedBlobName, UNDEFINED_VALUE)).build();
-    }
+  /**
+   * Builds a {@link BlobCoreMetadata} for a non existing core of a given name.
+   */
+  static public BlobCoreMetadata buildEmptyCoreMetadata(String sharedBlobName) {
+    return (new BlobCoreMetadataBuilder(sharedBlobName, UNDEFINED_VALUE)).build();
+  }
 
-    /**
-     * Adds a file to the set of "active" files listed in the metadata
-     */
-    public BlobCoreMetadataBuilder addFile(BlobCoreMetadata.BlobFile f) {
-        this.blobFiles.add(f);
-        return this;
-    }
+  /**
+   * Adds a file to the set of "active" files listed in the metadata
+   */
+  public BlobCoreMetadataBuilder addFile(BlobCoreMetadata.BlobFile f) {
+    this.blobFiles.add(f);
+    return this;
+  }
 
-    /**
-     * Removes a file from the set of "active" files listed in the metadata
-     */
-    public BlobCoreMetadataBuilder removeFile(BlobCoreMetadata.BlobFile f) {
-        boolean removed = this.blobFiles.remove(f);
-        assert removed; // If we remove things that are not there, likely a bug in our code
-        return this;
-    }
+  /**
+   * Removes a file from the set of "active" files listed in the metadata
+   */
+  public BlobCoreMetadataBuilder removeFile(BlobCoreMetadata.BlobFile f) {
+    boolean removed = this.blobFiles.remove(f);
+    assert removed; // If we remove things that are not there, likely a bug in our code
+    return this;
+  }
     
-    /**
-     * Adds a file to the set of files to delete listed in the metadata<p>
-     * This method should always be called with {@link #removeFile(BlobCoreMetadata.BlobFile)} above. Possibly it's
-     * better to only have a single method doing both operations (TODO).
-     */
-    public BlobCoreMetadataBuilder addFileToDelete(BlobCoreMetadata.BlobFileToDelete f) {
-        this.blobFilesToDelete.add(f);
-        return this;
-    }
+  /**
+   * Adds a file to the set of files to delete listed in the metadata<p>
+   * This method should always be called with {@link #removeFile(BlobCoreMetadata.BlobFile)} above. Possibly it's
+   * better to only have a single method doing both operations (TODO).
+   */
+  public BlobCoreMetadataBuilder addFileToDelete(BlobCoreMetadata.BlobFileToDelete f) {
+    this.blobFilesToDelete.add(f);
+    return this;
+  }
     
-    /**
-     * Returns an iterator on the set of files to delete.
-     * The returned iterator will be used to remove files from the set (as they are enqueued for hard delete from the Blob store).
-     */
-    public Iterator<BlobCoreMetadata.BlobFileToDelete> getDeletedFilesIterator() {
-        return this.blobFilesToDelete.iterator();
-    }
+  /**
+   * Returns an iterator on the set of files to delete.
+   * The returned iterator will be used to remove files from the set (as they are enqueued for hard delete from the Blob store).
+   */
+  public Iterator<BlobCoreMetadata.BlobFileToDelete> getDeletedFilesIterator() {
+    return this.blobFilesToDelete.iterator();
+  }
     
-    /**
-     * Removes a file from the set of "deleted" files listed in the metadata
-     */
-    public BlobCoreMetadataBuilder removeFilesFromDeleted(Set<BlobCoreMetadata.BlobFileToDelete> files) {
-        int originalSize = this.blobFilesToDelete.size();
-        boolean removed = this.blobFilesToDelete.removeAll(files);
-        int totalRemoved = originalSize - this.blobFilesToDelete.size();
-        
-        // If we remove things that are not there, likely a bug in our code
-        assert removed && (totalRemoved == files.size()); 
-        return this;
-    }
+  /**
+   * Removes a file from the set of "deleted" files listed in the metadata
+   */
+  public BlobCoreMetadataBuilder removeFilesFromDeleted(Set<BlobCoreMetadata.BlobFileToDelete> files) {
+    int originalSize = this.blobFilesToDelete.size();
+    boolean removed = this.blobFilesToDelete.removeAll(files);
+    int totalRemoved = originalSize - this.blobFilesToDelete.size();
+    
+    // If we remove things that are not there, likely a bug in our code
+    assert removed && (totalRemoved == files.size()); 
+    return this;
+  }
 
-    public BlobCoreMetadata build() {
-        // TODO make this fail if we find more than one segments_N files.
-        BlobCoreMetadata.BlobFile[] blobFilesArray = this.blobFiles.toArray(new BlobCoreMetadata.BlobFile[this.blobFiles.size()]);
-        BlobCoreMetadata.BlobFileToDelete[] blobFilesToDeleteArray = this.blobFilesToDelete.toArray(new BlobCoreMetadata.BlobFileToDelete[this.blobFilesToDelete.size()]);
+  public BlobCoreMetadata build() {
+    // TODO make this fail if we find more than one segments_N files.
+    BlobCoreMetadata.BlobFile[] blobFilesArray = this.blobFiles.toArray(new BlobCoreMetadata.BlobFile[this.blobFiles.size()]);
+    BlobCoreMetadata.BlobFileToDelete[] blobFilesToDeleteArray = this.blobFilesToDelete.toArray(new BlobCoreMetadata.BlobFileToDelete[this.blobFilesToDelete.size()]);
 
-        return new BlobCoreMetadata(this.sharedBlobName, blobFilesArray, blobFilesToDeleteArray, generation);
-    }
+    return new BlobCoreMetadata(this.sharedBlobName, blobFilesArray, blobFilesToDeleteArray, generation);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobException.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobException.java
index 4c34aef..372ddec 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobException.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobException.java
@@ -20,15 +20,15 @@ package org.apache.solr.store.blob.client;
  * Parent class of blob store related issues. Likely to change and maybe disappear but good enough for a PoC.
  */
 public class BlobException extends Exception {
-    public BlobException(Throwable cause) {
-        super(cause);
-    }
+  public BlobException(Throwable cause) {
+    super(cause);
+  }
 
-    public BlobException(String message) {
-        super(message);
-    }
+  public BlobException(String message) {
+    super(message);
+  }
 
-    public BlobException(String message, Throwable cause) {
-        super(message, cause);
-    }
+  public BlobException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/BlobServiceException.java b/solr/core/src/java/org/apache/solr/store/blob/client/BlobServiceException.java
index 66ebcaa..792043d 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/BlobServiceException.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/BlobServiceException.java
@@ -24,15 +24,15 @@ package org.apache.solr.store.blob.client;
  */
 public class BlobServiceException extends BlobException {
     
-    public BlobServiceException(Throwable cause) {
-        super(cause);
-    }
+  public BlobServiceException(Throwable cause) {
+    super(cause);
+  }
 
-    public BlobServiceException(String message) {
-        super(message);
-    }
+  public BlobServiceException(String message) {
+    super(message);
+  }
 
-    public BlobServiceException(String message, Throwable cause) {
-        super(message, cause);
-    }
+  public BlobServiceException(String message, Throwable cause) {
+    super(message, cause);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/CoreStorageClient.java b/solr/core/src/java/org/apache/solr/store/blob/client/CoreStorageClient.java
index 280839e..5bf07db 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/CoreStorageClient.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/CoreStorageClient.java
@@ -10,140 +10,140 @@ import java.util.List;
  */
 public interface CoreStorageClient {
     
-    /**
-     * Replaces the special CORE_METADATA_BLOB_FILENAME blob on the blob store for the core by a new version passed as a
-     * {@link BlobCoreMetadata} instance.
-     * 
-     * @param sharedStoreName name of the shared shard index data to write the metadata file for
-     * @param blobCoreMetadataName name of metadata file to write to the blob store
-     * @param bcm blob metadata to be serialized and written to the blob store
-     */
-    public void pushCoreMetadata(String sharedStoreName, String blobCoreMetadataName, BlobCoreMetadata bcm) throws BlobException;
+  /**
+   * Replaces the special CORE_METADATA_BLOB_FILENAME blob on the blob store for the core by a new version passed as a
+   * {@link BlobCoreMetadata} instance.
+   * 
+   * @param sharedStoreName name of the shared shard index data to write the metadata file for
+   * @param blobCoreMetadataName name of metadata file to write to the blob store
+   * @param bcm blob metadata to be serialized and written to the blob store
+   */
+  public void pushCoreMetadata(String sharedStoreName, String blobCoreMetadataName, BlobCoreMetadata bcm) throws BlobException;
     
-    /**
-     * Reads the special CORE_METADATA_BLOB_FILENAME blob on the blob store for the core and returns the corresponding
-     * {@link BlobCoreMetadata} object.
-     * 
-     * @param sharedStoreName name of the shared shard index data to get metadata for
-     * @param blobCoreMetadataName name of metadata file to pull from the blob store
-     * @return <code>null</code> if the core does not exist on the Blob store or method {@link #pushCoreMetadata} was
-     * never called for it. Otherwise returns the latest value written using {@link #pushCoreMetadata} ("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 BlobCoreMetadata pullCoreMetadata(String sharedStoreName, String blobCoreMetadataName) throws BlobException;
+  /**
+   * Reads the special CORE_METADATA_BLOB_FILENAME blob on the blob store for the core and returns the corresponding
+   * {@link BlobCoreMetadata} object.
+   * 
+   * @param sharedStoreName name of the shared shard index data to get metadata for
+   * @param blobCoreMetadataName name of metadata file to pull from the blob store
+   * @return <code>null</code> if the core does not exist on the Blob store or method {@link #pushCoreMetadata} was
+   * never called for it. Otherwise returns the latest value written using {@link #pushCoreMetadata} ("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 BlobCoreMetadata pullCoreMetadata(String sharedStoreName, String blobCoreMetadataName) throws BlobException;
     
-    /**
-     * Returns an input stream for the given blob. The caller must close the stream when done.
-     * 
-     * @param path the blob file key for the file to be pulled
-     * @return the blob's input stream
-     */
-    public InputStream pullStream(String path) throws BlobException;
+  /**
+   * Returns an input stream for the given blob. The caller must close the stream when done.
+   * 
+   * @param path the blob file key for the file to be pulled
+   * @return the blob's input stream
+   */
+  public InputStream pullStream(String path) throws BlobException;
     
-    /**
-     * Writes to the external store using an input stream for the given core. The unique
-     * path to the written blob is returned.
-     * 
-     * @param blobName name of the core to be pulled from the store
-     * @param is input stream of the core
-     * @param contentLength size of the stream's data
-     * @param fileNamePrefix have this string followed by a "." be how the "filename" of the written blob starts
-     * @return the unique path to the written blob. Expected to be of the form /path1/.../pathn/_filenamePrefix_._random string_
-     */
-    public String pushStream(String blobName, InputStream is, long contentLength, String fileNamePrefix) throws BlobException;
+  /**
+   * Writes to the external store using an input stream for the given core. The unique
+   * path to the written blob is returned.
+   * 
+   * @param blobName name of the core to be pulled from the store
+   * @param is input stream of the core
+   * @param contentLength size of the stream's data
+   * @param fileNamePrefix have this string followed by a "." be how the "filename" of the written blob starts
+   * @return the unique path to the written blob. Expected to be of the form /path1/.../pathn/_filenamePrefix_._random string_
+   */
+  public String pushStream(String blobName, InputStream is, long contentLength, String fileNamePrefix) throws BlobException;
     
-    /**
-     * Checks if the shard index data with the given shard metadata file exists
-     * 
-     * @param sharedStoreName name of the shared shard index data to check exists
-     * @param blobCoreMetadataName name of metadata file to check exists
-     * @return true if the core has blobs
-     */
-    public boolean coreMetadataExists(String sharedStoreName, String blobCoreMetadataName) throws BlobException;
+  /**
+   * Checks if the shard index data with the given shard metadata file exists
+   * 
+   * @param sharedStoreName name of the shared shard index data to check exists
+   * @param blobCoreMetadataName name of metadata file to check exists
+   * @return true if the core has blobs
+   */
+  public boolean coreMetadataExists(String sharedStoreName, String blobCoreMetadataName) throws BlobException;
     
-    /**
-     * Deletes all blob files associated with this blobName.
-     * 
-     * @param blobName core to delete
-     */
-    public void deleteCore(String blobName) throws BlobException;
+  /**
+   * Deletes all blob files associated with this blobName.
+   * 
+   * @param blobName core to delete
+   */
+  public void deleteCore(String blobName) throws BlobException;
     
-    /**
-     * Batch delete blob files from the blob store. Any blob file path that specifies a non-existent blob file will 
-     * not be treated as an error and should return success.
-     * 
-     * @param paths list of blob file keys to the files to be deleted
-     */
-    public void deleteBlobs(Collection<String> paths) throws BlobException;
+  /**
+   * Batch delete blob files from the blob store. Any blob file path that specifies a non-existent blob file will 
+   * not be treated as an error and should return success.
+   * 
+   * @param paths list of blob file keys to the files to be deleted
+   */
+  public void deleteBlobs(Collection<String> paths) throws BlobException;
+  
+  /**
+   * Retrieves an identifier for the cloud service providing the blobstore
+   * 
+   * @return string identifying the service provider 
+   */
+  public BlobstoreProviderType getStorageProvider();
     
-    /**
-     * Retrieves an identifier for the cloud service providing the blobstore
-     * 
-     * @return string identifying the service provider 
-     */
-    public BlobstoreProviderType getStorageProvider();
+  /**
+   * Returns an identifier for the geographical region a bucket is located in. Most blob storage providers utilize the concept of regions. In the unlikely case that we use a provider that 
+   * doesn't, we should update this method (and log lines that mostly use this) to reflect a more generic terminology.
+   * 
+   * @return string identifying the bucket's geographical region
+   */
+  public String getBucketRegion();
     
-    /**
-     * Returns an identifier for the geographical region a bucket is located in. Most blob storage providers utilize the concept of regions. In the unlikely case that we use a provider that 
-     * doesn't, we should update this method (and log lines that mostly use this) to reflect a more generic terminology.
-     * 
-     * @return string identifying the bucket's geographical region
-     */
-    public String getBucketRegion();
+  /**
+   * Returns the name of the bucket that this client will interact with
+   * 
+   * @return string name of the bucket this client is configured with
+   */
+  public String getBucketName();
     
-    /**
-     * Returns the name of the bucket that this client will interact with
-     * 
-     * @return string name of the bucket this client is configured with
-     */
-    public String getBucketName();
+  /**
+   * Returns the the end point this client is configured with
+   * 
+   * @return string name of the endpoint this client is configured with
+   */
+  public String getEndpoint();
     
-    /**
-     * Returns the the end point this client is configured with
-     * 
-     * @return string name of the endpoint this client is configured with
-     */
-    public String getEndpoint();
+  /**
+   * Tests whether or not a bucket exists. Throws an exception if something is wrong or fa
+   * 
+   * @return true if it does
+   */
+  public boolean doesBucketExist() throws BlobException;
     
-    /**
-     * Tests whether or not a bucket exists. Throws an exception if something is wrong or fa
-     * 
-     * @return true if it does
-     */
-    public boolean doesBucketExist() throws BlobException;
+  /**
+   * Closes any resources used by the core storage client
+   */
+  public void shutdown();
+  
+  /**
+   * Lists the blob file names of all of files listed under a given core name's blob store
+   * hierarchy that are older than the given timestamp value in milliseconds. Important to 
+   * note that that the wall clock of your caller will vary with that of the blob store service
+   * such that you'll likely see mismatch if you're trying to list very recent files.
+   * 
+   * This method is intended for observing significantly older modified files where clock skew
+   * is less of an issue.
+   * 
+   * @param blobName the core to be listed
+   * @param timestamp timestamp in milliseconds
+   * @return the list of blob files
+   */
+  public List<String> listCoreBlobFilesOlderThan(String blobName, long timestamp) throws BlobException;
     
-    /**
-     * Closes any resources used by the core storage client
-     */
-    public void shutdown();
-    
-    /**
-     * Lists the blob file names of all of files listed under a given core name's blob store
-     * hierarchy that are older than the given timestamp value in milliseconds. Important to 
-     * note that that the wall clock of your caller will vary with that of the blob store service
-     * such that you'll likely see mismatch if you're trying to list very recent files.
-     * 
-     * This method is intended for observing significantly older modified files where clock skew
-     * is less of an issue.
-     * 
-     * @param blobName the core to be listed
-     * @param timestamp timestamp in milliseconds
-     * @return the list of blob files
-     */
-    public List<String> listCoreBlobFilesOlderThan(String blobName, long timestamp) throws BlobException;
-    
-    /**
-     * Lists the common delimiter-terminated prefixes by blob keys beginning with the
-     * provided prefix string.
-     * 
-     * e.g. "core1/name1", "core1/name2" and "core2/name2"
-     * Passing the prefix argument "core" should return
-     * "core1/" and "core2/" i.e. the common delimiter-terminated
-     * prefixes shared by blob keys in the blob store.
-     * 
-     * @param prefix of the blob key to list by
-     * @return the list of common prefixes
-     */
-    public List<String> listCommonBlobPrefix(String prefix) throws BlobException;
+  /**
+   * Lists the common delimiter-terminated prefixes by blob keys beginning with the
+   * provided prefix string.
+   * 
+   * e.g. "core1/name1", "core1/name2" and "core2/name2"
+   * Passing the prefix argument "core" should return
+   * "core1/" and "core2/" i.e. the common delimiter-terminated
+   * prefixes shared by blob keys in the blob store.
+   * 
+   * @param prefix of the blob key to list by
+   * @return the list of common prefixes
+   */
+  public List<String> listCommonBlobPrefix(String prefix) throws BlobException;
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/client/ToFromJson.java b/solr/core/src/java/org/apache/solr/store/blob/client/ToFromJson.java
index dd9b142..7c21577 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/client/ToFromJson.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/client/ToFromJson.java
@@ -23,22 +23,22 @@ import com.google.gson.GsonBuilder;
  * Serializing/deserializing Json (mostly for {@link org.apache.solr.store.blob.client.BlobCoreMetadata}).
  */
 public class ToFromJson<T> {
-    /** Create easier to (human) read but a bit longer json output */
-    static final boolean PRETTY_JSON = true;
+  /** Create easier to (human) read but a bit longer json output */
+  static final boolean PRETTY_JSON = true;
 
-    /**
-     * Builds an object instance from a String representation of the Json.
-     */
-    public T fromJson(String input, Class<T> c) throws Exception {
-        Gson gson = new Gson();
-        return gson.fromJson(input, c);
-    }
+  /**
+   * Builds an object instance from a String representation of the Json.
+   */
+  public T fromJson(String input, Class<T> c) throws Exception {
+    Gson gson = new Gson();
+    return gson.fromJson(input, c);
+  }
 
-    /**
-     * Returns the Json String of the passed object instance.
-     */
-    public String toJson(T t) throws Exception {
-        Gson gson = PRETTY_JSON ? new GsonBuilder().setPrettyPrinting().create() : new Gson();
-        return gson.toJson(t);
-    }
+  /**
+   * Returns the Json String of the passed object instance.
+   */
+  public String toJson(T t) throws Exception {
+    Gson gson = PRETTY_JSON ? new GsonBuilder().setPrettyPrinting().create() : new Gson();
+    return gson.toJson(t);
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
index fceb569..c352e50 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
@@ -41,358 +41,356 @@ import org.slf4j.LoggerFactory;
 import java.lang.invoke.MethodHandles;
 
 /**
- * Class to sync between local and blob copies of a core using {@link org.apache.solr.store.blob.metadata.CorePushPull}
+ * Class to sync between local and blob copies of a core using
+ * {@link org.apache.solr.store.blob.metadata.CorePushPull}
  */
 public class BlobCoreSyncer {
-
-    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-    /**
-     * Threads wait for at most this duration before giving up on async pull to finish and returning with a PullInProgressException.
-     */
-    public static final long PULL_WAITING_MS = TimeUnit.SECONDS.toMillis(5);
-
-    /**
-     * Max number of threads for a core that can concurrently wait for the pull
-     * to complete instead of returning a PullInProgressException right away
-     */
-    public static final int MAX_PULL_WAITING_PER_CORE = 5;
-
-    /**
-     * Allow at most that number of threads to wait for async pulls to finish over all cores instead of returning
-     * a PullInProgressException right away.
-     */
-    public static final int MAX_PULL_WAITING_TOTAL = 20;
-
-    /** "Skipping pulling core" string is checked on Core App in SolrExchange.onResponseComplete()
-     * and defined there in constant SolrExchange.CORE_BEING_PULLED. Change here, change there!
-     * Use this string as prefix of any message returned in a PullInProgressException.
-     */
-    private static final String SKIPPING_PULLING_CORE = "Skipping pulling core";
-
-    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-    @GuardedBy("coreSyncsInFlight")
-    private int total_waiting_threads = 0;
-
-    /** The shared store name for the core currently being pulled from blob. Value is collection of objects used for synchronization by all waiting threads.
-     * If both the locks on this map and on a specific SyncOnPullWait in the map are needed, the lock on the map must be acquired first.
-     */
-    @GuardedBy("itself")
-    private final Map<String, Collection<SyncOnPullWait>> coreSyncsInFlight = Maps.newHashMap();
-
-    /**
-     * @return Total number of threads across all cores waiting for their respective core to be pulled from blob store
-     */
-    @VisibleForTesting
-    protected int getTotalWaitingThreads() {
-        synchronized (coreSyncsInFlight) {
-            return total_waiting_threads;
-        }
-    }
-
-    /**
-     * Each thread waiting for async pull to complete uses its own new instance of this class. This is done
-     * so that the pulling thread upon completion can do two things:
-     * <ol><li>
-     *    Notify each waiting (or about to wait) thread that the pull is done,
-     * </li><li>
-     *    Pass pull results to the waiting thread (for example indicate if an exception was thrown and the pull failed).
-     * </li></ol>
-     * A data structure in which all waiter threads share the same lock object (and the same flag/data indicating completion
-     * and its result) can be considered simpler but has the problem of when can this data structure be freed, often resorting
-     * to relatively complex code using weak references.<p>
-     * In the implementation done here, the entry in the map of cores to "waiting lists" ({@link BlobCoreSyncer#coreSyncsInFlight})
-     * can be removed at any time once all waiter threads have been notified, then the actual data structure for each
-     * waiter will be reclaimable by the JVM when the waiter has finished using it, without complex tricks.
-     */
-    private static class SyncOnPullWait {
-        private final CountDownLatch latch = new CountDownLatch(1);
-        private Exception exception = null; // If non null once latch is counted down (to 0), exception encountered by pulling thread
-    }
-
-    /**
-     * Returns a _hint_ that the given core might be locally empty because it is awaiting pull from Blob store.
-     * This is just a hint because as soon as the lock is released when the method returns, the status of the core could change.
-     * Because of that, in method {@link #pull(PushPullData, boolean, boolean, CoreContainer)} we need to check again.
-     */
-    public boolean isEmptyCoreAwaitingPull(CoreContainer cores, String coreName) {
-      return CorePullerFeeder.isEmptyCoreAwaitingPull(cores, coreName);
+  
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  /**
+   * Threads wait for at most this duration before giving up on async pull to finish and returning with a PullInProgressException.
+   */
+  public static final long PULL_WAITING_MS = TimeUnit.SECONDS.toMillis(5);
+
+  /**
+   * Max number of threads for a core that can concurrently wait for the pull
+   * to complete instead of returning a PullInProgressException right away
+   */
+  public static final int MAX_PULL_WAITING_PER_CORE = 5;
+
+  /**
+   * Allow at most that number of threads to wait for async pulls to finish over all cores instead of returning
+   * a PullInProgressException right away.
+   */
+  public static final int MAX_PULL_WAITING_TOTAL = 20;
+
+  /** "Skipping pulling core" string is checked on Core App in SolrExchange.onResponseComplete()
+   * and defined there in constant SolrExchange.CORE_BEING_PULLED. Change here, change there!
+   * Use this string as prefix of any message returned in a PullInProgressException.
+   */
+  private static final String SKIPPING_PULLING_CORE = "Skipping pulling core";
+
+  @GuardedBy("coreSyncsInFlight")
+  private int total_waiting_threads = 0;
+
+  /** The shared store name for the core currently being pulled from blob. Value is collection of objects used for synchronization by all waiting threads.
+   * If both the locks on this map and on a specific SyncOnPullWait in the map are needed, the lock on the map must be acquired first.
+   */
+  @GuardedBy("itself")
+  private final Map<String, Collection<SyncOnPullWait>> coreSyncsInFlight = Maps.newHashMap();
+
+  /**
+   * @return Total number of threads across all cores waiting for their respective core to be pulled from blob store
+   */
+  @VisibleForTesting
+  protected int getTotalWaitingThreads() {
+    synchronized (coreSyncsInFlight) {
+      return total_waiting_threads;
     }
+  }
+
+  /**
+   * Each thread waiting for async pull to complete uses its own new instance of this class. This is done
+   * so that the pulling thread upon completion can do two things:
+   * <ol><li>
+   *    Notify each waiting (or about to wait) thread that the pull is done,
+   * </li><li>
+   *    Pass pull results to the waiting thread (for example indicate if an exception was thrown and the pull failed).
+   * </li></ol>
+   * A data structure in which all waiter threads share the same lock object (and the same flag/data indicating completion
+   * and its result) can be considered simpler but has the problem of when can this data structure be freed, often resorting
+   * to relatively complex code using weak references.<p>
+   * In the implementation done here, the entry in the map of cores to "waiting lists" ({@link BlobCoreSyncer#coreSyncsInFlight})
+   * can be removed at any time once all waiter threads have been notified, then the actual data structure for each
+   * waiter will be reclaimable by the JVM when the waiter has finished using it, without complex tricks.
+   */
+  private static class SyncOnPullWait {
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private Exception exception = null; // If non null once latch is counted down (to 0), exception encountered by pulling thread
+  }
+
+  /**
+   * Returns a _hint_ that the given core might be locally empty because it is awaiting pull from Blob store.
+   * This is just a hint because as soon as the lock is released when the method returns, the status of the core could change.
+   * Because of that, in method {@link #pull(PushPullData, boolean, boolean, CoreContainer)} we need to check again.
+   */
+  public boolean isEmptyCoreAwaitingPull(CoreContainer cores, String coreName) {
+    return CorePullerFeeder.isEmptyCoreAwaitingPull(cores, coreName);
+  }
     
-    public void pull(String coreName, String shardName, String collectionName, CoreContainer cores,
-        boolean waitForSearcher, boolean emptyCoreAwaitingPull) {
-      // Initialize variables
-      SharedShardMetadataController sharedShardMetadataController = cores.getSharedStoreManager().getSharedShardMetadataController();
-      DocCollection collection = cores.getZkController().getClusterState().getCollection(collectionName);
-
-      Slice shard = collection.getSlicesMap().get(shardName);
-      if (shard != null) {
-        try {
-          if (!collection.getActiveSlices().contains(shard)) {
-            // unclear if there are side effects but logging for now
-            log.warn("Pulling shard " + shardName + " that is inactive!");
-          }
-          log.info("Pulling for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-          // creates the metadata node if it doesn't exist
-          sharedShardMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-
-          /*
-           * Get the metadataSuffix value from ZooKeeper or from a cache if an entry exists for the 
-           * given collection and shardName. If the leader has already changed, the conditional update
-           * later will fail and invalidate the cache entry if it exists. 
-           */
-          VersionedData data = sharedShardMetadataController.readMetadataValue(collectionName, shardName, 
-              /* readFromCache */ false);
-          Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
-          String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
-
-          String sharedShardName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
-
-          PushPullData pushPullData = new PushPullData.Builder()
-              .setCollectionName(collectionName)
-              .setShardName(shardName)
-              .setCoreName(coreName)
-              .setSharedStoreName(sharedShardName)
-              .setLastReadMetadataSuffix(metadataSuffix)
-              .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
-              .setZkVersion(data.getVersion())
-              .build();
-          pull(pushPullData, waitForSearcher, emptyCoreAwaitingPull, cores);
-        } catch (Exception ex) {
-          // wrap every thrown exception in a solr exception
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error trying to pull from blob store", ex);
+  public void pull(String coreName, String shardName, String collectionName, CoreContainer cores,
+      boolean waitForSearcher, boolean emptyCoreAwaitingPull) {
+    // Initialize variables
+    SharedShardMetadataController sharedShardMetadataController = cores.getSharedStoreManager().getSharedShardMetadataController();
+    DocCollection collection = cores.getZkController().getClusterState().getCollection(collectionName);
+
+    Slice shard = collection.getSlicesMap().get(shardName);
+    if (shard != null) {
+      try {
+        if (!collection.getActiveSlices().contains(shard)) {
+          // unclear if there are side effects but logging for now
+          log.warn("Pulling shard " + shardName + " that is inactive!");
         }
+        log.info("Pulling for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
+        // creates the metadata node if it doesn't exist
+        sharedShardMetadataController.ensureMetadataNodeExists(collectionName, shardName);
+
+        /*
+         * Get the metadataSuffix value from ZooKeeper or from a cache if an entry exists for the
+         * given collection and shardName. If the leader has already changed, the conditional update
+         * later will fail and invalidate the cache entry if it exists.
+         */
+        VersionedData data = sharedShardMetadataController.readMetadataValue(collectionName, shardName,
+            /* readFromCache */ false);
+        Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
+        String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
+
+        String sharedShardName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
+
+        PushPullData pushPullData = new PushPullData.Builder()
+          .setCollectionName(collectionName)
+          .setShardName(shardName)
+          .setCoreName(coreName)
+          .setSharedStoreName(sharedShardName)
+          .setLastReadMetadataSuffix(metadataSuffix)
+          .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
+          .setZkVersion(data.getVersion())
+          .build();
+        pull(pushPullData, waitForSearcher, emptyCoreAwaitingPull, cores);
+      } catch (Exception ex) {
+        // wrap every thrown exception in a solr exception
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error trying to pull from blob store", ex);
+      }
+    }
+  }
+
+  /**
+   * This method is used to pull in updates from blob when there is no local copy of the core available and when a thread
+   * needs the core to be available (i.e. waits for it).<p>
+   *
+   * A parameter <code>waitForSearcher</code> is therefore added and is always <code>true</code>
+   * as of today, but it is added so any additional calls to this method need to decide if they want wait for searcher or not.
+   * Note that when multiple threads wait for the same core, currently only the puller will create the index searcher and
+   * either wait for it to be available (waitForSearcher) or not (!waitForSearcher). If another thread waiting for the pull to complete wanted
+   * a waitForSearcher and the puller thread did not, the code should be modified. This doesn't happen as of today because this
+   * method is always called with waitForSearcher being true...
+   *
+   * TODO if Blob is not available on pull, we might treat this as a missing core exception from Core app and reindex. Likely need to treat Blob unavailable differently from no such core on blob
+   *
+   * @param emptyCoreAwaitingPull <code>true</code> if this pull method is called after the core got created empty locally
+   *                              but before the actual pull happened, in which case this method is called to wait until
+   *                              the pull finishes to avoid erroneously returning no results.
+   *
+   * @throws PullInProgressException In case a thread does not wait or times out before the async pull is finished
+   */
+  public void pull(PushPullData pushPullData, boolean waitForSearcher, boolean emptyCoreAwaitingPull, CoreContainer cores) throws PullInProgressException {
+    // Is there another thread already working on the async pull?
+    final boolean pullAlreadyInProgress;
+    // Indicates if thread waits for the pull to finish or too many waiters already
+    final boolean iWait;
+    // When iWait (is true), myWait is the object to use for the wait. Note pull thread might have completed before
+    // we start wait, so we need to test the state of the myWait object first.
+    final SyncOnPullWait myWait;
+
+    // Capturing the number of waiting threads to log the reason we can't wait... This is mostly for autobuild debug and likely to be removed afterwards. -1 should never be logged
+    int countCoreWaiters = -1;
+    int countTotalWaiters = -1;
+
+    // Only can have only one thread working on async pull of this core (and we do no logging while holding the lock)
+    // Let's understand what our role and actions are while holding the global lock and then execute on them without the lock.
+    synchronized (coreSyncsInFlight) {
+      if (emptyCoreAwaitingPull && !isEmptyCoreAwaitingPull(cores, pushPullData.getCoreName())) {
+        // Core was observed empty awaiting pull and is no longer awaiting pull. This means the pull happened.
+        return;
+      }
+      countTotalWaiters = total_waiting_threads; // for logging
+
+      Collection<SyncOnPullWait> collectionOfWaiters = coreSyncsInFlight.get(pushPullData.getSharedStoreName());
+      if (collectionOfWaiters != null) {
+        // Somebody is already working on async pull of this core. If possible let's add ourselves to the list of those who wait (then wait)
+        pullAlreadyInProgress = true;
+      } else {
+        // We are the first thread trying to pull this core, before releasing the lock we must setup everything
+        // to let the world know and let itself and other threads be able to wait until async pull is done if/when they so decide.
+        collectionOfWaiters = new ArrayList<>(MAX_PULL_WAITING_PER_CORE);
+        coreSyncsInFlight.put(pushPullData.getSharedStoreName(), collectionOfWaiters);
+        pullAlreadyInProgress = false;
+      }
+      int waiters = collectionOfWaiters.size();
+
+      countCoreWaiters = waiters; // for logging
+
+      iWait = total_waiting_threads < MAX_PULL_WAITING_TOTAL && waiters < MAX_PULL_WAITING_PER_CORE;
+      if (iWait) {
+        myWait = new SyncOnPullWait();
+        // Increase total waiting count and add ourselves as waiting for the current core.
+        total_waiting_threads++;
+        collectionOfWaiters.add(myWait);
+      } else {
+        // We cannot throw pull in progress exception here because this whole synchronized block is calculating
+        // subsequent actions and we might need to enqueue a pull before throwing that exception i.e. (!pullAlreadyInProgress)
+        //
+        // We can make it work that way if we really want to(although I don't see a reason) if we separate out this waiters calculation
+        // into a separate synchronized block after the enqueuing of pull request block
+        myWait = null; // Not used but has to be set
       }
     }
 
-    /**
-     * This method is used to pull in updates from blob when there is no local copy of the core available and when a thread
-     * needs the core to be available (i.e. waits for it).<p>
-     *
-     * A parameter <code>waitForSearcher</code> is therefore added and is always <code>true</code>
-     * as of today, but it is added so any additional calls to this method need to decide if they want wait for searcher or not.
-     * Note that when multiple threads wait for the same core, currently only the puller will create the index searcher and
-     * either wait for it to be available (waitForSearcher) or not (!waitForSearcher). If another thread waiting for the pull to complete wanted
-     * a waitForSearcher and the puller thread did not, the code should be modified. This doesn't happen as of today because this
-     * method is always called with waitForSearcher being true...
-     *
-     * TODO if Blob is not available on pull, we might treat this as a missing core exception from Core app and reindex. Likely need to treat Blob unavailable differently from no such core on blob
-     *
-     * @param emptyCoreAwaitingPull <code>true</code> if this pull method is called after the core got created empty locally
-     *                              but before the actual pull happened, in which case this method is called to wait until
-     *                              the pull finishes to avoid erroneously returning no results.
-     *
-     * @throws PullInProgressException In case a thread does not wait or times out before the async pull is finished
-     */
-    public void pull(PushPullData pushPullData, boolean waitForSearcher, boolean emptyCoreAwaitingPull, CoreContainer cores) throws PullInProgressException {
-        // Is there another thread already working on the async pull?
-        final boolean pullAlreadyInProgress;
-        // Indicates if thread waits for the pull to finish or too many waiters already
-        final boolean iWait;
-        // When iWait (is true), myWait is the object to use for the wait. Note pull thread might have completed before
-        // we start wait, so we need to test the state of the myWait object first.
-        final SyncOnPullWait myWait;
-
-        // Capturing the number of waiting threads to log the reason we can't wait... This is mostly for autobuild debug and likely to be removed afterwards. -1 should never be logged
-        int countCoreWaiters = -1;
-        int countTotalWaiters = -1;
-
-        // Only can have only one thread working on async pull of this core (and we do no logging while holding the lock)
-        // Let's understand what our role and actions are while holding the global lock and then execute on them without the lock.
-        synchronized (coreSyncsInFlight) {
-            if (emptyCoreAwaitingPull && !isEmptyCoreAwaitingPull(cores, pushPullData.getCoreName())) {
-                // Core was observed empty awaiting pull and is no longer awaiting pull. This means the pull happened.
-                return;
-            }
-            countTotalWaiters = total_waiting_threads; // for logging
-
-            Collection<SyncOnPullWait> collectionOfWaiters = coreSyncsInFlight.get(pushPullData.getSharedStoreName());
-            if (collectionOfWaiters != null) {
-                // Somebody is already working on async pull of this core. If possible let's add ourselves to the list of those who wait (then wait)
-                pullAlreadyInProgress = true;
-            } else {
-                // We are the first thread trying to pull this core, before releasing the lock we must setup everything
-                // to let the world know and let itself and other threads be able to wait until async pull is done if/when they so decide.
-                collectionOfWaiters = new ArrayList<>(MAX_PULL_WAITING_PER_CORE);
-                coreSyncsInFlight.put(pushPullData.getSharedStoreName(), collectionOfWaiters);
-                pullAlreadyInProgress = false;
-            }
-            int waiters = collectionOfWaiters.size();
-
-            countCoreWaiters = waiters; // for logging
-
-            iWait = total_waiting_threads < MAX_PULL_WAITING_TOTAL && waiters < MAX_PULL_WAITING_PER_CORE;
-            if (iWait) {
-                myWait = new SyncOnPullWait();
-                // Increase total waiting count and add ourselves as waiting for the current core.
-                total_waiting_threads++;
-                collectionOfWaiters.add(myWait);
-            } else {
-                // We cannot throw pull in progress exception here because this whole synchronized block is calculating
-                // subsequent actions and we might need to enqueue a pull before throwing that exception i.e. (!pullAlreadyInProgress)
-                //
-                // We can make it work that way if we really want to(although I don't see a reason) if we separate out this waiters calculation
-                // into a separate synchronized block after the enqueuing of pull request block
-                myWait = null; // Not used but has to be set
-            }
-        }
-
-        // Verify that emptyCoreAwaitingPull implies pullAlreadyInProgress: if the core was previously observed empty awaiting pull,
-        // it means a thread is already on it (and if that thread has finished we would have returned earlier from this call)
-        assert !emptyCoreAwaitingPull || pullAlreadyInProgress;
-
-        if (!pullAlreadyInProgress) {
-            // We are the first in here for that core so we get to enqueue async pull.
-            // If we are successful in enqueuing then pullFinished callback will take care of notifying and clearing
-            // out state around waiting threads(including this one if it end up waiting as well in the next block).
-            // That callback will take care of both successful and failed pulls.
-            // But if we fail in enqueuing then catch block here will declare pull as failed and will take care of notifying
-            // and clearing out state around waiting threads (it is possible that another thread has already started waiting before this thread
-            // get a chance to run this block)
-            try {
-                log.info("About to enqueue pull of core " + pushPullData.getSharedStoreName() + " (countTotalWaiters=" + countTotalWaiters + ")");
-
-                // enqueue an async pull
-                CorePullTracker corePullTracker = cores.getSharedStoreManager().getCorePullTracker();
-                corePullTracker.enqueueForPull(pushPullData, true, waitForSearcher);
-
-            } catch (Exception e) {
-                // as mentioned above in case of failed enqueue we are responsible for clearing up all waiting state
-                notifyEndOfPull(pushPullData.getSharedStoreName(), e);
-                if (e instanceof InterruptedException) {
-                    // We swallow the InterruptedException and spit instead a SolrException.
-                    // Need to let layers higher up know an interruption happened.
-                    Thread.currentThread().interrupt();
-                }
-
-                String msg = "Failed to enqueue pull of core " + pushPullData.getSharedStoreName() + " from blob";
-                SolrException se;
-                log.warn(msg, e);
-                if (e instanceof SolrException) {
-                    se = (SolrException) e;
-                } else {
-                    // Wrapping in SolrException to percolate all the way to the catch Throwable at end of SolrDispatchFilter.doFilter()
-                    // A failed pull should not appear as a missing core otherwise Core App reindexes in another core...
-                    // TODO remains to be seen if all pull's() need to have this exception thrown or if some prefer silent failures.
-                    se = new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
-                }
-                throw se;
-            }
+    // Verify that emptyCoreAwaitingPull implies pullAlreadyInProgress: if the core was previously observed empty awaiting pull,
+    // it means a thread is already on it (and if that thread has finished we would have returned earlier from this call)
+    assert !emptyCoreAwaitingPull || pullAlreadyInProgress;
+
+    if (!pullAlreadyInProgress) {
+      // We are the first in here for that core so we get to enqueue async pull.
+      // If we are successful in enqueuing then pullFinished callback will take care of notifying and clearing
+      // out state around waiting threads(including this one if it end up waiting as well in the next block).
+      // That callback will take care of both successful and failed pulls.
+      // But if we fail in enqueuing then catch block here will declare pull as failed and will take care of notifying
+      // and clearing out state around waiting threads (it is possible that another thread has already started waiting before this thread
+      // get a chance to run this block)
+      try {
+        log.info("About to enqueue pull of core " + pushPullData.getSharedStoreName() + " (countTotalWaiters=" + countTotalWaiters + ")");
+
+        // enqueue an async pull
+        CorePullTracker corePullTracker = cores.getSharedStoreManager().getCorePullTracker();
+        corePullTracker.enqueueForPull(pushPullData, true, waitForSearcher);
+
+      } catch (Exception e) {
+        // as mentioned above in case of failed enqueue we are responsible for clearing up all waiting state
+        notifyEndOfPull(pushPullData.getSharedStoreName(), e);
+        if (e instanceof InterruptedException) {
+          // We swallow the InterruptedException and spit instead a SolrException.
+          // Need to let layers higher up know an interruption happened.
+          Thread.currentThread().interrupt();
         }
 
-        // Now irrespective of us being a thread who initiated an async pull or just came after that, we might have
-        // qualified for waiting for async pull to finish. If so we will just do that.
-        if (iWait) {
-            try {
-                log.info("About to wait for pull of core " + pushPullData.getSharedStoreName() + " (countCoreWaiters=" + countCoreWaiters + " countTotalWaiters=" + countTotalWaiters + ")");
-
-                // Let's wait a bit maybe the pull completes in which case we don't have to throw an exception back.
-                // The other end of this lock activity happens in notifyEndOfPull() below
-                try {
-                    // The pull might have completed by now (i.e. completed since we left the synchronized(coreSyncsInFlight) block)
-                    // and that's ok the await() below will not block if not needed
-                    myWait.latch.await(PULL_WAITING_MS, TimeUnit.MILLISECONDS);
-                } catch (InterruptedException ie) {
-                    Thread.currentThread().interrupt();
-                    throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Interrupted while waiting for core pull " + pushPullData.getSharedStoreName(), ie);
-                }
-
-                // Our wait finished...
-                // Memory consistency effects of CountDownLatch: Until the count reaches zero, actions in a thread prior
-                // to calling countDown() happen-before actions following a successful return from a corresponding await() in another thread.
-                // We'll therefore see any updates done to the myWait object by the puller thread if it has finished pulling.
-                if (myWait.latch.getCount() != 0) {
-                    throwPullInProgressException(pushPullData.getSharedStoreName(), "still in progress and didn't complete while we waited");
-                } else {
-                    // Pull has completed. Success or failure?
-                    if (myWait.exception != null) {
-                        // We wrap in a SolrException the original exception from the puller thread.
-                        // (even if it's already a SolrException to make understanding what happens at runtime easier)
-                        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Puller thread failed to pull", myWait.exception);
-                    }
-
-                    // The pull has finished during our wait.
-                }
-            } finally {
-                // if wait is ended because of notifyEndOfPull() then it would have taken care of this state already and this essentially will be a no op.
-                // But if it has ended because of some other reason e.g. InterruptedException, then we will remove this thread from list of waiters.
-                //
-                // pullFinished notification may come later but that should be ok since it will only see waiters that are still waiting or none.
-                synchronized (coreSyncsInFlight) {
-                    Collection<SyncOnPullWait> collectionOfWaiters = coreSyncsInFlight.get(pushPullData.getSharedStoreName());
-                    if(collectionOfWaiters != null) {
-                        if(collectionOfWaiters.remove(myWait)){
-                            total_waiting_threads--;
-                        }
-                    }
-                }
-            }
+        String msg = "Failed to enqueue pull of core " + pushPullData.getSharedStoreName() + " from blob";
+        SolrException se;
+        log.warn(msg, e);
+        if (e instanceof SolrException) {
+          se = (SolrException) e;
         } else {
-            // Pull is in progress and we're not waiting for it to finish.
-            throwPullInProgressException(pushPullData.getSharedStoreName(), "as it is already in progress and enough threads waiting");
+          // Wrapping in SolrException to percolate all the way to the catch Throwable at end of SolrDispatchFilter.doFilter()
+          // A failed pull should not appear as a missing core otherwise Core App reindexes in another core...
+          // TODO remains to be seen if all pull's() need to have this exception thrown or if some prefer silent failures.
+          se = new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
         }
+        throw se;
+      }
     }
 
-    /**
-     * This is called whenever core from {@link CorePullTracker} finish its async pull(successfully or unsuccessfully)
-     * We use this to notify all waiting threads for a core that their wait has ended (if there are some waiting).
-     */
-    public void finishedPull(String sharedStoreName, CoreSyncStatus status, BlobCoreMetadata blobMetadata, String message) {
-        Exception pullException = null;
-        final boolean isPullSuccessful = (status.isSuccess() ||
-                // Following statuses are not considered success in strictest definition of pull but for BlobSyncer
-                // they are not error either and we would let the normal query flow do its job (likely return missing core exception)
-                status == CoreSyncStatus.BLOB_MISSING ||
-                status == CoreSyncStatus.BLOB_DELETED_FOR_PULL ||
-                // TODO:  likely changes needed here for W-5388477 Blob store corruption repair
-                status == CoreSyncStatus.BLOB_CORRUPT);
-        if (!isPullSuccessful) {
-            pullException = new SolrException(SolrException.ErrorCode.SERVER_ERROR, message);
-        }
-        notifyEndOfPull(sharedStoreName, pullException);
-    }
+    // Now irrespective of us being a thread who initiated an async pull or just came after that, we might have
+    // qualified for waiting for async pull to finish. If so we will just do that.
+    if (iWait) {
+      try {
+        log.info("About to wait for pull of core " + pushPullData.getSharedStoreName() + " (countCoreWaiters=" + countCoreWaiters + " countTotalWaiters=" + countTotalWaiters + ")");
 
-    private void throwPullInProgressException(String corename, String msgSuffix) throws PullInProgressException {
-        String msg = SKIPPING_PULLING_CORE + " " + corename + " from blob " + msgSuffix;
-        log.info(msg);
-        // Note that longer term, this is the place where we could decide that if the async
-        // pull was enqueued too long ago and nothing happened, then there's an issue worth gacking/alerting for.
-        throw new PullInProgressException(msg);
+        // Let's wait a bit maybe the pull completes in which case we don't have to throw an exception back.
+        // The other end of this lock activity happens in notifyEndOfPull() below
+        try {
+          // The pull might have completed by now (i.e. completed since we left the synchronized(coreSyncsInFlight) block)
+          // and that's ok the await() below will not block if not needed
+          myWait.latch.await(PULL_WAITING_MS, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException ie) {
+          Thread.currentThread().interrupt();
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Interrupted while waiting for core pull " + pushPullData.getSharedStoreName(), ie);
+        }
 
-    }
+        // Our wait finished...
+        // Memory consistency effects of CountDownLatch: Until the count reaches zero, actions in a thread prior
+        // to calling countDown() happen-before actions following a successful return from a corresponding await() in another thread.
+        // We'll therefore see any updates done to the myWait object by the puller thread if it has finished pulling.
+        if (myWait.latch.getCount() != 0) {
+          throwPullInProgressException(pushPullData.getSharedStoreName(), "still in progress and didn't complete while we waited");
+        } else {
+          // Pull has completed. Success or failure?
+          if (myWait.exception != null) {
+            // We wrap in a SolrException the original exception from the puller thread.
+            // (even if it's already a SolrException to make understanding what happens at runtime easier)
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Puller thread failed to pull", myWait.exception);
+          }
 
-    /**
-     * Called by the puller thread once pull has completed (successfully or not), to notify all waiter threads of the issue
-     * of the pull and to clean up this core in bookkeeping related to in-progress pulls({@link #pullEnded(String)}).
-     * Also serves the purpose of being a memory barrier so that the waiting threads can check their SyncOnPullWait instances
-     * for updates.
-     */
-    private void notifyEndOfPull(String sharedStoreName, Exception e) {
-        final Collection<SyncOnPullWait> collectionOfWaiters = pullEnded(sharedStoreName);
-        if (collectionOfWaiters != null) {
-            for (SyncOnPullWait w : collectionOfWaiters) {
-                // Need to set the exception before counting down on the latch because of the memory barrier effect of countDown()
-                w.exception = e;
-                w.latch.countDown();
-            }
+          // The pull has finished during our wait.
         }
-    }
-
-    /**
-     * Cleans up the core from bookkeeping related to in-progress pulls and returns the collection of waiters for that core.
-     * Collection of returned waiters could be null as well.
-     */
-    private Collection<SyncOnPullWait> pullEnded(String sharedStoreName) {
-        final Collection<SyncOnPullWait> collectionOfWaiters;
+      } finally {
+        // if wait is ended because of notifyEndOfPull() then it would have taken care of this state already and this essentially will be a no op.
+        // But if it has ended because of some other reason e.g. InterruptedException, then we will remove this thread from list of waiters.
+        //
+        // pullFinished notification may come later but that should be ok since it will only see waiters that are still waiting or none.
         synchronized (coreSyncsInFlight) {
-            // Note that threads waiting for the pull to finish have references on their individual SyncOnPullWait instances,
-            // so removing the entry from coreSyncsInFlight before the waiter threads are done waiting is ok.
-            collectionOfWaiters = coreSyncsInFlight.remove(sharedStoreName);
-            if(collectionOfWaiters != null) {
-                total_waiting_threads -= collectionOfWaiters.size();
+          Collection<SyncOnPullWait> collectionOfWaiters = coreSyncsInFlight.get(pushPullData.getSharedStoreName());
+          if (collectionOfWaiters != null) {
+            if (collectionOfWaiters.remove(myWait)){
+              total_waiting_threads--;
             }
+          }
         }
-        return collectionOfWaiters;
+      }
+    } else {
+      // Pull is in progress and we're not waiting for it to finish.
+      throwPullInProgressException(pushPullData.getSharedStoreName(), "as it is already in progress and enough threads waiting");
+    }
+  }
+
+  /**
+   * This is called whenever core from {@link CorePullTracker} finish its async pull(successfully or unsuccessfully)
+   * We use this to notify all waiting threads for a core that their wait has ended (if there are some waiting).
+   */
+  public void finishedPull(String sharedStoreName, CoreSyncStatus status, BlobCoreMetadata blobMetadata, String message) {
+    Exception pullException = null;
+    final boolean isPullSuccessful = (status.isSuccess() ||
+        // Following statuses are not considered success in strictest definition of pull but for BlobSyncer
+        // they are not error either and we would let the normal query flow do its job (likely return missing core exception)
+        status == CoreSyncStatus.BLOB_MISSING ||
+        status == CoreSyncStatus.BLOB_DELETED_FOR_PULL ||
+        // TODO:  likely changes needed here for W-5388477 Blob store corruption repair
+        status == CoreSyncStatus.BLOB_CORRUPT);
+    if (!isPullSuccessful) {
+      pullException = new SolrException(SolrException.ErrorCode.SERVER_ERROR, message);
+    }
+    notifyEndOfPull(sharedStoreName, pullException);
+  }
+
+  private void throwPullInProgressException(String corename, String msgSuffix) throws PullInProgressException {
+    String msg = SKIPPING_PULLING_CORE + " " + corename + " from blob " + msgSuffix;
+    log.info(msg);
+    // Note that longer term, this is the place where we could decide that if the async
+    // pull was enqueued too long ago and nothing happened, then there's an issue worth gacking/alerting for.
+    throw new PullInProgressException(msg);
+  }
+
+  /**
+   * Called by the puller thread once pull has completed (successfully or not), to notify all waiter threads of the issue
+   * of the pull and to clean up this core in bookkeeping related to in-progress pulls({@link #pullEnded(String)}).
+   * Also serves the purpose of being a memory barrier so that the waiting threads can check their SyncOnPullWait instances
+   * for updates.
+   */
+  private void notifyEndOfPull(String sharedStoreName, Exception e) {
+    final Collection<SyncOnPullWait> collectionOfWaiters = pullEnded(sharedStoreName);
+    if (collectionOfWaiters != null) {
+      for (SyncOnPullWait w : collectionOfWaiters) {
+        // Need to set the exception before counting down on the latch because of the memory barrier effect of countDown()
+        w.exception = e;
+        w.latch.countDown();
+      }
+    }
+  }
+
+  /**
+   * Cleans up the core from bookkeeping related to in-progress pulls and returns the collection of waiters for that core.
+   * Collection of returned waiters could be null as well.
+   */
+  private Collection<SyncOnPullWait> pullEnded(String sharedStoreName) {
+    final Collection<SyncOnPullWait> collectionOfWaiters;
+    synchronized (coreSyncsInFlight) {
+      // Note that threads waiting for the pull to finish have references on their individual SyncOnPullWait instances,
+      // so removing the entry from coreSyncsInFlight before the waiter threads are done waiting is ok.
+      collectionOfWaiters = coreSyncsInFlight.remove(sharedStoreName);
+      if (collectionOfWaiters != null) {
+        total_waiting_threads -= collectionOfWaiters.size();
+      }
     }
+    return collectionOfWaiters;
+  }
 }
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
index bd3562b..88d76ad 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
@@ -34,227 +34,227 @@ import com.google.common.collect.ImmutableSet;
  * and by implementing it separately we can add additional metadata to it as needed.
  */
 public class ServerSideMetadata {
-    
-    /**
-     * Files composing the core. They are are referenced from the core's current commit point's segments_N file
-     * which is ALSO included in this collection.
-     */
-    private final ImmutableCollection<CoreFileData> latestCommitFiles;
-
-    /**
-     * Index files related to current and previous commit points(if any).
-     * These files do not matter when pushing contents to blob but they do matter if blob content being pulled conflicts with them.
-     */
-    private final ImmutableCollection<CoreFileData> allCommitsFiles;
-
-    /**
-     * Hash of the directory content used to make sure the content doesn't change as we proceed to pull new files from Blob
-     * (if we need to pull new files from Blob)
-     */
-    private final 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 blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
-     * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
-     */
-    private final long generation;
-    private final SolrCore core;
-    private final String coreName;
-    private final CoreContainer container;
-
-    /**
-     * Given a core name, builds the local metadata
-     * 
-     * 
-     * @throws Exception if core corresponding to <code>coreName</code> can't be found.
-     */
-    public ServerSideMetadata(String coreName, CoreContainer container) throws Exception {
-        this.coreName = coreName;
-        this.container = container;
-        this.core = container.getCore(coreName);
-
-        if (core == null) {
-            throw new Exception("Can't find core " + coreName);
-        }
-
-        try {
-            IndexCommit latestCommit = core.getDeletionPolicy().getLatestCommit();
-            if (latestCommit == null) {
-                throw new BlobException("Core " + coreName + " has no available commit point");
-            }
-
-            generation = latestCommit.getGeneration();
-
-            // Work around possible bug returning same file multiple times by using a set here
-            // See org.apache.solr.handler.ReplicationHandler.getFileList()
-            ImmutableCollection.Builder<CoreFileData> latestCommitBuilder = new ImmutableSet.Builder<>();
-            ImmutableCollection.Builder<CoreFileData> allCommitsBuilder;
-
-            Directory coreDir = core.getDirectoryFactory().get(core.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
-            try {
-                // Capture now the hash and verify again if we need to pull content from the Blob store into this 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.
-                directoryHash = getSolrDirectoryHash(coreDir);
-
-                buildCommitFiles(coreDir, latestCommit, latestCommitBuilder);
-
-                // A note on listCommits says that it does not guarantee consistent results if a commit is in progress.
-                // But in blob context we serialize commits and pulls by proper locking therefore we should be good here.
-                List<IndexCommit> allCommits = DirectoryReader.listCommits(coreDir);
-
-                // optimization:  normally we would only be dealing with one commit point. In that case just reuse latest commit files builder.
-                if (allCommits.size() > 1 ){
-                    allCommitsBuilder = new ImmutableSet.Builder<>();
-                    for (IndexCommit commit: allCommits) {
-                        buildCommitFiles(coreDir, commit, allCommitsBuilder);
-                    }
-                } else {
-                    // we should always have a commit point as verified in the beginning of this method.
-                    assert allCommits.size() == 1 && allCommits.get(0).equals(latestCommit);
-                    allCommitsBuilder = latestCommitBuilder;
-                }
-            } finally {
-                core.getDirectoryFactory().release(coreDir);
-            }
-            latestCommitFiles = latestCommitBuilder.build();
-            allCommitsFiles = allCommitsBuilder.build();
-        } finally {
-            core.close();
-        }
+  
+  /**
+   * Files composing the core. They are are referenced from the core's current commit point's segments_N file
+   * which is ALSO included in this collection.
+   */
+  private final ImmutableCollection<CoreFileData> latestCommitFiles;
+
+  /**
+   * Index files related to current and previous commit points(if any).
+   * These files do not matter when pushing contents to blob but they do matter if blob content being pulled conflicts with them.
+   */
+  private final ImmutableCollection<CoreFileData> allCommitsFiles;
+
+  /**
+   * Hash of the directory content used to make sure the content doesn't change as we proceed to pull new files from Blob
+   * (if we need to pull new files from Blob)
+   */
+  private final 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 blob. In that scenario we would switch index to a new directory when pulling contents from blob. 
+   * Because in the presence of higher generation number locally, blob contents cannot establish their legitimacy.
+   */
+  private final long generation;
+  private final SolrCore core;
+  private final String coreName;
+  private final CoreContainer container;
+
+  /**
+   * Given a core name, builds the local metadata
+   * 
+   * 
+   * @throws Exception if core corresponding to <code>coreName</code> can't be found.
+   */
+  public ServerSideMetadata(String coreName, CoreContainer container) throws Exception {
+    this.coreName = coreName;
+    this.container = container;
+    this.core = container.getCore(coreName);
+
+    if (core == null) {
+      throw new Exception("Can't find core " + coreName);
     }
 
-    private void buildCommitFiles(Directory coreDir, IndexCommit commit, ImmutableCollection.Builder<CoreFileData> builder) throws IOException {
-        for (String fileName : commit.getFileNames()) {
-            // Note we add here all segment related files as well as the commit point's segments_N file
-            // Note commit points do not contain lock (write.lock) files.
-            try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
-                long length = indexInput.length();
-                long checksum = CodecUtil.retrieveChecksum(indexInput);
-                builder.add(new CoreFileData(fileName, length, checksum));
-            }
+    try {
+      IndexCommit latestCommit = core.getDeletionPolicy().getLatestCommit();
+      if (latestCommit == null) {
+        throw new BlobException("Core " + coreName + " has no available commit point");
+      }
+
+      generation = latestCommit.getGeneration();
+
+      // Work around possible bug returning same file multiple times by using a set here
+      // See org.apache.solr.handler.ReplicationHandler.getFileList()
+      ImmutableCollection.Builder<CoreFileData> latestCommitBuilder = new ImmutableSet.Builder<>();
+      ImmutableCollection.Builder<CoreFileData> allCommitsBuilder;
+
+      Directory coreDir = core.getDirectoryFactory().get(core.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+      try {
+        // Capture now the hash and verify again if we need to pull content from the Blob store into this 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.
+        directoryHash = getSolrDirectoryHash(coreDir);
+
+        buildCommitFiles(coreDir, latestCommit, latestCommitBuilder);
+
+        // A note on listCommits says that it does not guarantee consistent results if a commit is in progress.
+        // But in blob context we serialize commits and pulls by proper locking therefore we should be good here.
+        List<IndexCommit> allCommits = DirectoryReader.listCommits(coreDir);
+
+        // optimization:  normally we would only be dealing with one commit point. In that case just reuse latest commit files builder.
+        if (allCommits.size() > 1 ){
+          allCommitsBuilder = new ImmutableSet.Builder<>();
+          for (IndexCommit commit: allCommits) {
+            buildCommitFiles(coreDir, commit, allCommitsBuilder);
+          }
+        } else {
+          // we should always have a commit point as verified in the beginning of this method.
+          assert allCommits.size() == 1 && allCommits.get(0).equals(latestCommit);
+          allCommitsBuilder = latestCommitBuilder;
         }
+      } finally {
+        core.getDirectoryFactory().release(coreDir);
+      }
+      latestCommitFiles = latestCommitBuilder.build();
+      allCommitsFiles = allCommitsBuilder.build();
+    } finally {
+      core.close();
     }
-
-    public String getCoreName() {
-        return this.coreName;
-    }
-
-    public CoreContainer getCoreContainer() {
-        return this.container;
+  }
+
+  private void buildCommitFiles(Directory coreDir, IndexCommit commit, ImmutableCollection.Builder<CoreFileData> builder) throws IOException {
+    for (String fileName : commit.getFileNames()) {
+      // Note we add here all segment related files as well as the commit point's segments_N file
+      // Note commit points do not contain lock (write.lock) files.
+      try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
+        long length = indexInput.length();
+        long checksum = CodecUtil.retrieveChecksum(indexInput);
+        builder.add(new CoreFileData(fileName, length, checksum));
+      }
     }
-
-    public long getGeneration() {
-        return this.generation;
+  }
+
+  public String getCoreName() {
+      return this.coreName;
+  }
+
+  public CoreContainer getCoreContainer() {
+    return this.container;
+  }
+
+  public long getGeneration() {
+    return this.generation;
+  }
+
+  public String getDirectoryHash() {
+    return this.directoryHash;
+  }
+
+  public ImmutableCollection<CoreFileData> getLatestCommitFiles(){
+    return this.latestCommitFiles;
+  }
+
+  public ImmutableCollection<CoreFileData> getAllCommitsFiles() {
+    return this.allCommitsFiles;
+  }
+
+  /**
+   * Returns <code>true</code> 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.<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...
+   */
+  public boolean isSameDirectoryContent(Directory coreDir) throws NoSuchAlgorithmException, IOException {
+    return directoryHash.equals(getSolrDirectoryHash(coreDir));
+  }
+
+  /**
+   * 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) throws NoSuchAlgorithmException, IOException {
+    MessageDigest digest = MessageDigest.getInstance("sha1"); // not sure MD5 is available in Solr jars
+
+    String[] filesNames = coreDir.listAll();
+    // 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());
+        try {
+          digest.update(Long.toString(coreDir.fileLength(fileName)).getBytes());
+        } catch (FileNotFoundException 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.
+          digest.update(Long.toString(-42).getBytes());
+        }
+      }
     }
 
-    public String getDirectoryHash() {
-        return this.directoryHash;
+    final String hash = new String(Hex.encodeHex(digest.digest()));
+    return hash;
+  }
+
+  @Override
+  public String toString() {
+    return "collectionName=" + core.getCoreDescriptor().getCollectionName() +
+      " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() +
+      " coreName=" + core.getName() +
+      " generation=" + generation;
+  }
+
+  /**
+   * Information we capture per local core file (segments_N file *included*)
+   */
+  public static class CoreFileData {
+    /** Local file name, no path */
+    private final String fileName;
+    /** Size in bytes */
+    private final long fileSize;
+    private final long checksum;
+
+    CoreFileData(String fileName, long fileSize, long checksum) {
+      this.fileName = fileName;
+      this.fileSize = fileSize;
+      this.checksum = checksum;
     }
 
-    public ImmutableCollection<CoreFileData> getLatestCommitFiles(){
-        return this.latestCommitFiles;
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) return true;
+      if (o == null || getClass() != o.getClass()) return false;
+  
+      CoreFileData other = (CoreFileData) o;
+  
+      return Objects.equals(fileName, other.fileName) &&
+        Objects.equals(fileSize, other.fileSize) &&
+        Objects.equals(checksum, other.checksum);
     }
 
-    public ImmutableCollection<CoreFileData> getAllCommitsFiles() {
-        return this.allCommitsFiles;
+    public String getFileName() {
+      return fileName;
     }
 
-    /**
-     * Returns <code>true</code> 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.<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...
-     */
-    public boolean isSameDirectoryContent(Directory coreDir) throws NoSuchAlgorithmException, IOException {
-        return directoryHash.equals(getSolrDirectoryHash(coreDir));
+    public long getFileSize() {
+      return fileSize;
     }
 
-    /**
-     * 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) throws NoSuchAlgorithmException, IOException {
-        MessageDigest digest = MessageDigest.getInstance("sha1"); // not sure MD5 is available in Solr jars
-
-        String[] filesNames = coreDir.listAll();
-        // 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());
-                try {
-                    digest.update(Long.toString(coreDir.fileLength(fileName)).getBytes());
-                } catch (FileNotFoundException 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.
-                    digest.update(Long.toString(-42).getBytes());
-                }
-            }
-        }
-
-        final String hash = new String(Hex.encodeHex(digest.digest()));
-        return hash;
+    public long getChecksum() {
+      return checksum;
     }
 
     @Override
-    public String toString() {
-        return "collectionName=" + core.getCoreDescriptor().getCollectionName() +
-            " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() +
-            " coreName=" + core.getName() +
-            " generation=" + generation;
-    }
-
-    /**
-     * Information we capture per local core file (segments_N file *included*)
-     */
-    public static class CoreFileData {
-        /** Local file name, no path */
-        private final String fileName;
-        /** Size in bytes */
-        private final long fileSize;
-        private final long checksum;
-
-        CoreFileData(String fileName, long fileSize, long checksum) {
-            this.fileName = fileName;
-            this.fileSize = fileSize;
-            this.checksum = checksum;
-        }
-
-        @Override
-        public boolean equals(Object o) {
-            if (this == o) return true;
-            if (o == null || getClass() != o.getClass()) return false;
-
-            CoreFileData other = (CoreFileData) o;
-
-            return Objects.equals(fileName, other.fileName) &&
-                Objects.equals(fileSize, other.fileSize) &&
-                Objects.equals(checksum, other.checksum);
-        }
-
-        public String getFileName() {
-            return fileName;
-        }
-
-        public long getFileSize() {
-            return fileSize;
-        }
-
-        public long getChecksum() {
-            return checksum;
-        }
-
-        @Override
-        public int hashCode() {
-            return Objects.hash(fileName, fileSize, checksum);
-        }
+    public int hashCode() {
+      return Objects.hash(fileName, fileSize, checksum);
     }
+  }
 
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
index 2bc8054..b01ee94 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
@@ -21,7 +21,7 @@ import org.slf4j.LoggerFactory;
  */
 public class SharedStoreResolutionUtil {
 
-  private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static String SEGMENTS_N_PREFIX = "segments_";
 
@@ -153,7 +153,7 @@ public class SharedStoreResolutionUtil {
                   " localSize=%s blobSize=%s localChecksum=%s blobCheckSum=%s",
               distant.getSharedBlobName(), local.getCoreName(), bf.getSolrFileName(), bf.getBlobName(),
               cf.getFileSize(), bf.getFileSize(), cf.getChecksum(), bf.getChecksum());
-          logger.info(message);
+          log.info(message);
           localConflictingWithBlob = true;
         }
       }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleteManager.java b/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleteManager.java
index 3212cdf..a50139a 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleteManager.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/BlobDeleteManager.java
@@ -1,5 +1,6 @@
 package org.apache.solr.store.blob.process;
 
+import java.lang.invoke.MethodHandles;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.LinkedBlockingDeque;
@@ -11,12 +12,16 @@ import java.util.concurrent.TimeUnit;
 import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.metadata.CorePushPull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Manager of blobs (files) to delete, putting them in a queue (if space left on the queue) then consumed and processed
  * by {@link BlobDeleterTask}
  */
 public class BlobDeleteManager {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   
   /**
    * Limit to the number of blob files to delete accepted on the delete queue (and lost in case of server crash). When
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
index b92693e..0bae5f6 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
@@ -34,109 +34,108 @@ import org.slf4j.LoggerFactory;
  */
 public abstract class CoreSyncFeeder implements Runnable, Closeable {
 
-    private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
-    protected final CoreContainer cores;
-
-    /**
-     * Maximum number of elements in the queue, NOT counting re-inserts after failures. Total queue size might therefore
-     * exceed this value by the number of syncThreads which is around 5 or 10...
-     * <p>
-     * Note that this queue sits behind other tracking queues (see {@link CoreUpdateTracker} and
-     * {@link CorePullTracker}). The other queue has to be large, this one does not.
-     */
-    protected static final int ALMOST_MAX_WORKER_QUEUE_SIZE = 2000;
-
-    /**
-     * When a transient error occurs, the number of attempts to sync with Blob before giving up. Attempts are spaced by
-     * at least 10 seconds (see {@link CorePullTask#MIN_RETRY_DELAY_MS}) which means we'll retry for at least 90 seconds 
-     * before giving up. This is something to adjust as the implementation of the delay between retries is cleaned up, 
-     * see {@link CorePullTask}.
-     */
-    protected static final int MAX_ATTEMPTS = 10;
-
-    private final int numSyncThreads;
-
-    /**
-     * Used to interrupt close()
-     */
-    private volatile Thread executionThread;
-    private volatile boolean closed = false;
-
-    protected CoreSyncFeeder(CoreContainer cores, int numSyncThreads) {
-        this.numSyncThreads = numSyncThreads;
-        this.cores = cores;
-    }
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  protected final CoreContainer cores;
+
+  /**
+   * Maximum number of elements in the queue, NOT counting re-inserts after failures. Total queue size might therefore
+   * exceed this value by the number of syncThreads which is around 5 or 10...
+   * <p>
+   * Note that this queue sits behind other tracking queues (see {@link CoreUpdateTracker} and
+   * {@link CorePullTracker}). The other queue has to be large, this one does not.
+   */
+  protected static final int ALMOST_MAX_WORKER_QUEUE_SIZE = 2000;
+
+  /**
+   * When a transient error occurs, the number of attempts to sync with Blob before giving up. Attempts are spaced by
+   * at least 10 seconds (see {@link CorePullTask#MIN_RETRY_DELAY_MS}) which means we'll retry for at least 90 seconds
+   * before giving up. This is something to adjust as the implementation of the delay between retries is cleaned up,
+   * see {@link CorePullTask}.
+   */
+  protected static final int MAX_ATTEMPTS = 10;
+
+  private final int numSyncThreads;
+
+  /**
+   * Used to interrupt close()
+   */
+  private volatile Thread executionThread;
+  private volatile boolean closed = false;
+
+  protected CoreSyncFeeder(CoreContainer cores, int numSyncThreads) {
+    this.numSyncThreads = numSyncThreads;
+    this.cores = cores;
+  }
+
+  @Override
+  public void run() {
+    // Record where we run so we can be interrupted from close(). If we get interrupted before this point we will
+    // not
+    // close anything and not log anything, but that's ok we haven't started anything either.
+    this.executionThread = Thread.currentThread();
+    // If close() executed before runningFeeder was set, we need to exit
+    if (closed) { return; }
+    try {
+      // We'll be submitting tasks to a queue from which threads are going to pick them up and execute them.
+      // PeerSyncer uses a ThreadPoolExecutor for this. Here instead we explicitly start the threads to
+      // simplify our life (and the code), because by not using an executor we're not forced to use a
+      // BlockingQueue<Runnable> as the work queue and instead use a DeduplicatingList.
+      NamedThreadFactory threadFactory = new NamedThreadFactory(getMonsterThreadName());
+      // TODO set daemon?
+      Set<Thread> syncerThreads = new HashSet<>();
+      for (int i = 0; i < numSyncThreads; i++) {
+        Thread t = threadFactory.newThread(this.getSyncer());
+        syncerThreads.add(t);
+      }
 
-    @Override
-    public void run() {
-        // Record where we run so we can be interrupted from close(). If we get interrupted before this point we will
-        // not
-        // close anything and not log anything, but that's ok we haven't started anything either.
-        this.executionThread = Thread.currentThread();
-        // If close() executed before runningFeeder was set, we need to exit
-        if (closed) { return; }
-        try {
-            // We'll be submitting tasks to a queue from which threads are going to pick them up and execute them.
-            // PeerSyncer uses a ThreadPoolExecutor for this. Here instead we explicitly start the threads to
-            // simplify our life (and the code), because by not using an executor we're not forced to use a
-            // BlockingQueue<Runnable> as the work queue and instead use a DeduplicatingList.
-            NamedThreadFactory threadFactory = new NamedThreadFactory(getMonsterThreadName());
-            // TODO set daemon?
-            Set<Thread> syncerThreads = new HashSet<>();
-            for (int i = 0; i < numSyncThreads; i++) {
-                Thread t = threadFactory.newThread(this.getSyncer());
-                syncerThreads.add(t);
-            }
-
-            try {
-                // Starting the threads after having created them so that we can interrupt all threads in the finally
-                for (Thread t : syncerThreads) {
-                    t.start();
-                }
-
-                feedTheMonsters();
-            } catch (Throwable e) {
-                if (!closed) {
-                  log.error("CoreSyncFeeder thread encountered an error and is exiting "
-                      + "while close() was not called. " + ExceptionUtils.getStackTrace(e));
-                }
-            } finally {
-                // If we stop, have our syncer "thread pool" stop as well since there's not much they can do anyway
-                // then...
-                for (Thread t : syncerThreads) {
-                    t.interrupt();
-                }
-            }
-
-        } finally {
-            this.executionThread = null;
+      try {
+        // Starting the threads after having created them so that we can interrupt all threads in the finally
+        for (Thread t : syncerThreads) {
+          t.start();
         }
-    }
-
-    boolean shouldContinueRunning() {
-      return !this.cores.isShutDown();
-    }
 
-    @Override
-    public void close() {
-      if (!closed) {
-        closed = true;
-        Thread thread = this.executionThread;
-        if (thread != null) {
-          this.executionThread = null; // race to set to null but ok to try to interrupt twice
-          log.info(String.format("Closing CoreSyncFeeder; interrupting execution thread %s.", thread.getName()));
-          thread.interrupt();
-        } else {
-          log.warn("Closing CoreSyncFeeder before any syncer thread was started. Weird.");
+        feedTheMonsters();
+      } catch (Throwable e) {
+        if (!closed) {
+          log.error("CoreSyncFeeder thread encountered an error and is exiting "
+              + "while close() was not called. " + ExceptionUtils.getStackTrace(e));
+        }
+      } finally {
+        // If we stop, have our syncer "thread pool" stop as well since there's not much they can do anyway
+        // then...
+        for (Thread t : syncerThreads) {
+          t.interrupt();
         }
       }
+    } finally {
+      this.executionThread = null;
+    }
+  }
+
+  boolean shouldContinueRunning() {
+    return !this.cores.isShutDown();
+  }
+
+  @Override
+  public void close() {
+    if (!closed) {
+      closed = true;
+      Thread thread = this.executionThread;
+      if (thread != null) {
+        this.executionThread = null; // race to set to null but ok to try to interrupt twice
+        log.info(String.format("Closing CoreSyncFeeder; interrupting execution thread %s.", thread.getName()));
+        thread.interrupt();
+      } else {
+        log.warn("Closing CoreSyncFeeder before any syncer thread was started. Weird.");
+      }
     }
+  }
 
-    abstract String getMonsterThreadName();
+  abstract String getMonsterThreadName();
 
-    abstract void feedTheMonsters() throws InterruptedException;
+  abstract void feedTheMonsters() throws InterruptedException;
 
-    abstract Runnable getSyncer();
+  abstract Runnable getSyncer();
 
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
index 48d065e..4a428c2 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
@@ -1,6 +1,4 @@
 package org.apache.solr.store.blob.util;
-import static org.junit.Assert.assertTrue;
-
 import java.lang.invoke.MethodHandles;
 import java.util.Map;
 import java.util.UUID;
@@ -57,7 +55,7 @@ public class BlobStoreUtils {
    * @throws SolrException if the local core was not successfully sync'd.
    */
   public static void syncLocalCoreWithSharedStore(String collectionName, String coreName, String shardName, CoreContainer coreContainer) throws SolrException {
-    assertTrue(coreContainer.isZooKeeperAware());
+    assert coreContainer.isZooKeeperAware();
 
     ZkController zkController = coreContainer.getZkController();
     SharedShardMetadataController sharedMetadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController();
diff --git a/solr/core/src/java/org/apache/solr/store/blob/util/DeduplicatingList.java b/solr/core/src/java/org/apache/solr/store/blob/util/DeduplicatingList.java
index df46227..ed03fa1 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/util/DeduplicatingList.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/util/DeduplicatingList.java
@@ -24,161 +24,161 @@ import java.util.Map;
  */
 public class DeduplicatingList<K, V extends DeduplicatingList.Deduplicatable<K>> {
 
-    /**
-     * We need the "V" parameter of {@link DeduplicatingList} to be able to provide us the "K" parameter used as key.
-     */
-    public interface Deduplicatable<DK> {
-        DK getDedupeKey();
+  /**
+   * We need the "V" parameter of {@link DeduplicatingList} to be able to provide us the "K" parameter used as key.
+   */
+  public interface Deduplicatable<DK> {
+    DK getDedupeKey();
+  }
+
+  /**
+   * Given v1 and v2 (corresponding to type V in {@link DeduplicatingList}) to be stored in the list and
+   * that have the same key (as per {@link Deduplicatable#getDedupeKey()}, parameter MK here), returns a merged value that
+   * has the same key and that can replace the two entries in the list (based on domain specific knowledge that this is ok).<p>
+   * The behaviour of this method is undefined and totally unpredictable if v1 and v2 do not have the same key. Be warned :)
+   */
+  public interface Merger<MK, MV extends DeduplicatingList.Deduplicatable<MK>> {
+    MV merge(MV v1, MV v2);
+  }
+
+  /**
+   * Building a linked list from scratch here to allow access to specific list entries (tracked in another
+   * data structure) in constant time.
+   * Instances must only be accessed with {@link #lock} held, as they are mutable.
+   */
+  static private class TaskListNode<TV> {
+    TaskListNode next;
+    TV elem;
+
+    TaskListNode(TaskListNode next, TV elem) {
+      this.next = next;
+      this.elem = elem;
     }
-
-    /**
-     * Given v1 and v2 (corresponding to type V in {@link DeduplicatingList}) to be stored in the list and
-     * that have the same key (as per {@link Deduplicatable#getDedupeKey()}, parameter MK here), returns a merged value that
-     * has the same key and that can replace the two entries in the list (based on domain specific knowledge that this is ok).<p>
-     * The behaviour of this method is undefined and totally unpredictable if v1 and v2 do not have the same key. Be warned :)
-     */
-    public interface Merger<MK, MV extends DeduplicatingList.Deduplicatable<MK>> {
-        MV merge(MV v1, MV v2);
-    }
-
-    /**
-     * Building a linked list from scratch here to allow access to specific list entries (tracked in another
-     * data structure) in constant time.
-     * Instances must only be accessed with {@link #lock} held, as they are mutable.
-     */
-    static private class TaskListNode<TV> {
-        TaskListNode next;
-        TV elem;
-
-        TaskListNode(TaskListNode next, TV elem) {
-            this.next = next;
-            this.elem = elem;
+  }
+
+  /**
+   * Guards access to all the rest of the data here.
+   */
+  private final Object lock = new Object();
+
+  private final int almostMaxListSize;
+  private final DeduplicatingList.Merger<K, V> merger;
+
+  private TaskListNode<V> head = null;
+  private TaskListNode<V> tail = null;
+
+  private final Map<K, TaskListNode<V>> keyToListNode = new HashMap<>();
+
+  private int size = 0;
+
+  /**
+   * Builds an empty list of a given maximum size.
+   * @param almostMaxListSize The maximum number of "new" elements accepted in the list, excluding element reenqueues (of which
+   *                     there are expected to be very few and related to number of processing threads in a thread pool for
+   *                     example). When that number is reached, {@link #addDeduplicated} blocks until the List size
+   *                     is reduced enough.
+   */
+  public DeduplicatingList(int almostMaxListSize, DeduplicatingList.Merger<K, V> merger) {
+    this.almostMaxListSize = almostMaxListSize;
+    this.merger = merger;
+  }
+
+  /**
+   * Adds an entry to the tail of the list unless there's already an existing entry in the list the added entry can be merged with.
+   * If that's the case, the added entry is merged instead of being added.
+   * @param isReenqueue <ul><li>when <code>true</code>, the put is allowed to make the list exceed its {@link #almostMaxListSize}. In practice
+   *                    we'll exceed at most by a few units.</li>
+   *                    <li>when <code>false</code>, if the list is too full, the call blocks waiting until the insert can be done</li></ul>
+   */
+  public void addDeduplicated(final V value, boolean isReenqueue) throws InterruptedException {
+    synchronized (lock) {
+      TaskListNode<V> existingEntry = keyToListNode.get(value.getDedupeKey());
+
+      if (existingEntry != null) {
+        // We already have an entry for that core, merge.
+        existingEntry.elem = merger.merge(existingEntry.elem, value);
+
+        assert existingEntry.elem != null;
+      } else {
+        // Wait (if needed) until the insert can be done
+        while (!isReenqueue && size >= almostMaxListSize) {
+          lock.wait();
         }
-    }
-
-    /**
-     * Guards access to all the rest of the data here.
-     */
-    private final Object lock = new Object();
-
-    private final int almostMaxListSize;
-    private final DeduplicatingList.Merger<K, V> merger;
 
-    private TaskListNode<V> head = null;
-    private TaskListNode<V> tail = null;
-
-    private final Map<K, TaskListNode<V>> keyToListNode = new HashMap<>();
-
-    private int size = 0;
-
-    /**
-     * Builds an empty list of a given maximum size.
-     * @param almostMaxListSize The maximum number of "new" elements accepted in the list, excluding element reenqueues (of which
-     *                     there are expected to be very few and related to number of processing threads in a thread pool for
-     *                     example). When that number is reached, {@link #addDeduplicated} blocks until the List size
-     *                     is reduced enough.
-     */
-    public DeduplicatingList(int almostMaxListSize, DeduplicatingList.Merger<K, V> merger) {
-        this.almostMaxListSize = almostMaxListSize;
-        this.merger = merger;
+        addToTail(value);
+      }
     }
-
-    /**
-     * Adds an entry to the tail of the list unless there's already an existing entry in the list the added entry can be merged with.
-     * If that's the case, the added entry is merged instead of being added.
-     * @param isReenqueue <ul><li>when <code>true</code>, the put is allowed to make the list exceed its {@link #almostMaxListSize}. In practice
-     *                    we'll exceed at most by a few units.</li>
-     *                    <li>when <code>false</code>, if the list is too full, the call blocks waiting until the insert can be done</li></ul>
-     */
-    public void addDeduplicated(final V value, boolean isReenqueue) throws InterruptedException {
-        synchronized (lock) {
-            TaskListNode<V> existingEntry = keyToListNode.get(value.getDedupeKey());
-
-            if (existingEntry != null) {
-                // We already have an entry for that core, merge.
-                existingEntry.elem = merger.merge(existingEntry.elem, value);
-
-                assert existingEntry.elem != null;
-            } else {
-                // Wait (if needed) until the insert can be done
-                while (!isReenqueue && size >= almostMaxListSize) {
-                    lock.wait();
-                }
-
-                addToTail(value);
-            }
-        }
+  }
+
+  /**
+   * @return the value sitting at the head of the list or blocks waiting until there's one to return if the list is empty.
+   */
+  public V removeFirst() throws InterruptedException {
+    synchronized (lock) {
+      while (size == 0) {
+        lock.wait();
+      }
+
+      return removeHead();
     }
-
-    /**
-     * @return the value sitting at the head of the list or blocks waiting until there's one to return if the list is empty.
-     */
-    public V removeFirst() throws InterruptedException {
-        synchronized (lock) {
-            while (size == 0) {
-                lock.wait();
-            }
-
-            return removeHead();
-        }
+  }
+
+  /**
+   * @return number of entries in the list
+   */
+  public int size() {
+    synchronized (lock) {
+      return size;
     }
-
-    /**
-     * @return number of entries in the list
-     */
-    public int size() {
-        synchronized (lock) {
-            return size;
-        }
+  }
+
+  /**
+   * The key for the value being added should not already be present in the list.
+   * The {@link #lock} must be held when calling this method.
+   */
+  private void addToTail(final V value) {
+    TaskListNode<V> newNode = new TaskListNode<>(null, value);
+
+    if (tail == null) {
+      // List is empty
+      assert head == null;
+      assert size == 0;
+      head = newNode;
+      // Wake up threads blocked in getTask(), if any
+      lock.notifyAll();
+    } else {
+      // List not empty (means nobody is blocked sleeping)
+      assert head != null;
+      tail.next = newNode;
     }
 
-    /**
-     * The key for the value being added should not already be present in the list.
-     * The {@link #lock} must be held when calling this method.
-     */
-    private void addToTail(final V value) {
-        TaskListNode<V> newNode = new TaskListNode<>(null, value);
-
-        if (tail == null) {
-            // List is empty
-            assert head == null;
-            assert size == 0;
-            head = newNode;
-            // Wake up threads blocked in getTask(), if any
-            lock.notifyAll();
-        } else {
-            // List not empty (means nobody is blocked sleeping)
-            assert head != null;
-            tail.next = newNode;
-        }
-
-        TaskListNode<V> old = keyToListNode.put(value.getDedupeKey(), newNode);
-        assert old == null;
-
-        tail = newNode;
-        size++;
+    TaskListNode<V> old = keyToListNode.put(value.getDedupeKey(), newNode);
+    assert old == null;
+
+    tail = newNode;
+    size++;
+  }
+
+  /**
+   * The {@link #lock} must be held while calling this method.
+   * This method can only be called if the list is not empty.
+   * @return the value at the head of the list (inserted before all other values into list).
+   */
+  private V removeHead() {
+    assert size > 0;
+    TaskListNode<V> oldHead = head;
+    head = oldHead.next;
+    if (head == null) {
+      // list now empty, we've removed the last element
+      assert size == 1;
+      assert tail == oldHead;
+
+      tail = null;
     }
+    size--;
+    TaskListNode<V> fromSet = keyToListNode.remove(oldHead.elem.getDedupeKey());
+    assert oldHead == fromSet;
 
-    /**
-     * The {@link #lock} must be held while calling this method.
-     * This method can only be called if the list is not empty.
-     * @return the value at the head of the list (inserted before all other values into list).
-     */
-    private V removeHead() {
-        assert size > 0;
-        TaskListNode<V> oldHead = head;
-        head = oldHead.next;
-        if (head == null) {
-            // list now empty, we've removed the last element
-            assert size == 1;
-            assert tail == oldHead;
-
-            tail = null;
-        }
-        size--;
-        TaskListNode<V> fromSet = keyToListNode.remove(oldHead.elem.getDedupeKey());
-        assert oldHead == fromSet;
-
-        return oldHead.elem;
-    }
+    return oldHead.elem;
+  }
 }
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 d3c1087..9b44ef4 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
@@ -18,7 +18,6 @@
 package org.apache.solr.update.processor;
 
 import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
-import static org.junit.Assert.assertEquals;
 
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
@@ -1096,7 +1095,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     String coreName = req.getCore().getName();
     String shardName = cloudDesc.getShardId();
     String collectionName = cloudDesc.getCollectionName();
-    assertEquals(replicaType, Replica.Type.SHARED);
+    assert Replica.Type.SHARED.equals(replicaType);
     // Peers and subShardLeaders should only forward the update request to leader replica,
     // hence not need to sync with the blob store at this point.
     if (!isLeader || isSubShardLeader) {
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
index 6121199..9ccbf39 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
@@ -54,6 +54,7 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
   
   @BeforeClass
   public static void setupTest() throws Exception {
+    assumeWorkingMockito();
     // set up the temp directory for a local blob store
     localBlobDir = createTempDir("tempDir");
     System.setProperty(LocalStorageClient.BLOB_STORE_LOCAL_FS_ROOT_DIR_PROPERTY, localBlobDir.resolve("LocalBlobStore/").toString());
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
index acc07ea..d145d7c 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
@@ -25,13 +25,13 @@ import java.util.Collections;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.BlobCoreMetadata.BlobFile;
 import org.apache.solr.store.blob.client.BlobCoreMetadataBuilder;
 import org.apache.solr.store.blob.metadata.ServerSideMetadata.CoreFileData;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetadataResolutionResult;
-import org.apache.solr.store.shared.SolrCloudSharedStoreTestCase;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -40,10 +40,10 @@ import com.google.common.collect.ImmutableSet;
 /**
  * Unit tests for {@link SharedStoreResolutionUtil}.
  */
-public class SharedStoreResolutionUtilTest extends SolrCloudSharedStoreTestCase {
-  
+public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
+
   @BeforeClass
-  public static void beforeClass() {
+  public static void setupTests() {
     assumeWorkingMockito();
   }
   
diff --git a/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java b/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
index d1d5803..3cb3c86 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
@@ -58,7 +58,8 @@ public class BlobStoreUtilsTest extends SolrCloudSharedStoreTestCase {
   private static CoreStorageClient storageClient;
   
   @BeforeClass
-  public static void setupTestClass() throws Exception {    
+  public static void setupTestClass() throws Exception {
+    assumeWorkingMockito();
     sharedStoreRootPath = createTempDir("tempDir");
     storageClient = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
     
diff --git a/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java b/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
index 041cd52..845e780 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
@@ -50,7 +50,8 @@ public class SharedShardMetadataControllerTest extends SolrCloudTestCase {
   static SolrCloudManager cloudManager; 
   
   @BeforeClass
-  public static void setupCluster() throws Exception {    
+  public static void setupCluster() throws Exception {
+    assumeWorkingMockito();
     configureCluster(1)
       .addConfig("conf", configset("cloud-minimal"))
       .configure();
diff --git a/solr/core/src/test/org/apache/solr/update/processor/DistributedZkUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/DistributedZkUpdateProcessorTest.java
index 0a5f9e6..254180f 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/DistributedZkUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/DistributedZkUpdateProcessorTest.java
@@ -57,7 +57,8 @@ public class DistributedZkUpdateProcessorTest extends SolrCloudSharedStoreTestCa
   private static CoreStorageClient storageClient;
   
   @BeforeClass
-  public static void setupTestClass() throws Exception {    
+  public static void setupTestClass() throws Exception {
+    assumeWorkingMockito();
     sharedStoreRootPath = createTempDir("tempDir");
     storageClient = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
     


[lucene-solr] 01/11: @W-6475448 Allow CorePullerFeeder to be provisioned with a callback in tests and… (#386)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c7ad68fd304e6846d35c4af0d7e5d575233a1c46
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Thu Sep 19 21:05:43 2019 -0400

    @W-6475448 Allow CorePullerFeeder to be provisioned with a callback in tests and… (#386)
    
    * Allow CorePullerFeeder to be provisioned with a callback in tests and allow testing async pulls
---
 .../solr/store/blob/process/BlobProcessUtil.java   | 18 +++++--
 .../solr/store/blob/process/CorePullTask.java      |  2 +-
 .../solr/store/blob/process/CorePullerFeeder.java  |  6 ++-
 .../solr/store/shared/SharedStoreManager.java      |  8 +++
 .../shared/SimpleSharedStoreEndToEndPullTest.java  | 58 +++++++++++++++++++---
 .../store/shared/SolrCloudSharedStoreTestCase.java | 19 +++++--
 6 files changed, 93 insertions(+), 18 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java b/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
index d090952..164818d 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
@@ -30,9 +30,18 @@ public class BlobProcessUtil {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
   private volatile CorePullerFeeder runningFeeder = null;
   
+  /*
+   * Start the Blob store async core pull machinery
+   */
   public BlobProcessUtil(CoreContainer coreContainer) {
-    // Start the Blob store sync core push and async core pull machinery
-    runningFeeder = initializeCorePullerFeeder(coreContainer);
+    this(coreContainer, new CorePullerFeeder(coreContainer));
+  }
+  
+  /*
+   * Start the Blob store async core pull machinery
+   */
+  public BlobProcessUtil(CoreContainer coreContainer, CorePullerFeeder cpf) {    
+    runningFeeder = initializeCorePullerFeeder(cpf);
   }
   
   /**
@@ -44,11 +53,10 @@ public class BlobProcessUtil {
   
   /**
    * Initializes the CorePullerFeeder and starts running thread
-   * @param cores CoreContainer
+   * @param cpf CorePullerFeeder
    * @return CorePullerFeeder 
    */
-  private CorePullerFeeder initializeCorePullerFeeder(CoreContainer cores) {
-    CorePullerFeeder cpf = new CorePullerFeeder(cores);
+  private CorePullerFeeder initializeCorePullerFeeder(CorePullerFeeder cpf) {
     Thread t = new Thread(cpf);
     t.setName("blobPullerFeeder-" + t.getName());
     t.start();
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index 2ba1f6c..e8af082 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -446,7 +446,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
      * @param message
      *            Human readable message explaining a failure, or <code>null</code> if no message available.
      */
-    void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status, String message)
+    public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status, String message)
         throws InterruptedException;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index 4cae45d..b950d67 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -57,6 +57,10 @@ public class CorePullerFeeder extends CoreSyncFeeder {
   String getMonsterThreadName() {
     return PULLER_THREAD_PREFIX;
   }
+  
+  protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
+    return callback;
+  }
 
   @Override
   void feedTheMonsters() throws InterruptedException {
@@ -69,7 +73,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       PullCoreInfo pci = tracker.getCoreToPull();
 
       // Add the core to the list consumed by the thread doing the actual work
-      CorePullTask pt = new CorePullTask(cores, pci, callback);
+      CorePullTask pt = new CorePullTask(cores, pci, getCorePullTaskCallback());
       pullTaskQueue.addDeduplicated(pt, /* isReenqueue */ false);
       syncsEnqueuedSinceLastLog++;
 
diff --git a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
index 530cbef..3dd525a 100644
--- a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
+++ b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
@@ -50,6 +50,14 @@ public class SharedStoreManager {
     this.blobStorageProvider = blobStorageProvider;
   }
   
+  @VisibleForTesting
+  public void initBlobProcessUtil(BlobProcessUtil processUtil) {
+    if (blobProcessUtil != null) {
+      blobProcessUtil.shutdown();
+    }
+    blobProcessUtil = processUtil;
+  }
+  
   /*
    * Initiates a SharedShardMetadataController if it doesn't exist and returns one 
    */
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
index 4e660f6..4a7921e 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
@@ -17,9 +17,14 @@
 package org.apache.solr.store.shared;
 
 import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
@@ -28,7 +33,13 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.CoreStorageClient;
+import org.apache.solr.store.blob.process.BlobProcessUtil;
+import org.apache.solr.store.blob.process.CorePullTask;
+import org.apache.solr.store.blob.process.CorePullTask.PullCoreCallback;
+import org.apache.solr.store.blob.process.CorePullerFeeder;
+import org.apache.solr.store.blob.process.CoreSyncStatus;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -64,11 +75,23 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
     setupCluster(2);
     CloudSolrClient cloudClient = cluster.getSolrClient();
     
-    // setup the test harness
+    // setup the test harnesses
+    Map<String, CountDownLatch> cdlMap = new HashMap<>();
+    
+    CountDownLatch latch1 = new CountDownLatch(1);
+    JettySolrRunner solrProcess1 = cluster.getJettySolrRunner(0);
     CoreStorageClient storageClient1 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
-    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient1), cluster.getJettySolrRunner(0));
+    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient1), solrProcess1);
+    configureTestBlobProcessForNode(solrProcess1, setupCallback(latch1));
+    
+    CountDownLatch latch2 = new CountDownLatch(1);
+    JettySolrRunner solrProcess2 = cluster.getJettySolrRunner(1);
     CoreStorageClient storageClient2 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
-    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient2), cluster.getJettySolrRunner(1));
+    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient2), solrProcess2);
+    configureTestBlobProcessForNode(solrProcess2, setupCallback(latch2));
+    
+    cdlMap.put(solrProcess1.getNodeName(), latch1);
+    cdlMap.put(solrProcess2.getNodeName(), latch2);
     
     String collectionName = "sharedCollection";
     int maxShardsPerNode = 1;
@@ -121,9 +144,9 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
       resp = followerDirectClient.query(params);
       assertEquals(0, resp.getResults().getNumFound());
       
-      // TODO super ugly and inappropriate but the pull shouldn't take long. At some point we'll
-      // make our end-to-end async testing nicer by supporting test listeners for the async tasks 
-      Thread.sleep(5000);
+      // wait until pull is finished
+      CountDownLatch latch = cdlMap.get(followerReplica.getNodeName());
+      assertTrue(latch.await(120, TimeUnit.SECONDS));
       
       // do another query to verify we've pulled everything
       resp = followerDirectClient.query(params);
@@ -140,4 +163,27 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
       core.close();
     }
   }
+  
+  private BlobProcessUtil configureTestBlobProcessForNode(JettySolrRunner runner, PullCoreCallback callback) {
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {
+      @Override
+      protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
+        return callback;
+      }
+    };
+    BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
+    setupTestBlobProcessUtilForNode(testUtil, runner);
+    return testUtil;
+  }
+  
+  private PullCoreCallback setupCallback(CountDownLatch latch) {
+    return new PullCoreCallback() {
+      @Override
+      public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+          String message) throws InterruptedException {
+        assertTrue(status.isSuccess());
+        latch.countDown();
+      }
+    };
+  }
 }
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java b/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
index 0e7749f..738ff3f 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
@@ -24,6 +24,7 @@ import org.apache.solr.cloud.MiniSolrCloudCluster;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.client.LocalStorageClient;
+import org.apache.solr.store.blob.process.BlobProcessUtil;
 import org.apache.solr.store.blob.provider.BlobStorageProvider;
 
 /**
@@ -69,11 +70,19 @@ public class SolrCloudSharedStoreTestCase extends SolrCloudTestCase {
   }
   
   /**
-   * adds the test harness to each solr process in the MiniSolrCluster
+   * Configures the Solr process with the given BlobStorageProvider
    */
-  protected static void setupTestSharedClientForNode(BlobStorageProvider providerTestHarness, JettySolrRunner solrRunner) {
+  protected static void setupTestSharedClientForNode(BlobStorageProvider testBlobStorageProvider, JettySolrRunner solrRunner) {
     SharedStoreManager manager = solrRunner.getCoreContainer().getSharedStoreManager();
-    manager.initBlobStorageProvider(providerTestHarness);
+    manager.initBlobStorageProvider(testBlobStorageProvider);
+  }
+  
+  /**
+   * Configures the Solr process with the given BlobProcessUtil
+   */
+  protected static void setupTestBlobProcessUtilForNode(BlobProcessUtil testBlobProcessUtil, JettySolrRunner solrRunner) {
+    SharedStoreManager manager = solrRunner.getCoreContainer().getSharedStoreManager();
+    manager.initBlobProcessUtil(testBlobProcessUtil);
   }
   
   /**
@@ -86,13 +95,13 @@ public class SolrCloudSharedStoreTestCase extends SolrCloudTestCase {
   }
   
   protected static BlobStorageProvider getBlobStorageProviderTestInstance(CoreStorageClient client) {
-    BlobStorageProvider providerTestHarness = new BlobStorageProvider() {
+    BlobStorageProvider testBlobStorageProvider = new BlobStorageProvider() {
       @Override
       public CoreStorageClient getClient() {
         return client;
       }
     };
-    return providerTestHarness;
+    return testBlobStorageProvider;
   }
 
 }


[lucene-solr] 09/11: @W-6587412 Concurrent indexing, pull and pushes (#403)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 581f468f9914ce2488201efbed42fd43dc44b481
Author: Bilal Waheed <mw...@salesforce.com>
AuthorDate: Mon Oct 21 17:34:39 2019 -0400

    @W-6587412 Concurrent indexing, pull and pushes (#403)
    
    * @W-6587412 Concurrent indexing, pull and pushes
    
    -Pulls are protected from each other, local indexing and pushes.
    -Local indexing and pushes are protected from pulls.
    -Pushes are protected from each other.
    -Implicit hard commit on each indexing batch.
    -Unlike before we only ensure existence of zookeeper metadata node once.
    -In steady state we only push to shared store and update zk. we do not read from either zk or shared store. we rely on cached values with soft guarantee of being in sync.
    -We take away soft guarantee if conditional update to zk fails. Next indexing request then sync with zk and shared store first.
    -Sub shard push is handled for happy path only. Non-happy path are not understood yet. If they happen they should be detected and an exception thrown.
    
    -Core pulls deduped on core name instead of share shard name. Since at the local level we are dealing with core not a shard.
    -CorePullTask does not need pullsInFlight anymore because of new pull write lock.
    -PushPullData is cleaned up of version/metadata info. The accompanying task merge logic is gone(not needed).
    -SharedShardMetadataController cache is removed(was not enough and not needed anymore).
    -SharedMetadataResolutionResult has a separate collection of files to be deleted. Files to be pulled cannot play that role(with conflict handling pull collection can be everything blob has to offer)
    
    -take snapshot of active segment files in one go, so that later on pushing time they are not removed underneath us. this is important to not fail indexing attempts unnecessarily.
    -Reserve a commit point before capturing it.
    -Added guards and comments around lock contention and fairness
    -Added optimization where if we have nothing to push we can conclude without creating expensive ServerSideMetadata.
    -Removed the readFromSharedStoreIfNecessary call from commit processing, since it is not needed there.
---
 .../solr/handler/admin/RequestApplyUpdatesOp.java  |  35 +-
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  21 +-
 .../solr/store/blob/metadata/BlobCoreSyncer.java   |  35 +-
 .../solr/store/blob/metadata/CorePushPull.java     |  92 +---
 .../solr/store/blob/metadata/PushPullData.java     |  61 +--
 .../store/blob/metadata/ServerSideMetadata.java    | 197 ++++++---
 .../blob/metadata/SharedStoreResolutionUtil.java   |  22 +-
 .../solr/store/blob/process/CorePullTask.java      | 149 ++++---
 .../solr/store/blob/process/CorePullTracker.java   |  44 +-
 .../solr/store/blob/process/CorePullerFeeder.java  |  88 +---
 .../solr/store/blob/process/CorePullerThread.java  |  91 +++-
 .../apache/solr/store/blob/process/CorePusher.java | 187 ++++++--
 .../solr/store/blob/process/CoreSyncStatus.java    |   2 -
 .../solr/store/blob/process/CoreUpdateTracker.java |  20 -
 .../solr/store/blob/util/BlobStoreUtils.java       | 118 ++---
 .../shared/SharedCoreConcurrencyController.java    | 351 +++++++++++++++
 .../solr/store/shared/SharedStoreManager.java      |  17 +-
 .../metadata/SharedShardMetadataController.java    | 117 ++---
 .../processor/DistributedZkUpdateProcessor.java    | 146 +++++-
 .../solr/store/blob/SharedStorageSplitTest.java    |   2 +-
 .../solr/store/blob/metadata/CorePushPullTest.java |  31 +-
 .../metadata/SharedStoreResolutionUtilTest.java    |  25 +-
 .../blob/process/PullMergeDeduplicationTest.java   |  96 +---
 .../solr/store/blob/util/BlobStoreUtilsTest.java   |  26 +-
 .../store/shared/SharedCoreConcurrencyTest.java    | 488 +++++++++++++++++++++
 .../shared/SimpleSharedStoreEndToEndPullTest.java  |  12 +-
 .../shared/SimpleSharedStoreEndToEndPushTest.java  |   9 +-
 .../store/shared/SolrCloudSharedStoreTestCase.java |  10 +-
 .../SharedShardMetadataControllerTest.java         | 159 +------
 29 files changed, 1758 insertions(+), 893 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/admin/RequestApplyUpdatesOp.java b/solr/core/src/java/org/apache/solr/handler/admin/RequestApplyUpdatesOp.java
index 9277c08..3658ed0 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/RequestApplyUpdatesOp.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/RequestApplyUpdatesOp.java
@@ -17,16 +17,22 @@
 
 package org.apache.solr.handler.admin;
 
+import java.io.IOException;
 import java.util.concurrent.Future;
 
 import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.Replica;
 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.store.blob.client.BlobCoreMetadataBuilder;
 import org.apache.solr.store.blob.process.CoreUpdateTracker;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.apache.solr.update.UpdateLog;
 
 class RequestApplyUpdatesOp implements CoreAdminHandler.CoreAdminOp {
@@ -76,18 +82,39 @@ class RequestApplyUpdatesOp implements CoreAdminHandler.CoreAdminOp {
   }
 
 
-  private void pushToSharedStore(SolrCore core) {
+  private void pushToSharedStore(SolrCore core) throws IOException {
     // Push the index to blob storage before we set our state to ACTIVE
     CloudDescriptor cloudDesc = core.getCoreDescriptor().getCloudDescriptor();
     if (cloudDesc.getReplicaType().equals(Replica.Type.SHARED)) {
       CoreContainer cc = core.getCoreContainer();
       CoreUpdateTracker sharedCoreTracker = new CoreUpdateTracker(cc);
 
+      String collectionName = cloudDesc.getCollectionName();
+      String shardName = cloudDesc.getShardId();
+      String coreName = core.getName();
+      SharedShardMetadataController metadataController = cc.getSharedStoreManager().getSharedShardMetadataController();
+      // creates the metadata node
+      metadataController.ensureMetadataNodeExists(collectionName, shardName);
+      SharedShardVersionMetadata shardVersionMetadata = metadataController.readMetadataValue(collectionName, shardName);
+      // TODO: We should just be initialized to a default value since this is a new shard.  
+      //       As of now we are only taking care of basic happy path. We still need to evaluate what will happen
+      //       if a split is abandoned because of failure(e.g. long GC pause) and is re-tried?
+      //       How to make sure our re-attempt wins even when the ghost of previous attempt resumes and intervenes?
+      if (!SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE.equals(shardVersionMetadata.getMetadataSuffix())) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "New sub shard has zk information that is not default");
+      }
+
+      // sync local cache with zk's default information i.e. equivalent of no-op pull 
+      SharedCoreConcurrencyController concurrencyController = cc.getSharedStoreManager().getSharedCoreConcurrencyController();
+      String sharedBlobName = Assign.buildSharedShardName(collectionName, shardName);
+      concurrencyController.updateCoreVersionMetadata(collectionName, shardName, coreName,
+          shardVersionMetadata, BlobCoreMetadataBuilder.buildEmptyCoreMetadata(sharedBlobName));
+
       sharedCoreTracker.persistShardIndexToSharedStore(
           cc.getZkController().zkStateReader.getClusterState(),
-          cloudDesc.getCollectionName(),
-          cloudDesc.getShardId(),
-          core.getName());
+          collectionName,
+          shardName,
+          coreName);
     }
   }
 }
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 c03a743..f4d495a 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -71,6 +71,7 @@ import org.apache.solr.common.params.CommonParams;
 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;
@@ -349,14 +350,19 @@ public class HttpSolrCall {
           solrReq = parser.parse(core, path, req);
         }
 
-        // don't enqueue a pull on updates as those will already trigger their own synchronous pulls
-        if (cores.isZooKeeperAware() && !doesPathContainUpdate()) {
+        if (cores.isZooKeeperAware()) {
           String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
           DocCollection collection = getCollection(collectionName);
           boolean belongsToSharedCollection =
               (collection != null) ? collection.getSharedIndex() : false;
           if (belongsToSharedCollection) {
-            enqueuePullFromSharedStore(core);
+            if (doesPathContainUpdate()) {
+              // Shared 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 {
+              enqueuePullFromSharedStore(core);
+            }
           }
         }
 
@@ -373,6 +379,15 @@ public class HttpSolrCall {
     action = PASSTHROUGH;
   }
 
+  private void addCommitIfAbsent() {
+    Boolean currentValue = solrReq.getParams().getBool(UpdateParams.COMMIT);
+    if (currentValue == null || currentValue == false) {
+      ModifiableSolrParams params = new ModifiableSolrParams(solrReq.getParams());
+      params.set(UpdateParams.COMMIT, "true");
+      solrReq.setParams(params);
+    }
+  }
+
   /**
    * Attempt to initiate a pull from the shared store. It's the client responsibility to ensure
    * only Shared replicas use this method.
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
index c352e50..a97c864 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
@@ -16,29 +16,29 @@
  */
 package org.apache.solr.store.blob.metadata;
 
-import java.util.*;
-import java.util.concurrent.*;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import com.amazonaws.annotation.GuardedBy;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Maps;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
-//import net.jcip.annotations.GuardedBy;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.PullInProgressException;
-import org.apache.solr.store.blob.process.*;
-import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.process.CorePullTracker;
+import org.apache.solr.store.blob.process.CorePullerFeeder;
+import org.apache.solr.store.blob.process.CoreSyncStatus;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.lang.invoke.MethodHandles;
 
 /**
  * Class to sync between local and blob copies of a core using
@@ -133,18 +133,6 @@ public class BlobCoreSyncer {
           log.warn("Pulling shard " + shardName + " that is inactive!");
         }
         log.info("Pulling for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-        // creates the metadata node if it doesn't exist
-        sharedShardMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-
-        /*
-         * Get the metadataSuffix value from ZooKeeper or from a cache if an entry exists for the
-         * given collection and shardName. If the leader has already changed, the conditional update
-         * later will fail and invalidate the cache entry if it exists.
-         */
-        VersionedData data = sharedShardMetadataController.readMetadataValue(collectionName, shardName,
-            /* readFromCache */ false);
-        Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
-        String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
 
         String sharedShardName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
 
@@ -153,9 +141,6 @@ public class BlobCoreSyncer {
           .setShardName(shardName)
           .setCoreName(coreName)
           .setSharedStoreName(sharedShardName)
-          .setLastReadMetadataSuffix(metadataSuffix)
-          .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
-          .setZkVersion(data.getVersion())
           .build();
         pull(pushPullData, waitForSearcher, emptyCoreAwaitingPull, cores);
       } catch (Exception ex) {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
index a96c5a5..817f230 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/CorePushPull.java
@@ -12,6 +12,7 @@ import java.util.Set;
 import java.util.concurrent.Future;
 import java.util.stream.Collectors;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.Directory;
@@ -31,11 +32,10 @@ import org.apache.solr.store.blob.metadata.ServerSideMetadata.CoreFileData;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetadataResolutionResult;
 import org.apache.solr.store.blob.process.BlobDeleteManager;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Class pushing updates from the local core to the Blob Store and pulling updates from Blob store to local core.
  * This class knows about the Solr core directory structure and can translate it into file system abstractions since that's
@@ -80,13 +80,6 @@ public class CorePushPull {
     }
 
     /**
-     * Calls {@link #pushToBlobStore(long, int)}  with current epoch time and attempt no 0.  
-     */
-    public BlobCoreMetadata pushToBlobStore() throws Exception {
-       return pushToBlobStore(System.currentTimeMillis(), 0);
-    }
-    
-    /**
      * Writes to the Blob store all the files that should be written to it, then updates and writes the {@link BlobCoreMetadata}.
      * After that call, the Blob store is fully updated for the core.<p>
      * In case of exception, it means that the new {@link BlobCoreMetadata} has not been written which means that the old
@@ -96,12 +89,12 @@ public class CorePushPull {
      * This method (need to verify this is indeed the case!) can be used either to push small updates from the local core
      * to Blob or to completely overwrite the Blob content for the core with the local content.
      * 
-     * @param requestQueuedTimeMs epoch time in milliseconds when the push request was queued(meaningful in case of async pushing)
-     *                            only used for logging purposes
-     * @param attempt 0 based attempt number (meaningful in case of pushing with retry mechanism)
-     *                only used for logging purposes 
+     * @param currentMetadataSuffix suffix of the core.metadata file corresponding to {@link CorePushPull#blobMetadata}
+     *                              TODO: there is an existing todo with delete logic where this parameter is consumed  
+     *                                    with that add a metadataSuffix field to {@link BlobCoreMetadata} 
+     * @param newMetadataSuffix suffix of the new core.metadata file to be created as part of this push
      */
-    public BlobCoreMetadata pushToBlobStore(long requestQueuedTimeMs, int attempt) throws Exception {
+    public BlobCoreMetadata pushToBlobStore(String currentMetadataSuffix, String newMetadataSuffix) throws Exception {
       long startTimeMs = System.currentTimeMillis();
       try {
         SolrCore solrCore = container.getCore(pushPullData.getCoreName());
@@ -113,39 +106,9 @@ public class CorePushPull {
           // Creating the new BlobCoreMetadata as a modified clone of the existing one
           BlobCoreMetadataBuilder bcmBuilder = new BlobCoreMetadataBuilder(blobMetadata, solrServerMetadata.getGeneration());
 
-          // First copy index files over to a temp directory and then push to blob store from there. 
-          // This is to avoid holding a lock over index directory involving network operation.
-          //
-          // Ideally, we don't need to lock source index directory to make temp copy because...:
-          // -all index files are write-once (http://mail-archives.apache.org/mod_mbox/lucene-java-user/201509.mbox/%3C00c001d0ed85$ee839b20$cb8ad160$@thetaphi.de%3E)
-          // -there is a possibility of a missing segment file because of merge but that could have happened even before reaching this point.
-          //  Whatever the timing maybe that will result in a failure (exception) when copying to temp and that will abort the push
-          // -segment file deletion(because of merge) and copying to temp happening concurrently should be ok as well since delete will wait 
-          //  for copy to finish (https://stackoverflow.com/questions/2028874/what-happens-to-an-open-file-handle-on-linux-if-the-pointed-file-gets-moved-del)
-          // ...But SfdcFSDirectoryFactory, that is based off CachingDirectoryFactory, can return a cached instance of Directory and
-          // that cache is only based off path and is rawLockType agnostic. Therefore passing "none" as rawLockType will not be honored if same path
-          // was accessed before with some other rawLockType until CachingDirectoryFactory#doneWithDirectory is called. 
-          // There is an overload that takes forceNew boolean and supposed to be returning new instance but CachingDirectoryFactory does not seem to honor that. 
-          // It is not worth fighting that therefore we lock the source index directory before copying to temp directory. If this much locking turns out
-          // to be problematic we can revisit this.
-          // 
-          // And without source locking we really don't need temp directory, one reason to still might have it is to avoid starting a push to blob store 
-          // that can potentially be stopped in the middle because of a concurrent merge deleting the segment files being pushed. 
-
-          // create a temp directory (within the core local folder).
-          String tempIndexDirPath = solrCore.getDataDir() + "index.push." + System.nanoTime();
-          Directory tempIndexDir = solrCore.getDirectoryFactory().get(tempIndexDirPath, DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
+          Directory snapshotIndexDir = solrCore.getDirectoryFactory().get(solrServerMetadata.getSnapshotDirPath(), DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
           try {
-            Directory indexDir = solrCore.getDirectoryFactory().get(solrCore.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, solrCore.getSolrConfig().indexConfig.lockType);
-            try {
-              // copy index files to the temp directory
-              for (CoreFileData cfd : resolvedMetadataResult.getFilesToPush()) {
-                copyFileToDirectory(indexDir, cfd.getFileName(), tempIndexDir);
-              }
-            } finally {
-              solrCore.getDirectoryFactory().release(indexDir);
-            }
-            
+
             /*
              * Removing from the core metadata the files that should no longer be there.
              * 
@@ -162,20 +125,21 @@ public class CorePushPull {
              * 
              * The deletion logic will move out of this class in the future and make this less confusing. 
              */
-            for (BlobCoreMetadata.BlobFile d : resolvedMetadataResult.getFilesToPull()) {
+            for (BlobCoreMetadata.BlobFile d : resolvedMetadataResult.getFilesToDelete()) {
                 bcmBuilder.removeFile(d);
                 BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete(d, System.currentTimeMillis());
                 bcmBuilder.addFileToDelete(bftd);
             }
             
             // add the old core.metadata file to delete
-            if (!pushPullData.getLastReadMetadataSuffix().equals("-1")) {
+            if (!currentMetadataSuffix.equals(SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE)) {
               // TODO This may be inefficient but we'll likely remove this when CorePushPull is refactored to have deletion elsewhere
+              //      could be added to resolvedMetadataResult#getFilesToDelete()
               ToFromJson<BlobCoreMetadata> converter = new ToFromJson<>();
               String json = converter.toJson(blobMetadata);
               int bcmSize = json.getBytes().length;
               
-              String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(pushPullData.getLastReadMetadataSuffix());
+              String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(currentMetadataSuffix);
               String coreMetadataPath = blobMetadata.getSharedBlobName() + "/" + blobCoreMetadataName;
               // so far checksum is not used for metadata file
               BlobCoreMetadata.BlobFileToDelete bftd = new BlobCoreMetadata.BlobFileToDelete("", coreMetadataPath, bcmSize, BlobCoreMetadataBuilder.UNDEFINED_VALUE, System.currentTimeMillis());
@@ -186,21 +150,13 @@ public class CorePushPull {
             // But this is untrue/totally false/misleading. SnapPuller has File all over.
             for (CoreFileData cfd : resolvedMetadataResult.getFilesToPush()) {
               // Sanity check that we're talking about the same file (just sanity, Solr doesn't update files so should never be different)
-              assert cfd.getFileSize() == tempIndexDir.fileLength(cfd.getFileName());
+              assert cfd.getFileSize() == snapshotIndexDir.fileLength(cfd.getFileName());
 
-              String blobPath = pushFileToBlobStore(coreStorageClient, tempIndexDir, cfd.getFileName(), cfd.getFileSize());
+              String blobPath = pushFileToBlobStore(coreStorageClient, snapshotIndexDir, cfd.getFileName(), cfd.getFileSize());
               bcmBuilder.addFile(new BlobCoreMetadata.BlobFile(cfd.getFileName(), blobPath, cfd.getFileSize(), cfd.getChecksum()));
             }
           } finally {
-            try {
-              // Remove temp directory
-              solrCore.getDirectoryFactory().doneWithDirectory(tempIndexDir);
-              solrCore.getDirectoryFactory().remove(tempIndexDir);
-            } catch (Exception e) {
-              log.warn("Cannot remove temp directory " + tempIndexDirPath, e);
-            } finally {
-              solrCore.getDirectoryFactory().release(tempIndexDir);
-            }
+            solrCore.getDirectoryFactory().release(snapshotIndexDir);
           }
           
           // delete what we need
@@ -208,7 +164,7 @@ public class CorePushPull {
           
           BlobCoreMetadata newBcm = bcmBuilder.build();
 
-          String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(pushPullData.getNewMetadataSuffix());          
+          String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(newMetadataSuffix);          
           coreStorageClient.pushCoreMetadata(blobMetadata.getSharedBlobName(), blobCoreMetadataName, newBcm);
           return newBcm;
         } finally {
@@ -219,7 +175,7 @@ public class CorePushPull {
         long bytesTransferred = resolvedMetadataResult.getFilesToPush().stream().mapToLong(cfd -> cfd.getFileSize()).sum();
         
         // todo correctness stuff
-        logBlobAction("PUSH", filesAffected, bytesTransferred, requestQueuedTimeMs, attempt, startTimeMs);
+        logBlobAction("PUSH", filesAffected, bytesTransferred, startTimeMs, 0, startTimeMs);
       }
     }
 
@@ -444,18 +400,6 @@ public class CorePushPull {
         return blobPath;
     }
 
-    private void removeTempDirectory(SolrCore solrCore, String tempDirPath, Directory tempDir) throws IOException {
-        try {
-            // Remove temp directory
-            solrCore.getDirectoryFactory().doneWithDirectory(tempDir);
-            solrCore.getDirectoryFactory().remove(tempDir);
-        } catch (Exception e) {
-            log.warn("Cannot remove temp directory " + tempDirPath, e);
-        } finally {
-            solrCore.getDirectoryFactory().release(tempDir);
-        }
-    }
-    
     /**
      * Logs soblb line for push or pull action 
      * TODO: This is for callers of this method.
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/PushPullData.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/PushPullData.java
index d79316f..1723488 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/PushPullData.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/PushPullData.java
@@ -16,8 +16,6 @@
  */
 package org.apache.solr.store.blob.metadata;
 
-import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
-
 /*
  * Class containing information needed to complete a push or a pull to a shared store in 
  * a Shared Collection.
@@ -33,36 +31,13 @@ public class PushPullData {
    */
   protected String sharedStoreName;
   
-  /**
-   * Unique value of the metadataSuffix for the last persisted shard index in the shared store.
-   * If this value is equal to {@link SharedShardMetadataController#METADATA_NODE_DEFAULT_VALUE} then 
-   * the shard index this instance is associated with has not yet been successfully persisted
-   */
-  protected String lastReadMetadataSuffix;
-  
-  /*
-   * Unique value of the metadataSuffix that will be appended to the core metadata file name if this 
-   * PushPullData instance is association with a push operation to the shared store.
-   */
-  protected String newMetadataSuffix;
-  
-  /*
-   * Value originating from a ZooKeeper node used to handle conditionally and safely update the 
-   * core.metadata file written to the shared store.
-   */
-  protected int version;
-  
   public PushPullData() {}
   
-  public PushPullData(String collectionName, String shardName, String coreName, String sharedStoreName, String lastReadMetadataSuffix,
-      String newMetadataSuffix, int version) {
+  public PushPullData(String collectionName, String shardName, String coreName, String sharedStoreName) {
     this.collectionName = collectionName;
     this.shardName = shardName;
     this.coreName = coreName;
     this.sharedStoreName = sharedStoreName;
-    this.lastReadMetadataSuffix = lastReadMetadataSuffix;
-    this.newMetadataSuffix = newMetadataSuffix;
-    this.version = version;
   }
 
   public String getCoreName() {
@@ -81,23 +56,10 @@ public class PushPullData {
     return sharedStoreName;
   }
 
-  public String getLastReadMetadataSuffix() {
-    return lastReadMetadataSuffix;
-  }
-  
-  public String getNewMetadataSuffix() {
-    return newMetadataSuffix;
-  }
-  
-  public int getZkVersion() {
-    return version;
-  }
-  
   @Override
   public String toString() {
-    return "collectionName=" + collectionName + " shardName=" + shardName + " coreName=" + 
-        sharedStoreName + " coreName=" + coreName + " lastReadMetadataSuffix=" + lastReadMetadataSuffix +
-        " newMetadataSuffix=" + newMetadataSuffix + " lastReadZkVersion=" + version;
+    return "collectionName=" + collectionName + " shardName=" + shardName + " sharedStoreName=" + 
+        sharedStoreName + " coreName=" + coreName;
   }
 
   public static class Builder {
@@ -124,24 +86,9 @@ public class PushPullData {
       return this;
     }
     
-    public Builder setLastReadMetadataSuffix(String lastReadMetadataSuffix) {
-      data.lastReadMetadataSuffix = lastReadMetadataSuffix;
-      return this;
-    }
-    
-    public Builder setNewMetadataSuffix(String newMetadataSuffix) {
-      data.newMetadataSuffix = newMetadataSuffix;
-      return this;
-    }
-    
-    public Builder setZkVersion(int version) {
-      data.version = version;
-      return this;
-    }
-    
     public PushPullData build() {
       return new PushPullData(data.collectionName, data.shardName, data.coreName, 
-          data.sharedStoreName, data.lastReadMetadataSuffix, data.newMetadataSuffix, data.version);
+          data.sharedStoreName);
     }    
   }
 
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
index 88d76ad..241cad7 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/ServerSideMetadata.java
@@ -2,39 +2,46 @@ package org.apache.solr.store.blob.metadata;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.file.NoSuchFileException;
 import java.security.MessageDigest;
 import java.security.NoSuchAlgorithmException;
 import java.util.Arrays;
-import java.util.List;
 import java.util.Objects;
 
+import com.google.common.collect.ImmutableCollection;
+import com.google.common.collect.ImmutableCollection.Builder;
+import com.google.common.collect.ImmutableSet;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.BlobException;
-
-import com.google.common.collect.ImmutableCollection;
-import com.google.common.collect.ImmutableSet;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Object capturing the metadata of a shard index on a Solr node. 
- * 
+ *
  * This works in conjunction with {@link BlobCoreMetadata} to find the differences between 
  * local (Solr node) and remote (Blob store) commit point for a core.<p>
- * 
+ *
  * This object is somewhere between {@link org.apache.lucene.index.IndexCommit} and {@link org.apache.lucene.index.SegmentInfos}
  * and by implementing it separately we can add additional metadata to it as needed.
  */
 public class ServerSideMetadata {
-  
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final int MAX_ATTEMPTS_TO_CAPTURE_COMMIT_POINT = 5;
   /**
    * Files composing the core. They are are referenced from the core's current commit point's segments_N file
    * which is ALSO included in this collection.
@@ -63,83 +70,155 @@ public class ServerSideMetadata {
   private final SolrCore core;
   private final String coreName;
   private final CoreContainer container;
+  /**
+   * path of snapshot directory if we are supposed to take snapshot of the active segment files, otherwise, null
+   */
+  private final String snapshotDirPath;
+
+  public ServerSideMetadata(String coreName, CoreContainer container) throws Exception {
+    this(coreName, container, false);
+  }
 
   /**
    * Given a core name, builds the local metadata
-   * 
-   * 
+   *
+   * @param takeSnapshot whether to take snapshot of active segments or not. If true then the snapshot directory path can be 
+   *                     found through {@link #getSnapshotDirPath()}.
+   *
    * @throws Exception if core corresponding to <code>coreName</code> can't be found.
    */
-  public ServerSideMetadata(String coreName, CoreContainer container) throws Exception {
+  public ServerSideMetadata(String coreName, CoreContainer container, boolean takeSnapshot) throws Exception {
     this.coreName = coreName;
     this.container = container;
     this.core = container.getCore(coreName);
 
     if (core == null) {
-      throw new Exception("Can't find core " + coreName);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Can't find core " + coreName);
     }
 
     try {
-      IndexCommit latestCommit = core.getDeletionPolicy().getLatestCommit();
-      if (latestCommit == null) {
-        throw new BlobException("Core " + coreName + " has no available commit point");
-      }
+      Directory coreDir = core.getDirectoryFactory().get(core.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+      try {
 
-      generation = latestCommit.getGeneration();
+        if (takeSnapshot) {
+          snapshotDirPath = core.getDataDir() + "index.snapshot." + System.nanoTime();
+        } else {
+          snapshotDirPath = null;
+        }
 
-      // Work around possible bug returning same file multiple times by using a set here
-      // See org.apache.solr.handler.ReplicationHandler.getFileList()
-      ImmutableCollection.Builder<CoreFileData> latestCommitBuilder = new ImmutableSet.Builder<>();
-      ImmutableCollection.Builder<CoreFileData> allCommitsBuilder;
+        ImmutableCollection.Builder<CoreFileData> latestCommitBuilder;
+        IndexCommit latestCommit;
+        int attempt = 1;
+        // we don't have an atomic way of capturing a commit point i.e. there is a slight chance of losing files between 
+        // getting a latest commit and reserving it. Therefore, we try to capture commit point in a loop with maximum 
+        // number of attempts. 
+        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 ImmutableSet.Builder<>();
+            latestCommit = tryCapturingLatestCommit(coreDir, latestCommitBuilder);
+            break;
+          } catch (FileNotFoundException | NoSuchFileException ex) {
+            attempt++;
+            if (attempt > MAX_ATTEMPTS_TO_CAPTURE_COMMIT_POINT) {
+              throw ex;
+            }
+            log.info(String.format("Failed to capture commit point: core=%s attempt=%s reason=%s",
+                coreName, attempt, ex.getMessage()));
+          }
+        }
+
+        generation = latestCommit.getGeneration();
+        latestCommitFiles = latestCommitBuilder.build();
 
-      Directory coreDir = core.getDirectoryFactory().get(core.getIndexDir(), DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
-      try {
         // Capture now the hash and verify again if we need to pull content from the Blob store into this 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.
+        // TODO: revise with "design assumptions around pull pipeline" mentioned in allCommits TODO below
         directoryHash = getSolrDirectoryHash(coreDir);
 
-        buildCommitFiles(coreDir, latestCommit, latestCommitBuilder);
-
-        // A note on listCommits says that it does not guarantee consistent results if a commit is in progress.
-        // But in blob context we serialize commits and pulls by proper locking therefore we should be good here.
-        List<IndexCommit> allCommits = DirectoryReader.listCommits(coreDir);
-
-        // optimization:  normally we would only be dealing with one commit point. In that case just reuse latest commit files builder.
-        if (allCommits.size() > 1 ){
-          allCommitsBuilder = new ImmutableSet.Builder<>();
-          for (IndexCommit commit: allCommits) {
-            buildCommitFiles(coreDir, commit, allCommitsBuilder);
-          }
-        } else {
-          // we should always have a commit point as verified in the beginning of this method.
-          assert allCommits.size() == 1 && allCommits.get(0).equals(latestCommit);
-          allCommitsBuilder = latestCommitBuilder;
-        }
+        allCommitsFiles = latestCommitFiles;
+        // TODO: allCommits was added to detect special cases where inactive file segments can potentially conflict
+        //       with whats in shared store. But given the recent understanding of semantics around index directory locks
+        //       we need to revise our design assumptions around pull pipeline, including this one.
+        //       Disabling this for now so that unreliability around introspection of older commits 
+        //       might not get in the way of steady state indexing.
+//        // A note on listCommits says that it does not guarantee consistent results if a commit is in progress.
+//        // But in blob context we serialize commits and pulls by proper locking therefore we should be good here.
+//        List<IndexCommit> allCommits = DirectoryReader.listCommits(coreDir);
+//
+//        // we should always have a commit point as verified in the beginning of this method.
+//        assert (allCommits.size() > 1) || (allCommits.size() == 1 && allCommits.get(0).equals(latestCommit));
+//
+//        // optimization:  normally we would only be dealing with one commit point. In that case just reuse latest commit files builder.
+//        ImmutableCollection.Builder<CoreFileData> allCommitsBuilder = latestCommitBuilder;
+//        if (allCommits.size() > 1) {
+//          allCommitsBuilder = new ImmutableSet.Builder<>();
+//          for (IndexCommit commit : allCommits) {
+//            // no snapshot for inactive segments files
+//            buildCommitFiles(coreDir, commit, allCommitsBuilder, /* snapshotDir */ null);
+//          }
+//        }
+//        allCommitsFiles = allCommitsBuilder.build();
       } finally {
         core.getDirectoryFactory().release(coreDir);
       }
-      latestCommitFiles = latestCommitBuilder.build();
-      allCommitsFiles = allCommitsBuilder.build();
     } finally {
       core.close();
     }
   }
 
+  private IndexCommit tryCapturingLatestCommit(Directory coreDir, Builder<CoreFileData> latestCommitBuilder) throws BlobException, IOException {
+    IndexDeletionPolicyWrapper deletionPolicy = core.getDeletionPolicy();
+    IndexCommit latestCommit = deletionPolicy.getLatestCommit();
+    if (latestCommit == null) {
+      throw new BlobException("Core " + core.getName() + " has no available commit point");
+    }
+
+    deletionPolicy.saveCommitPoint(latestCommit.getGeneration());
+    try {
+      buildCommitFiles(coreDir, latestCommit, latestCommitBuilder);
+      return latestCommit;
+    } finally {
+      deletionPolicy.releaseCommitPoint(latestCommit.getGeneration());
+    }
+  }
+
   private void buildCommitFiles(Directory coreDir, IndexCommit commit, ImmutableCollection.Builder<CoreFileData> builder) throws IOException {
-    for (String fileName : commit.getFileNames()) {
-      // Note we add here all segment related files as well as the commit point's segments_N file
-      // Note commit points do not contain lock (write.lock) files.
-      try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
-        long length = indexInput.length();
-        long checksum = CodecUtil.retrieveChecksum(indexInput);
-        builder.add(new CoreFileData(fileName, length, checksum));
+    Directory snapshotDir = null;
+    try {
+      if (snapshotDirPath != null) {
+        snapshotDir = core.getDirectoryFactory().get(snapshotDirPath, DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+      }
+      for (String fileName : commit.getFileNames()) {
+        // Note we add here all segment related files as well as the commit point's segments_N file
+        // Note commit points do not contain lock (write.lock) files.
+        try (final IndexInput indexInput = coreDir.openInput(fileName, IOContext.READONCE)) {
+          long length = indexInput.length();
+          long checksum = CodecUtil.retrieveChecksum(indexInput);
+          builder.add(new CoreFileData(fileName, length, checksum));
+        }
+        if (snapshotDir != null) {
+          // take snapshot of the file
+          snapshotDir.copyFrom(coreDir, fileName, fileName, DirectoryFactory.IOCONTEXT_NO_CACHE);
+        }
+      }
+    } catch (Exception ex) {
+      if (snapshotDir != null) {
+        core.getDirectoryFactory().doneWithDirectory(snapshotDir);
+        core.getDirectoryFactory().remove(snapshotDir);
+      }
+      throw ex;
+    } finally {
+      if (snapshotDir != null) {
+        core.getDirectoryFactory().release(snapshotDir);
       }
     }
   }
 
   public String getCoreName() {
-      return this.coreName;
+    return this.coreName;
   }
 
   public CoreContainer getCoreContainer() {
@@ -162,6 +241,10 @@ public class ServerSideMetadata {
     return this.allCommitsFiles;
   }
 
+  public String getSnapshotDirPath() {
+    return snapshotDirPath;
+  }
+
   /**
    * Returns <code>true</code> 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.<p>
@@ -206,9 +289,9 @@ public class ServerSideMetadata {
   @Override
   public String toString() {
     return "collectionName=" + core.getCoreDescriptor().getCollectionName() +
-      " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() +
-      " coreName=" + core.getName() +
-      " generation=" + generation;
+        " shardName=" + core.getCoreDescriptor().getCloudDescriptor().getShardId() +
+        " coreName=" + core.getName() +
+        " generation=" + generation;
   }
 
   /**
@@ -231,12 +314,12 @@ public class ServerSideMetadata {
     public boolean equals(Object o) {
       if (this == o) return true;
       if (o == null || getClass() != o.getClass()) return false;
-  
+
       CoreFileData other = (CoreFileData) o;
-  
+
       return Objects.equals(fileName, other.fileName) &&
-        Objects.equals(fileSize, other.fileSize) &&
-        Objects.equals(checksum, other.checksum);
+          Objects.equals(fileSize, other.fileSize) &&
+          Objects.equals(checksum, other.checksum);
     }
 
     public String getFileName() {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
index b01ee94..50a2c5d 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtil.java
@@ -30,6 +30,8 @@ public class SharedStoreResolutionUtil {
     private final Collection<CoreFileData> filesToPush;
     // blob files needed to be pulled
     private final Collection<BlobFile> filesToPull;
+    // blob files needed to be deleted
+    private final Collection<BlobFile> filesToDelete;
     // Whether the local index contents conflict with contents to be pulled from blob. If they do we will move the
     // core to new index dir when pulling blob contents
     // Two cases:
@@ -39,7 +41,7 @@ public class SharedStoreResolutionUtil {
     
     
     public SharedMetadataResolutionResult(Collection<CoreFileData> filesToPush, 
-        Collection<BlobFile> filesToPull, boolean localConflictingWithBlob) {
+        Collection<BlobFile> filesToPull, Collection<BlobFile> filesToDelete, boolean localConflictingWithBlob) {
       if (filesToPush == null) {
         this.filesToPush = Collections.emptySet();
       } else {
@@ -52,6 +54,12 @@ public class SharedStoreResolutionUtil {
         this.filesToPull = filesToPull;
       }
 
+      if (filesToDelete == null) {
+        this.filesToDelete = Collections.emptySet();
+      } else {
+        this.filesToDelete = filesToDelete;
+      }
+
       this.localConflictingWithBlob = localConflictingWithBlob;
     }
     
@@ -62,7 +70,11 @@ public class SharedStoreResolutionUtil {
     public Collection<BlobFile> getFilesToPull() {
       return filesToPull;
     }
-    
+
+    public Collection<BlobFile> getFilesToDelete() {
+      return filesToDelete;
+    }
+
     public boolean isLocalConflictingWithBlob(){
       return localConflictingWithBlob;
     }
@@ -108,7 +120,7 @@ public class SharedStoreResolutionUtil {
         || distant.getBlobFiles().length == 0) {
       // The shard index data does not exist on the shared store. All we can do is push. 
       // We've computed localFilesMissingOnBlob above, and blobFilesMissingLocally is empty as it should be.
-      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), false);
+      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), blobFilesMissingLocally.values(), false);
     }
     
     // Verify we find one and only one segments_N file to download from Blob.
@@ -136,7 +148,7 @@ public class SharedStoreResolutionUtil {
     if (local == null) {
       // The shard index data does not exist locally. All we can do is pull.  
       // We've computed blobFilesMissingLocally and localFilesMissingOnBlob is empty as it should be.
-      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), false);
+      return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), blobFilesMissingLocally.values(), blobFilesMissingLocally.values(), false);
     }
 
     boolean localConflictingWithBlob = false;
@@ -170,7 +182,7 @@ public class SharedStoreResolutionUtil {
     // resolver to produce list of files to be pulled from blob and list of files to be pulled(read copied) from local index directory.
     // But that would have unnecessarily convoluted the design of this resolver.
     Collection<BlobFile> filesToPull = localConflictingWithBlob ? Arrays.asList(distant.getBlobFiles()) : blobFilesMissingLocally.values();
-    return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), filesToPull, localConflictingWithBlob);
+    return new SharedMetadataResolutionResult(localFilesMissingOnBlob.values(), filesToPull, blobFilesMissingLocally.values(), localConflictingWithBlob);
   }
   
   /** Identify the segments_N file in Blob files. */
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index 0e1260d..05dcd84 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -23,6 +23,8 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Throwables;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
@@ -30,6 +32,7 @@ import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.client.BlobCoreMetadataBuilder;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.metadata.CorePushPull;
 import org.apache.solr.store.blob.metadata.ServerSideMetadata;
@@ -39,12 +42,14 @@ import org.apache.solr.store.blob.process.CorePullerFeeder.PullCoreInfo;
 import org.apache.solr.store.blob.provider.BlobStorageProvider;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.apache.solr.store.blob.util.DeduplicatingList;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreStage;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Throwables;
-
 /**
  * Code for pulling updates on a specific core to the Blob store. see {@CorePushTask} for the push version of this.
  */
@@ -63,12 +68,10 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
   private final PullCoreInfo pullCoreInfo;
   
   /**
-   * Data structures injected as dependencies that track the core pulls occurring
-   * in flight and the cores that have been created and not pulled. These should
-   * be passed in via a constructor from CorePullerFeeder where they are defined
+   * Data structure injected as dependencies that track the cores that have been created and not pulled. 
+   * This should be passed in via a constructor from CorePullerFeeder where it is defined
    * and are unique per CorePullerFeeder (itself a singleton).
    */
-  private final HashMap<String, Long> pullsInFlight;
   private final Set<String> coresCreatedNotPulledYet;
   
   private final long queuedTimeMs;
@@ -76,23 +79,20 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
   private long lastAttemptTimestamp;
   private final PullCoreCallback callback;
 
-  CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, PullCoreCallback callback,
-      HashMap<String, Long> pullsInFlight, Set<String> coresCreatedNotPulledYet) {
+  CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, PullCoreCallback callback, Set<String> coresCreatedNotPulledYet) {
     this(coreContainer, pullCoreInfo, System.currentTimeMillis(), 0, 0L, 
-        callback, pullsInFlight, coresCreatedNotPulledYet);
+        callback, coresCreatedNotPulledYet);
   }
 
   @VisibleForTesting
   CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, long queuedTimeMs, int attempts,
-      long lastAttemptTimestamp, PullCoreCallback callback, HashMap<String, Long> pullsInFlight, 
-      Set<String> coresCreatedNotPulledYet) {
+      long lastAttemptTimestamp, PullCoreCallback callback, Set<String> coresCreatedNotPulledYet) {
     this.coreContainer = coreContainer;
     this.pullCoreInfo = pullCoreInfo;
     this.queuedTimeMs = queuedTimeMs;
     this.attempts = attempts;
     this.lastAttemptTimestamp = lastAttemptTimestamp;
     this.callback = callback;
-    this.pullsInFlight = pullsInFlight;
     this.coresCreatedNotPulledYet = coresCreatedNotPulledYet;
   }
   
@@ -101,7 +101,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
    */
   @Override
   public String getDedupeKey() {
-    return this.pullCoreInfo.getSharedStoreName();
+    return this.pullCoreInfo.getCoreName();
   }
 
   /**
@@ -153,7 +153,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       // We merge the tasks.
       return new CorePullTask(task1.coreContainer, mergedPullCoreInfo,
           Math.min(task1.queuedTimeMs, task2.queuedTimeMs), mergedAttempts, mergedLatAttemptsTimestamp,
-          task1.callback, task1.pullsInFlight, task1.coresCreatedNotPulledYet);
+          task1.callback, task1.coresCreatedNotPulledYet);
     }
   }
 
@@ -185,12 +185,16 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
     return this.queuedTimeMs;
   }
 
+  public CoreContainer getCoreContainer() {
+    return coreContainer;
+  }
+
   /**
    * Pulls the local core updates from the Blob store then calls the task callback to notify the
    * {@link CorePullerFeeder} of success or failure of the operation, give an indication of the reason the periodic
    * puller can decide to retry or not.
    */
-  void pullCoreFromBlob() throws InterruptedException {
+  void pullCoreFromBlob(boolean isLeaderPulling) throws InterruptedException {
     BlobCoreMetadata blobMetadata = null;
     if (coreContainer.isShutDown()) {
       this.callback.finishedPull(this, blobMetadata, CoreSyncStatus.SHUTTING_DOWN, null);
@@ -199,23 +203,6 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       return;
     }
 
-    synchronized (pullsInFlight) {
-      Long pullInFlightTimestamp = pullsInFlight.get(pullCoreInfo.getSharedStoreName());
-      if (pullInFlightTimestamp != null) {
-        // Another pull is in progress, we'll retry later.
-        // Note we can't just cancel this pull, because the other pull might be working on a previous commit
-        // point.
-        long prevPullMs = System.currentTimeMillis() - pullInFlightTimestamp;
-        this.callback.finishedPull(this, blobMetadata, CoreSyncStatus.CONCURRENT_SYNC,
-            "Skipping core pull for " + pullCoreInfo.getSharedStoreName()
-            + " because another thread is currently pulling it (started " + prevPullMs
-            + " ms ago).");
-        return;
-      } else {
-        pullsInFlight.put(pullCoreInfo.getSharedStoreName(), System.currentTimeMillis());
-      }
-    }
-
     // Copying the non final variables so we're clean wrt the Java memory model and values do not change as we go
     // (even though we know that no other thread can be working on this CorePullTask when we handle it here).
     final int attemptsCopy = getAttempts();
@@ -228,6 +215,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       }
     }
 
+    SharedCoreConcurrencyController concurrencyController = coreContainer.getSharedStoreManager().getSharedCoreConcurrencyController();
     CoreSyncStatus syncStatus = CoreSyncStatus.FAILURE;
     // Auxiliary information related to pull outcome. It can be metadata resolver message which can be null or exception detail in case of failure 
     String message = null;
@@ -236,8 +224,30 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       BlobStorageProvider blobProvider = coreContainer.getSharedStoreManager().getBlobStorageProvider(); 
       CoreStorageClient blobClient = blobProvider.getClient();
 
+      SharedCoreVersionMetadata coreVersionMetadata = concurrencyController.getCoreVersionMetadata(pullCoreInfo.getCollectionName(),
+          pullCoreInfo.getShardName(),
+          pullCoreInfo.getCoreName());
+
+      SharedShardMetadataController metadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController();
+      SharedShardVersionMetadata shardVersionMetadata =  metadataController.readMetadataValue(pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName());
+      
+      if(concurrencyController.areVersionsEqual(coreVersionMetadata, shardVersionMetadata)) {
+        // already in sync
+        this.callback.finishedPull(this, coreVersionMetadata.getBlobCoreMetadata(), CoreSyncStatus.SUCCESS_EQUIVALENT, null);
+        return;
+      } 
+      if (SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE.equals(shardVersionMetadata.getMetadataSuffix())) {
+        // no-op pull
+        BlobCoreMetadata emptyBlobCoreMetadata = BlobCoreMetadataBuilder.buildEmptyCoreMetadata(pullCoreInfo.getSharedStoreName());
+        concurrencyController.updateCoreVersionMetadata(pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName(), pullCoreInfo.getCoreName(), 
+            shardVersionMetadata, emptyBlobCoreMetadata, isLeaderPulling);
+        this.callback.finishedPull(this, emptyBlobCoreMetadata, CoreSyncStatus.SUCCESS_EQUIVALENT, null);
+        return;
+      }
+
+      concurrencyController.recordState(pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName(), pullCoreInfo.getCoreName(), SharedCoreStage.BlobPullStarted);
       // Get blob metadata
-      String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(pullCoreInfo.getLastReadMetadataSuffix());
+      String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(shardVersionMetadata.getMetadataSuffix());
       blobMetadata = blobClient.pullCoreMetadata(pullCoreInfo.getSharedStoreName(), blobCoreMetadataName);
       
       // Handle callback
@@ -264,10 +274,8 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
           // If we get to this point, we're setting the "created not pulled yet" status of the core here (only place
           // in the code where this happens) and we're clearing it in the finally below.
           // We're not leaking entries in coresCreatedNotPulledYet that might stay there forever...
-          synchronized (pullsInFlight) {
-            synchronized (coresCreatedNotPulledYet) {
-              coresCreatedNotPulledYet.add(pullCoreInfo.getSharedStoreName());
-            }
+          synchronized (coresCreatedNotPulledYet) {
+            coresCreatedNotPulledYet.add(pullCoreInfo.getSharedStoreName());
           }
           createCore(pullCoreInfo);
         } else {
@@ -288,9 +296,14 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       if (resolutionResult.getFilesToPull().size() > 0) {
         BlobDeleteManager deleteManager = coreContainer.getSharedStoreManager().getBlobDeleteManager();
         CorePushPull cp = new CorePushPull(blobClient, deleteManager, pullCoreInfo, resolutionResult, serverMetadata, blobMetadata);
+        // TODO: we are computing/tracking attempts but we are not passing it along
         cp.pullUpdateFromBlob(/* waitForSearcher */ true);
+        concurrencyController.updateCoreVersionMetadata(pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName(), pullCoreInfo.getCoreName(), 
+            shardVersionMetadata, blobMetadata, isLeaderPulling);
         syncStatus = CoreSyncStatus.SUCCESS;
       } else {
+        log.warn(String.format("Why there are no files to pull even when we do not match with the version in zk? collection=%s shard=%s core=%s",
+            pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName(), pullCoreInfo.getCoreName()));
         syncStatus = CoreSyncStatus.SUCCESS_EQUIVALENT;
       }
 
@@ -308,43 +321,43 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       message = Throwables.getStackTraceAsString(e);
       log.warn("Failed (attempt=" + attemptsCopy + ") to pull core " + pullCoreInfo.getSharedStoreName(), e);
     } finally {
-      // Remove ourselves from the in flight set before calling the callback method (just in case it takes
-      // forever)
-      synchronized (pullsInFlight) {
-        // No matter how the pull ends (success or any kind of error), we don't want to consider the core as awaiting pull,
-        // since it doesn't anymore (code is inline here rather than in a method or in notifyEndOfPull() to make
-        // it clear how coresCreatedNotPulledYet is managed).
-        synchronized (coresCreatedNotPulledYet) {
-          // TODO: Can we move this business of core creation and deletion outside of this task so that
-          //       we may not sub-optimally repeatedly create/delete core in case of reattempt of a transient pull error?
-          //       or get whether a reattempt will be made or not, and if there is a guaranteed reattempt then do not delete it
-          if (coresCreatedNotPulledYet.remove(pullCoreInfo.getSharedStoreName())) {
-            if (!syncStatus.isSuccess()) {
-              // If we created the core and we could not pull successfully then we should cleanup after ourselves by deleting it
-              // otherwise queries can incorrectly return 0 results from that core.
-              if(coreExists(pullCoreInfo.getCoreName())) {
-                try {
-                  // try to delete core within 3 minutes. In future when we time box our pull task then we 
-                  // need to make sure this value is within that bound. 
-                  // CoreDeleter.deleteCoreByName(coreContainer, pullCoreInfo.coreName, 3, TimeUnit.MINUTES);
-                  // TODO: need to migrate deleter
-                } catch (Exception ex) {
-                  // TODO: should we gack?
-                  //       can we do anything more here since we are unable to delete and we are leaving an empty core behind
-                  //       when we should not. Should we keep the core in coresCreatedNotPulledYet and try few more times
-                  //       but at some point we would have to let it go
-                  //       So may be, few more attempts here and then gack
-                  log.warn("CorePullTask successfully created local core but failed to pull it" +
-                      " and now is unable to delete that local core " + pullCoreInfo.getCoreName(), ex);
-                }
+      // No matter how the pull ends (success or any kind of error), we don't want to consider the core as awaiting pull,
+      // since it doesn't anymore (code is inline here rather than in a method or in notifyEndOfPull() to make
+      // it clear how coresCreatedNotPulledYet is managed).
+      synchronized (coresCreatedNotPulledYet) {
+        // TODO: Can we move this business of core creation and deletion outside of this task so that
+        //       we may not sub-optimally repeatedly create/delete core in case of reattempt of a transient pull error?
+        //       or get whether a reattempt will be made or not, and if there is a guaranteed reattempt then do not delete it
+        if (coresCreatedNotPulledYet.remove(pullCoreInfo.getSharedStoreName())) {
+          if (!syncStatus.isSuccess()) {
+            // If we created the core and we could not pull successfully then we should cleanup after ourselves by deleting it
+            // otherwise queries can incorrectly return 0 results from that core.
+            if (coreExists(pullCoreInfo.getCoreName())) {
+              try {
+                // try to delete core within 3 minutes. In future when we time box our pull task then we 
+                // need to make sure this value is within that bound. 
+                // CoreDeleter.deleteCoreByName(coreContainer, pullCoreInfo.coreName, 3, TimeUnit.MINUTES);
+                // TODO: need to migrate deleter
+              } catch (Exception ex) {
+                // TODO: should we gack?
+                //       can we do anything more here since we are unable to delete and we are leaving an empty core behind
+                //       when we should not. Should we keep the core in coresCreatedNotPulledYet and try few more times
+                //       but at some point we would have to let it go
+                //       So may be, few more attempts here and then gack
+                log.warn("CorePullTask successfully created local core but failed to pull it" +
+                    " and now is unable to delete that local core " + pullCoreInfo.getCoreName(), ex);
               }
             }
           }
         }
-        pullsInFlight.remove(pullCoreInfo.getSharedStoreName());
       }
     }
     this.callback.finishedPull(this, blobMetadata, syncStatus, message);
+    concurrencyController.recordState(pullCoreInfo.getCollectionName(), pullCoreInfo.getShardName(), pullCoreInfo.getCoreName(), SharedCoreStage.BlobPullFinished);
+  }
+
+  void finishedPull(BlobCoreMetadata blobCoreMetadata, CoreSyncStatus syncStatus, String message) throws InterruptedException {
+    this.callback.finishedPull(this, blobCoreMetadata, syncStatus, message);
   }
 
   /**
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTracker.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTracker.java
index 6d53bc6..ecebc83 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTracker.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTracker.java
@@ -16,28 +16,24 @@
  */
 package org.apache.solr.store.blob.process;
 
+import javax.servlet.http.HttpServletRequest;
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.Map;
 
-import javax.servlet.http.HttpServletRequest;
-
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.servlet.SolrRequestParsers;
 import org.apache.solr.store.blob.metadata.PushPullData;
 import org.apache.solr.store.blob.process.CorePullerFeeder.PullCoreInfo;
-import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.apache.solr.store.blob.util.DeduplicatingList;
-import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
-
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -70,15 +66,21 @@ public class CorePullTracker {
   /**
    * If the local core is stale, enqueues it to be pulled in from blob
    * TODO: add stricter checks so that we don't pull on every request
+   *       one suggestion: 
+   *       For leaders we only need to do this once when they become leader so that 
+   *       they can pull without waiting for indexing to refresh a stale core
+   *       In other words, we should make use of 
+   *       {@link SharedCoreVersionMetadata#isSoftGuaranteeOfEquality()}
+   *       {@link SharedCoreConcurrencyController#updateCoreVersionMetadata(String, String, String, boolean)}
+   *       and knowledge of leader/follower and when they change.
+   *       
    */
   public void enqueueForPullIfNecessary(String requestPath, SolrCore core, String collectionName,
-      CoreContainer cores) throws IOException, SolrException {
+      CoreContainer cores) throws SolrException {
     // Initialize variables
     String coreName = core.getName();
     String shardName = core.getCoreDescriptor().getCloudDescriptor().getShardId();
-    SharedShardMetadataController sharedShardMetadataController = cores.getSharedStoreManager().getSharedShardMetadataController();
     DocCollection collection = cores.getZkController().getClusterState().getCollection(collectionName);
-
     Slice shard = collection.getSlicesMap().get(shardName);
     if (shard != null) {
       try {
@@ -87,18 +89,6 @@ public class CorePullTracker {
           log.warn("Enqueueing a pull for shard " + shardName + " that is inactive!");
         }
         log.info("Enqueue a pull for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-        // creates the metadata node if it doesn't exist
-        sharedShardMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-
-        /*
-         * Get the metadataSuffix value from ZooKeeper or from a cache if an entry exists for the 
-         * given collection and shardName. If the leader has already changed, the conditional update
-         * later will fail and invalidate the cache entry if it exists. 
-         */
-        VersionedData data = sharedShardMetadataController.readMetadataValue(collectionName, shardName, 
-            /* readFromCache */ true);
-        Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
-        String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
 
         String sharedShardName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
 
@@ -107,12 +97,9 @@ public class CorePullTracker {
             .setShardName(shardName)
             .setCoreName(coreName)
             .setSharedStoreName(sharedShardName)
-            .setLastReadMetadataSuffix(metadataSuffix)
-            .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
-            .setZkVersion(data.getVersion())
             .build();
 
-        enqueueForPullIfNecessary(requestPath, pushPullData, cores);
+        enqueueForPullIfNecessary(requestPath, pushPullData);
 
       } catch (Exception ex) {
         // wrap every thrown exception in a solr exception
@@ -125,8 +112,7 @@ public class CorePullTracker {
    * If the local core is stale, enqueues it to be pulled in from blob Note : If there is no coreName available in the
    * requestPath we simply ignore the request.
    */
-  public void enqueueForPullIfNecessary(String requestPath, PushPullData pushPullData, 
-      CoreContainer cores) throws IOException {
+  public void enqueueForPullIfNecessary(String requestPath, PushPullData pushPullData) {
     // TODO: do we need isBackgroundPullEnabled in addition to isBlobEnabled? If not we should remove this.
     // TODO: always pull for this hack - want to check if local core is up to date
     if (isBackgroundPullEnabled && pushPullData.getSharedStoreName() != null && shouldPullStale(requestPath)) {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index 4f5a223..5fdfa6b 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -17,9 +17,9 @@
 package org.apache.solr.store.blob.process;
 
 import java.lang.invoke.MethodHandles;
-import java.util.HashMap;
 import java.util.Set;
 
+import com.google.common.collect.Sets;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.metadata.BlobCoreSyncer;
@@ -28,9 +28,6 @@ import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
 /**
  * A pull version of {@link CoreSyncFeeder} then will continually ({@link #feedTheMonsters()}) to load up a work queue (
  * {@link #pullTaskQueue}) with such tasks {@link CorePullTask} to keep the created threads busy :) The tasks will be
@@ -49,23 +46,11 @@ public class CorePullerFeeder extends CoreSyncFeeder {
 
   protected final DeduplicatingList<String, CorePullTask> pullTaskQueue;
 
-  /**
-   * Cores currently being pulled and timestamp of pull start (to identify stuck ones in logs)
-   *
-   * Note, it is the client's responsibility to synchronize accesses
-   */
-  private final HashMap<String, Long> pullsInFlight = Maps.newHashMap();
-
   /** Cores unknown locally that got created as part of the pull process but for which no data has been pulled yet
    * from Blob store. If we ignore this transitory state, these cores can be accessed locally and simply look empty.
    * We'd rather treat threads attempting to access such cores like threads attempting to access an unknown core and
    * do a pull (or more likely wait for an ongoing pull to finish).<p>
    *
-   * When this lock has to be taken as well as {@link #pullsInFlight}, then {@link #pullsInFlight} has to be taken first.
-   * Reading this set implies acquiring the monitor of the set (as if @GuardedBy("itself")), but writing to the set
-   * additionally implies holding the {@link #pullsInFlight}. This guarantees that while {@link #pullsInFlight}
-   * is held, no element in the set is changing.
-   *
    * Note, it is the client's responsibility to synchronize accesses
    */
   private final Set<String> coresCreatedNotPulledYet = Sets.newHashSet();
@@ -102,16 +87,17 @@ public class CorePullerFeeder extends CoreSyncFeeder {
     return callback;
   }
 
-  protected HashMap<String, Long> getPullsInFlight() {
-    return pullsInFlight;
-  }
-
   protected Set<String> getCoresCreatedNotPulledYet() {
     return coresCreatedNotPulledYet;
   }
 
   @Override
   void feedTheMonsters() throws InterruptedException {
+    while (cores.getSharedStoreManager() == null) {
+      // todo: Fix cyclic initialization sequence
+      // if thread starts early it will be killed since the initialization of sharedStoreManager has triggered the
+      // creation of this thread and following line will throw NPE.
+    }
     CorePullTracker tracker = cores.getSharedStoreManager().getCorePullTracker();
     final long minMsBetweenLogs = 15000;
     long lastLoggedTimestamp = 0L;
@@ -121,7 +107,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       PullCoreInfo pci = tracker.getCoreToPull();
 
       // Add the core to the list consumed by the thread doing the actual work
-      CorePullTask pt = new CorePullTask(cores, pci, getCorePullTaskCallback(), pullsInFlight, coresCreatedNotPulledYet);
+      CorePullTask pt = new CorePullTask(cores, pci, getCorePullTaskCallback(), coresCreatedNotPulledYet);
       pullTaskQueue.addDeduplicated(pt, /* isReenqueue */ false);
       syncsEnqueuedSinceLastLog++;
 
@@ -149,24 +135,21 @@ public class CorePullerFeeder extends CoreSyncFeeder {
     private final boolean createCoreIfAbsent;
 
     PullCoreInfo(PushPullData data, boolean createCoreIfAbsent, boolean waitForSearcher) {
-      super(data.getCollectionName(), data.getShardName(), data.getCoreName(), data.getSharedStoreName(), 
-          data.getLastReadMetadataSuffix(), data.getNewMetadataSuffix(), data.getZkVersion());
+      super(data.getCollectionName(), data.getShardName(), data.getCoreName(), data.getSharedStoreName());
       this.waitForSearcher = waitForSearcher;
       this.createCoreIfAbsent = createCoreIfAbsent;
     }
 
     PullCoreInfo(String collectionName, String shardName, String coreName, String sharedStoreName,
-        String lastReadMetadataSuffix, String newMetadataSuffix, int zkVersion,
-        boolean createCoreIfAbsent, boolean waitForSearcher) {
-      super(collectionName, shardName, coreName, sharedStoreName, lastReadMetadataSuffix,
-          newMetadataSuffix, zkVersion);
+                 boolean createCoreIfAbsent, boolean waitForSearcher) {
+      super(collectionName, shardName, coreName, sharedStoreName);
       this.waitForSearcher = waitForSearcher;
       this.createCoreIfAbsent = createCoreIfAbsent;
     }
 
     @Override
     public String getDedupeKey() {
-      return sharedStoreName;
+      return coreName;
     }
 
     public boolean shouldWaitForSearcher() {
@@ -192,7 +175,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       assert v1.getSharedStoreName().equals(v2.getSharedStoreName());
       assert v1.getDedupeKey().equals(v2.getDedupeKey());
       assert v1.getCoreName().equals(v2.getCoreName());
-      
+
       // Merging the version number here implies an ordering on the pull operation
       // enqueued as we want higher version-ed operations to be what the pulling
       // mechanisms pull off of due to presence of metadataSuffix information
@@ -201,46 +184,13 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       boolean waitForSearcher = false;
       boolean createCoreIfAbsent = false;
 
-      // TODO newMetadataSuffix isn't used in the pull pipeline but the argument is nevertheless
-      // required when enqueuing a new pull and propagated downward. We should refactor 
-      // PullCoreInfo and PushPullData to make these concerns only relevant where they are needed
-      String newMetadataSuffix = null;
-      String lastReadMetadataSuffix = null;
-      int version = -1;
-      
-      // if the versions are the same then the last read metadata suffix should be the same
-      if (v1.getZkVersion() == v2.getZkVersion()) {
-        assert v1.getLastReadMetadataSuffix().equals(v2.getLastReadMetadataSuffix());
-        lastReadMetadataSuffix = v1.getLastReadMetadataSuffix();
-        // this doesn't matter which structure it comes from
-        newMetadataSuffix = v1.getNewMetadataSuffix();
-        version = v1.getZkVersion();
-        
-        // if one needs to wait then merged will have to wait as well
-        waitForSearcher = v1.waitForSearcher || v2.waitForSearcher;
-        // if one wants to create core if absent then merged will have to create as well
-        createCoreIfAbsent = v1.createCoreIfAbsent || v2.createCoreIfAbsent;
-      } else if (v1.getZkVersion() > v2.getZkVersion()) {
-        // version number increments on updates so the higher version will result in a pull
-        // from the most up-to-date state on blob
-        lastReadMetadataSuffix = v1.getLastReadMetadataSuffix();
-        newMetadataSuffix = v1.getNewMetadataSuffix();
-        version = v1.getZkVersion();
-        
-        waitForSearcher = v1.waitForSearcher;
-        createCoreIfAbsent = v1.createCoreIfAbsent;
-      } else {
-        lastReadMetadataSuffix = v2.getLastReadMetadataSuffix();
-        newMetadataSuffix = v2.getNewMetadataSuffix();
-        version = v2.getZkVersion();
-        
-        waitForSearcher = v2.waitForSearcher;
-        createCoreIfAbsent = v2.createCoreIfAbsent;
-      }
-      
-      return new PullCoreInfo(v1.getCollectionName(), v1.getShardName(), v1.getCoreName(), 
-          v1.getSharedStoreName(), lastReadMetadataSuffix, newMetadataSuffix, version, 
-          createCoreIfAbsent, waitForSearcher); 
+      // if one needs to wait then merged will have to wait as well
+      waitForSearcher = v1.waitForSearcher || v2.waitForSearcher;
+      // if one wants to create core if absent then merged will have to create as well
+      createCoreIfAbsent = v1.createCoreIfAbsent || v2.createCoreIfAbsent;
+
+      return new PullCoreInfo(v1.getCollectionName(), v1.getShardName(), v1.getCoreName(),
+          v1.getSharedStoreName(), createCoreIfAbsent, waitForSearcher);
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
index ce4003e..d1ad641 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
@@ -16,11 +16,21 @@
  */
 package org.apache.solr.store.blob.process;
 
-import org.apache.solr.store.blob.util.DeduplicatingList;
+import java.lang.invoke.MethodHandles;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Throwables;
+import org.apache.solr.cloud.CloudDescriptor;
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.store.blob.util.DeduplicatingList;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.lang.invoke.MethodHandles;
 
 /**
  * A thread (there are a few of these created in {@link CorePullerFeeder#run}) that dequeues {@link CorePullTask} from a
@@ -47,9 +57,47 @@ public class CorePullerThread implements Runnable {
       try {
         // This call blocks if work queue is empty
         task = workQueue.removeFirst();
-        // TODO: we should timebox this request in case we are stuck for long time
-        task.pullCoreFromBlob();
 
+        CorePullerFeeder.PullCoreInfo info = task.getPullCoreInfo();
+        String collectionName = info.getCollectionName();
+        String shardName = info.getShardName();
+        String coreName = info.getCoreName();
+        SharedCoreConcurrencyController concurrencyController = task.getCoreContainer().getSharedStoreManager().getSharedCoreConcurrencyController();
+
+        SharedCoreVersionMetadata coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, coreName);
+        // TODO: On leaders we can rely on soft guarantee of equality since indexing can correct them if they are wrong
+        //       There is a work item to make use of that knowledge and not enqueue pulls on leaders when soft guarantee of equality
+        //       is available. But until that work item is done this a stop gap measure to dismiss those unnecessary pulls.
+        //       The reason it is important to add this:
+        //       - is to prevent those unnecessary pulls to contend on pull write lock and cause any trouble to indexing which needs read lock.
+        //       The reason it is considered a stop gap measure:
+        //       - we need not to enqueue pull requests to begin with
+        //       - this isLeader computation is not complete, it does not handle cases when core is absent
+        //       - this isLeader computation might not be performant and efficient. I don't know if caching is involved here or not.
+        boolean isLeaderPulling = isLeader(task.getCoreContainer(), coreName);
+        if (coreVersionMetadata.isSoftGuaranteeOfEquality() && isLeaderPulling) {
+          // already in sync
+          task.finishedPull(coreVersionMetadata.getBlobCoreMetadata(), CoreSyncStatus.SUCCESS_EQUIVALENT, null);
+          // move to next task
+          continue;
+        }
+
+        ReentrantReadWriteLock corePullLock = concurrencyController.getCorePullLock(info.getCollectionName(), info.getShardName(), info.getCoreName());
+        // Try to acquire write lock, if possible. otherwise, we don't want to hold this background thread
+        // because it can be used for other cores in the mean time
+        // Avoiding the barging overload #tryLock() to let default fairness scheme play out, although, the way
+        // we are using this lock barging overload would not disrupt things that much 
+        if (corePullLock.writeLock().tryLock(0, TimeUnit.MILLISECONDS)) {
+          try {
+            // TODO: we should timebox this request in case we are stuck for long time
+            task.pullCoreFromBlob(isLeaderPulling);
+          } finally {
+            corePullLock.writeLock().unlock();
+          }
+        } else {
+          log.info(String.format("Could not acquire pull write lock, going back to task queue, pullCoreInfo=%s", task.getPullCoreInfo().toString()));
+          workQueue.addDeduplicated(task, true);
+        }
       } catch (InterruptedException ie) {
         log.info("Puller thread " + Thread.currentThread().getName()
             + " got interrupted. Shutting down Blob CorePullerFeeder if not already.");
@@ -60,9 +108,42 @@ public class CorePullerThread implements Runnable {
         break;
       } catch (Exception e) {
         // Exceptions other than InterruptedException should not stop the business
-        String taskInfo = task == null ? "" : String.format("Attempt=%s to pull core %s ", task.getAttempts(), task.getPullCoreInfo().getSharedStoreName()) ;
+        String taskInfo = "";
+        if (task != null) {
+          try {
+            taskInfo = String.format("Attempt=%s to pull core %s ", task.getAttempts(), task.getPullCoreInfo().getCoreName());
+            task.finishedPull(null, CoreSyncStatus.FAILURE, Throwables.getStackTraceAsString(e));
+          } catch (Exception fpe) {
+            log.warn("Cleaning up of pull task encountered a failure.", fpe);
+          }
+        }
         log.warn("CorePullerThread encountered a failure. " + taskInfo, e);
       }
     }
   }
+
+  // TODO: This is temporary, see detailed note where it is consumed above.
+  private boolean isLeader(CoreContainer coreContainer, String coreName) throws InterruptedException {
+    if (!coreContainer.isZooKeeperAware()) {
+      // not solr cloud
+      return false;
+    }
+    CoreDescriptor coreDescriptor = coreContainer.getCoreDescriptor(coreName);
+    if (coreDescriptor == null) {
+      // core does not exist
+      return false;
+    }
+    CloudDescriptor cd = coreDescriptor.getCloudDescriptor();
+    if (cd == null || cd.getReplicaType() != Replica.Type.SHARED) {
+      // not a shared replica
+      return false;
+    }
+    ZkController zkController = coreContainer.getZkController();
+    Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(cd.getCollectionName(), cd.getShardId());
+    if (leaderReplica == null || !cd.getCoreNodeName().equals(leaderReplica.getName())) {
+      // not a leader replica
+      return false;
+    }
+    return true;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
index 1b6717d..97f51aa 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePusher.java
@@ -1,13 +1,19 @@
 package org.apache.solr.store.blob.process;
 
+import java.io.IOException;
 import java.lang.invoke.MethodHandles;
+import java.util.concurrent.locks.ReentrantLock;
 
+import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.DirectoryFactory;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
-import org.apache.solr.store.blob.client.BlobCoreMetadataBuilder;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.metadata.CorePushPull;
 import org.apache.solr.store.blob.metadata.PushPullData;
@@ -16,7 +22,11 @@ import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetadataResolutionResult;
 import org.apache.solr.store.blob.provider.BlobStorageProvider;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreStage;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -28,77 +38,162 @@ import org.slf4j.LoggerFactory;
  * changed locally and needs to be persisted to a shared store (blob store). 
  */
 public class CorePusher {
-  
+
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  
+
   private CoreContainer coreContainer;
-  
+
   public CorePusher(CoreContainer coreContainer) {
     this.coreContainer = coreContainer;
   }
-    
+
   /**
    * Pushes the local core updates to the Blob store and logs whether the push succeeded or failed.
    */
   public void pushCoreToBlob(PushPullData pushPullData) throws Exception {
-    BlobStorageProvider blobProvider = coreContainer.getSharedStoreManager().getBlobStorageProvider(); 
+    BlobStorageProvider blobProvider = coreContainer.getSharedStoreManager().getBlobStorageProvider();
     CoreStorageClient blobClient = blobProvider.getClient();
-    BlobDeleteManager deleteManager = coreContainer.getSharedStoreManager().getBlobDeleteManager(); // TODO, use a real client
-        
-    BlobCoreMetadata blobCoreMetadata = null;
-    log.info("Push to shared store initiating with PushPullData= " + pushPullData.toString());
-    
-    // Read the metadata file from shared store if this isn't the first push of this index shard
+    BlobDeleteManager deleteManager = coreContainer.getSharedStoreManager().getBlobDeleteManager();
     try {
-      if (!pushPullData.getLastReadMetadataSuffix().equals(
-          SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE)) {
-  
-        String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(pushPullData.getLastReadMetadataSuffix());
-        blobCoreMetadata = blobClient.pullCoreMetadata(pushPullData.getSharedStoreName(), blobCoreMetadataName);
-  
-        if (blobCoreMetadata == null) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "The shard index " + pushPullData.getSharedStoreName() + 
-              " is missing for shard " + pushPullData.getShardName() + " for collection " + pushPullData.getCollectionName() + 
-              " using metadataSuffix " + pushPullData.getLastReadMetadataSuffix());
+      String shardName = pushPullData.getShardName();
+      String collectionName = pushPullData.getCollectionName();
+      String coreName = pushPullData.getCoreName();
+      SharedCoreConcurrencyController concurrencyController = coreContainer.getSharedStoreManager().getSharedCoreConcurrencyController();
+      ReentrantLock corePushLock = concurrencyController.getCorePushLock(collectionName, shardName, coreName);
+      String snapshotDirPath = null;
+      // TODO: Timebox the following push logic, in case we are stuck for long time. We would also need to respect any
+      //       time constraints that comes with indexing request since we will be doing wasteful work if the client has already 
+      //       bailed on us. This might be better done as part of bigger work item where we spike out how to allocate/configure
+      //       time quotas in cooperation with client.
+      // acquire push lock to serialize blob updates so that concurrent updates can't race with each other 
+      // and cause push failures because one was able to advance zk to newer version.
+      //
+      // Only need to read this if we ever establish starvation on push lock and want to do something about it:
+      // First thing we do after acquiring the lock is to see if latest commit's generation is equal to what we pushed last.
+      // If equal then we have nothing to push, its an optimization that saves us from creating somewhat expensive ServerSideMetadata.
+      // That equality check can also be duplicated here before acquiring the push lock but that would just be a simple optimization.
+      // It will not save us from starvation. Because that condition being "true" also means that there is no active pusher,
+      // so there is no question of starvation.
+      // One option could be to snapshot a queue of pusher threads, we are working on behalf of,
+      // before we capture the commit point to push. Once finished pushing, we can dismiss all those threads together.
+      long startTimeMs = System.currentTimeMillis();
+      corePushLock.lock();
+      try {
+        long lockAcquisitionTime = System.currentTimeMillis() - startTimeMs;
+        SolrCore core = coreContainer.getCore(coreName);
+        if (core == null) {
+          throw new SolrException(ErrorCode.SERVER_ERROR, "Can't find core " + coreName);
+        }
+        try {
+          concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.BlobPushStarted);
+
+          IndexCommit latestCommit = core.getDeletionPolicy().getLatestCommit();
+          if (latestCommit == null) {
+            throw new SolrException(ErrorCode.SERVER_ERROR, "Core " + coreName + " has no available commit point");
+          }
+
+          SharedCoreVersionMetadata coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, coreName);
+          if (latestCommit.getGeneration() == coreVersionMetadata.getBlobCoreMetadata().getGeneration()) {
+            // 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.
+            // This optimization saves us from creating somewhat expensive ServerSideMetadata.
+            log.info(String.format("Nothing to push, pushLockTime=%s pushPullData=%s", lockAcquisitionTime, pushPullData.toString()));
+            return;
+          }
+
+          log.info("Push to shared store initiating with PushPullData= " + pushPullData.toString());
+          // Resolve the differences between the local shard index data and shard index data on shared store
+          // if there is any
+          ServerSideMetadata localCoreMetadata = new ServerSideMetadata(coreName, coreContainer, /* takeSnapshot */true);
+          snapshotDirPath = localCoreMetadata.getSnapshotDirPath();
+          SharedMetadataResolutionResult resolutionResult = SharedStoreResolutionUtil.resolveMetadata(
+              localCoreMetadata, coreVersionMetadata.getBlobCoreMetadata());
+
+          if (resolutionResult.getFilesToPush().isEmpty()) {
+            log.warn(String.format("Why there is nothing to push even when there is a newer commit point since last push," +
+                " pushLockTime=%s pushPullData=%s", lockAcquisitionTime, pushPullData.toString()));
+            return;
+          }
+
+          String newMetadataSuffix = BlobStoreUtils.generateMetadataSuffix();
+          // begin the push process 
+          CorePushPull pushPull = new CorePushPull(blobClient, deleteManager, pushPullData, resolutionResult, localCoreMetadata, coreVersionMetadata.getBlobCoreMetadata());
+          BlobCoreMetadata blobCoreMetadata = pushPull.pushToBlobStore(coreVersionMetadata.getMetadataSuffix(), newMetadataSuffix);
+          concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.BlobPushed);
+          // at this point we've pushed the new metadata file with the newMetadataSuffix and now need to write to zookeeper
+          SharedShardMetadataController shardSharedMetadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController();
+          SharedShardVersionMetadata newShardVersionMetadata = null;
+          try {
+            newShardVersionMetadata = shardSharedMetadataController.updateMetadataValueWithVersion(pushPullData.getCollectionName(), pushPullData.getShardName(),
+                newMetadataSuffix, coreVersionMetadata.getVersion());
+          } catch (Exception ex) {
+            boolean isVersionMismatch = false;
+            Throwable cause = ex;
+            while (cause != null) {
+              if (cause instanceof BadVersionException) {
+                isVersionMismatch = true;
+                break;
+              }
+              cause = cause.getCause();
+            }
+            if (isVersionMismatch) {
+              // conditional update of zookeeper failed, take away soft guarantee of equality.
+              // That will make sure before processing next indexing batch, we sync with zookeeper and pull from shared store.
+              concurrencyController.updateCoreVersionMetadata(collectionName, shardName, coreName, false);
+            }
+            throw ex;
+          }
+          concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.ZkUpdateFinished);
+
+          assert newMetadataSuffix.equals(newShardVersionMetadata.getMetadataSuffix());
+          // after successful update to zookeeper, update core version metadata with new version info
+          // and we can also give soft guarantee that core is up to date w.r.to shared store, until unless failures happen and leadership changes 
+          concurrencyController.updateCoreVersionMetadata(collectionName, shardName, coreName, newShardVersionMetadata, blobCoreMetadata, /* softGuaranteeOfEquality */ true);
+          concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.LocalCacheUpdateFinished);
+          log.info(String.format("Successfully pushed to shared store, pushLockTime=%s pushPullData=%s", lockAcquisitionTime, pushPullData.toString()));
+        } finally {
+          try {
+            concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.BlobPushFinished);
+            if (snapshotDirPath != null) {
+              // we are done with push we can now remove the snapshot directory
+              removeSnapshotDirectory(core, snapshotDirPath);
+            }
+          } finally {
+            core.close();
+          }
         }
-      } else {
-        blobCoreMetadata = BlobCoreMetadataBuilder.buildEmptyCoreMetadata(pushPullData.getSharedStoreName());
-        log.info("This is the first time that shard " + pushPullData.getShardName() + " for collection " + 
-            pushPullData.getCollectionName() + " is getting pushed to blob store.");
+      } finally {
+        corePushLock.unlock();
       }
-      
-      // Resolve the differences between the local shard index data and shard index data on shared store
-      // if there is any
-      ServerSideMetadata localShardMetadata = new ServerSideMetadata(pushPullData.getCoreName(), coreContainer);
-      SharedMetadataResolutionResult resolutionResult = SharedStoreResolutionUtil.resolveMetadata(
-          localShardMetadata, blobCoreMetadata);
-      
-      // begin the push process 
-      CorePushPull pushPull = new CorePushPull(blobClient, deleteManager, pushPullData, resolutionResult, localShardMetadata, blobCoreMetadata);
-      pushPull.pushToBlobStore();
-      
-      // at this point we've pushed the new metadata file with the newMetadataSuffix and now need to write to zookeeper
-      SharedShardMetadataController shardSharedMetadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController(); 
-      shardSharedMetadataController.updateMetadataValueWithVersion(pushPullData.getCollectionName(), pushPullData.getShardName(),
-          pushPullData.getNewMetadataSuffix(), pushPullData.getZkVersion());
-      log.info("Successfully pushed to shared store");
-      
       // TODO - make error handling a little nicer?
     } catch (InterruptedException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "CorePusher was interrupted while pushing to blob store", e);
     } catch (IndexNotFoundException infe) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "CorePusher failed because the core " + pushPullData.getCoreName() +
-        " for the shard " + pushPullData.getShardName() + " was not found", infe);
+          " for the shard " + pushPullData.getShardName() + " was not found", infe);
     } catch (SolrException e) {
       Throwable t = e.getCause();
       if (t instanceof BadVersionException) {
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "CorePusher failed to push because the node "
-            + "version doesn't match. This shard is no longer the leader.", t); 
+            + "version doesn't match.", t);
       }
       throw e;
     } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "CorePusher failed to push shard index for " 
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "CorePusher failed to push shard index for "
           + pushPullData.getShardName() + " due to unexpected exception", e);
     }
   }
+
+  private void removeSnapshotDirectory(SolrCore core, String snapshotDirPath) throws IOException {
+    Directory snapshotDir = core.getDirectoryFactory().get(snapshotDirPath, DirectoryFactory.DirContext.DEFAULT, core.getSolrConfig().indexConfig.lockType);
+    try {
+      core.getDirectoryFactory().doneWithDirectory(snapshotDir);
+      core.getDirectoryFactory().remove(snapshotDir);
+    } catch (Exception e) {
+      log.warn("Cannot remove snapshot directory " + snapshotDirPath, e);
+    } finally {
+      core.getDirectoryFactory().release(snapshotDir);
+    }
+  }
 }
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncStatus.java b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncStatus.java
index 960c726..4405e3a 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncStatus.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncStatus.java
@@ -50,8 +50,6 @@ public enum CoreSyncStatus {
   BLOB_DELETED_FOR_PULL(false, false),
   /** Core was not pushed because Blob version more up to date */
   BLOB_FRESHER(true, false),
-  /** No attempt to push/pull the core was made because another task was working on it */
-  CONCURRENT_SYNC(false, true),
   /** No attempt to push/pull the core was made as system was shutting down */
   SHUTTING_DOWN(false, false),
   /** The task was merged with another duplicate task in the queue (deduplicated) */
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CoreUpdateTracker.java b/solr/core/src/java/org/apache/solr/store/blob/process/CoreUpdateTracker.java
index 8b8105b..8ef502e 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CoreUpdateTracker.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CoreUpdateTracker.java
@@ -1,18 +1,14 @@
 package org.apache.solr.store.blob.process;
 
 import java.lang.invoke.MethodHandles;
-import java.util.Map;
 
-import org.apache.solr.client.solrj.cloud.autoscaling.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.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.store.blob.metadata.PushPullData;
-import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,19 +58,6 @@ public class CoreUpdateTracker {
           log.warn("Performing a push for shard " + shardName + " that is inactive!");
         }
         log.info("Initiating push for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-        // creates the metadata node if it doesn't exist
-        shardSharedMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-
-        /*
-         * Get the metadataSuffix value from ZooKeeper or from a cache if an entry exists for the 
-         * given collection and shardName. If the leader has already changed, the conditional update
-         * later will fail and invalidate the cache entry if it exists. 
-         */
-        VersionedData data = shardSharedMetadataController.readMetadataValue(collectionName, shardName, 
-            /* readFromCache */ true);
-
-        Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
-        String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
 
         String sharedShardName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
 
@@ -83,9 +66,6 @@ public class CoreUpdateTracker {
             .setShardName(shardName)
             .setCoreName(coreName)
             .setSharedStoreName(sharedShardName)
-            .setLastReadMetadataSuffix(metadataSuffix)
-            .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
-            .setZkVersion(data.getVersion())
             .build();
         CorePusher pusher = new CorePusher(coreContainer);
         pusher.pushCoreToBlob(pushPullData);
diff --git a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
index 4a428c2..86407c8 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/util/BlobStoreUtils.java
@@ -1,17 +1,15 @@
 package org.apache.solr.store.blob.util;
 import java.lang.invoke.MethodHandles;
-import java.util.Map;
 import java.util.UUID;
 
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.client.BlobCoreMetadataBuilder;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.metadata.CorePushPull;
 import org.apache.solr.store.blob.metadata.PushPullData;
@@ -19,7 +17,10 @@ import org.apache.solr.store.blob.metadata.ServerSideMetadata;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetadataResolutionResult;
 import org.apache.solr.store.blob.process.BlobDeleteManager;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreStage;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,76 +55,77 @@ public class BlobStoreUtils {
    * local content.
    * @throws SolrException if the local core was not successfully sync'd.
    */
-  public static void syncLocalCoreWithSharedStore(String collectionName, String coreName, String shardName, CoreContainer coreContainer) throws SolrException {
+  public static void syncLocalCoreWithSharedStore(String collectionName, String coreName, String shardName, CoreContainer coreContainer,
+                                                  SharedShardVersionMetadata shardVersionMetadata, boolean isLeaderPulling) throws SolrException {
     assert coreContainer.isZooKeeperAware();
 
     ZkController zkController = coreContainer.getZkController();
-    SharedShardMetadataController sharedMetadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController();
     DocCollection collection = zkController.getClusterState().getCollection(collectionName);
     CoreStorageClient blobClient = coreContainer.getSharedStoreManager().getBlobStorageProvider().getClient();
-    log.info("sync intialized for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-    
+    log.info("sync initialized for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
+
     Slice shard = collection.getSlicesMap().get(shardName);
     if (shard != null) {
       try {
-        sharedMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-        String sharedStoreName = (String)shard.get(ZkStateReader.SHARED_SHARD_NAME);
-        // Fetch the latest metadata from ZK.
-        // TODO: this can be optimized, depends on correct handling of leadership change.
-        VersionedData data = sharedMetadataController.readMetadataValue(collectionName, shardName, false);
-
-        Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
-        String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
-        if (SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE.equals(metadataSuffix)) {
+        String sharedStoreName = (String) shard.get(ZkStateReader.SHARED_SHARD_NAME);
+        SharedCoreConcurrencyController concurrencyController = coreContainer.getSharedStoreManager().getSharedCoreConcurrencyController();
+        if (SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE.equals(shardVersionMetadata.getMetadataSuffix())) {
+          //no-op pull
+          BlobCoreMetadata emptyBlobCoreMetadata = BlobCoreMetadataBuilder.buildEmptyCoreMetadata(sharedStoreName);
+          concurrencyController.updateCoreVersionMetadata(collectionName, shardName, coreName, shardVersionMetadata, emptyBlobCoreMetadata, isLeaderPulling);
           log.info("sync successful, nothing to pull, collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
-          return ;
+          return;
         }
-        // Get blob metadata
-        String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(metadataSuffix);
-        BlobCoreMetadata blobstoreMetadata = blobClient.pullCoreMetadata(sharedStoreName, blobCoreMetadataName);
-        if (null == blobstoreMetadata) {
-          // Zookepeer and blob are out of sync, could be due to eventual consistency model in blob or something else went wrong.
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              "cannot get core.metadata file from shared store, blobCoreMetadataName=" + blobCoreMetadataName +
-              " shard=" + shardName +
-              " collectionName=" + collectionName +
-              " sharedStoreName=" + sharedStoreName );
-        } else if (blobstoreMetadata.getIsDeleted()) {
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-              "core.metadata file is marked deleted in shared store, blobCoreMetadataName=" + blobCoreMetadataName +
-              " shard=" + shardName +
-              " collectionName=" + collectionName +
-              " sharedStoreName=" + sharedStoreName );
-        } else if (blobstoreMetadata.getIsCorrupt()) {
-          log.warn("core.Metadata file is marked corrpt, skipping sync, collection=" + collectionName + 
-              " shard=" + shardName + " coreName=" + coreName + " sharedStoreName=" + sharedStoreName );
-          return ;
-        }
-
-        // Get local metadata + resolve with blob metadata
-        ServerSideMetadata serverMetadata = new ServerSideMetadata(coreName, coreContainer);
-        SharedMetadataResolutionResult resolutionResult = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobstoreMetadata);
-        PushPullData pushPullData = new PushPullData.Builder()
-            .setCollectionName(collectionName)
-            .setShardName(shardName)
-            .setCoreName(coreName)
-            .setSharedStoreName(sharedStoreName)
-            .setLastReadMetadataSuffix(metadataSuffix)
-            .setNewMetadataSuffix(BlobStoreUtils.generateMetadataSuffix())
-            .setZkVersion(data.getVersion())
-            .build();
+        concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.BlobPullStarted);
+        try {
+          // Get blob metadata
+          String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(shardVersionMetadata.getMetadataSuffix());
+          BlobCoreMetadata blobstoreMetadata = blobClient.pullCoreMetadata(sharedStoreName, blobCoreMetadataName);
+          if (null == blobstoreMetadata) {
+            // Zookepeer and blob are out of sync, could be due to eventual consistency model in blob or something else went wrong.
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "cannot get core.metadata file from shared store, blobCoreMetadataName=" + blobCoreMetadataName +
+                    " shard=" + shardName +
+                    " collectionName=" + collectionName +
+                    " sharedStoreName=" + sharedStoreName);
+          } else if (blobstoreMetadata.getIsDeleted()) {
+            throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                "core.metadata file is marked deleted in shared store, blobCoreMetadataName=" + blobCoreMetadataName +
+                    " shard=" + shardName +
+                    " collectionName=" + collectionName +
+                    " sharedStoreName=" + sharedStoreName);
+          } else if (blobstoreMetadata.getIsCorrupt()) {
+            log.warn("core.Metadata file is marked corrpt, skipping sync, collection=" + collectionName +
+                " shard=" + shardName + " coreName=" + coreName + " sharedStoreName=" + sharedStoreName);
+            return;
+          }
 
-        if (resolutionResult.getFilesToPull().size() > 0) {
+          // Get local metadata + resolve with blob metadata
+          ServerSideMetadata serverMetadata = new ServerSideMetadata(coreName, coreContainer);
+          SharedMetadataResolutionResult resolutionResult = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobstoreMetadata);
+          PushPullData pushPullData = new PushPullData.Builder()
+              .setCollectionName(collectionName)
+              .setShardName(shardName)
+              .setCoreName(coreName)
+              .setSharedStoreName(sharedStoreName)
+              .build();
 
-          BlobDeleteManager deleteManager = coreContainer.getSharedStoreManager().getBlobDeleteManager();
-          CorePushPull cp = new CorePushPull(blobClient, deleteManager, pushPullData, resolutionResult, serverMetadata, blobstoreMetadata);
-          cp.pullUpdateFromBlob(/* waitForSearcher */ true);
-        } else {
-          log.info("sync successful, nothing to pull for collection=" + collectionName + " shard=" + shardName + " coreName=" + coreName);
+          if (resolutionResult.getFilesToPull().size() > 0) {
+            BlobDeleteManager deleteManager = coreContainer.getSharedStoreManager().getBlobDeleteManager();
+            CorePushPull cp = new CorePushPull(blobClient, deleteManager, pushPullData, resolutionResult, serverMetadata, blobstoreMetadata);
+            cp.pullUpdateFromBlob(/* waitForSearcher */ true);
+            concurrencyController.updateCoreVersionMetadata(pushPullData.getCollectionName(), pushPullData.getShardName(), pushPullData.getCoreName(),
+                shardVersionMetadata, blobstoreMetadata, isLeaderPulling);
+          } else {
+            log.warn(String.format("Why there are no files to pull even when we do not match with the version in zk? collection=%s shard=%s core=%s",
+                collectionName, shardName, coreName));
+          }
+        } finally {
+          concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.BlobPullFinished);
         }
       } catch (Exception ex) {
         // wrap every thrown exception in a solr exception
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error occured pulling shard=" + shardName + " collection=" + collectionName + " from shared store "+ ex);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error occured pulling shard=" + shardName + " collection=" + collectionName + " from shared store " + ex);
       }
     } else {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Sync requested for unknown shard=" + shardName + " in collection=" + collectionName);
diff --git a/solr/core/src/java/org/apache/solr/store/shared/SharedCoreConcurrencyController.java b/solr/core/src/java/org/apache/solr/store/shared/SharedCoreConcurrencyController.java
new file mode 100644
index 0000000..9a9d52e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/store/shared/SharedCoreConcurrencyController.java
@@ -0,0 +1,351 @@
+/*
+ * 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.store.shared;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.StringUtils;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.metadata.PushPullData;
+import org.apache.solr.store.blob.process.CorePullTask;
+import org.apache.solr.store.blob.process.CorePullerThread;
+import org.apache.solr.store.blob.process.CorePusher;
+import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * This class helps coordinate synchronization of concurrent indexing, pushes and pulls
+ * happening on a core of a shared collection {@link DocCollection#getSharedIndex()}
+ */
+public class SharedCoreConcurrencyController {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  /**
+   * Time indexing thread needs to wait to try acquiring pull write lock before checking if someone else has already done the pull.
+   */
+  public static int SECONDS_TO_WAIT_INDEXING_PULL_WRITE_LOCK = 5;
+  /**
+   * Max attempts by indexing thread to try acquiring pull write lock before bailing out. Ideally bail out scenario should never happen.
+   * If it does then either we are too slow in pulling and can tune this value or something else is wrong.
+   */
+  public static int MAX_ATTEMPTS_INDEXING_PULL_WRITE_LOCK = 10;
+
+  private final CoreContainer cores;
+  /**
+   * This cache maintains the shared store version the each core is at or ahead of(core has to sometimes be ahead of
+   * shared store given indexing first happens locally before being propagated to shared store).
+   * todo: need to add eviction strategy.
+   */
+  private final ConcurrentHashMap<String, SharedCoreVersionMetadata> coresVersionMetadata;
+
+  public SharedCoreConcurrencyController(CoreContainer cores) {
+    this.cores = cores;
+    coresVersionMetadata = new ConcurrentHashMap<>();
+  }
+
+  /**
+   * Returns true if {@link SharedCoreVersionMetadata} and {@link SharedShardVersionMetadata} represent the same version; otherwise false
+   */
+  public boolean areVersionsEqual(SharedCoreVersionMetadata coreVersionMetadata, SharedShardVersionMetadata shardVersionMetadata) {
+    boolean isVersionNumberSame = coreVersionMetadata.getVersion() == shardVersionMetadata.getVersion();
+    boolean isMetadataSuffixSame = StringUtils.equals(coreVersionMetadata.getMetadataSuffix(), shardVersionMetadata.getMetadataSuffix());
+
+    if (isVersionNumberSame && isMetadataSuffixSame) {
+      return true;
+    }
+    if (isVersionNumberSame || isMetadataSuffixSame) {
+      log.warn(String.format("Why only one of version number and metadata suffix matches?" +
+              " coreVersionNumber=%s shardVersionNumber=%s" +
+              " coreMetadataSuffix=%s shardMetadataSuffix=%s",
+          coreVersionMetadata.getVersion(), shardVersionMetadata.getVersion(),
+          coreVersionMetadata.getMetadataSuffix(), shardVersionMetadata.getMetadataSuffix()));
+    }
+    return false;
+  }
+
+  /**
+   * Logs the current {@link SharedCoreStage} a core is at.
+   */
+  public void recordState(String collectionName, String shardName, String coreName, SharedCoreStage stage) {
+    log.info(String.format("RecordSharedCoreStage: collection=%s shard=%s core=%s stage=%s", collectionName, shardName, coreName, stage));
+  }
+
+  /**
+   * Returns a {@link ReentrantReadWriteLock} corresponding to the core. It protects pulls from each other and indexing from pulls.
+   * A write lock is required whenever pulling contents into a core from shared store.
+   * A read lock is required for the whole duration of indexing on the core(including the push to shared store {@link CorePusher#pushCoreToBlob(PushPullData)}.)
+   */
+  public ReentrantReadWriteLock getCorePullLock(String collectionName, String shardName, String coreName) {
+    SharedCoreVersionMetadata coreVersionMetadata = getOrCreateCoreVersionMetadata(collectionName, shardName, coreName);
+    return coreVersionMetadata.getCorePullLock();
+  }
+
+  /**
+   * Returns a {@link ReentrantLock} corresponding to the core. It protects shared store pushes from each other.
+   * This lock is required for pushing the core to shared store {@link CorePusher#pushCoreToBlob(PushPullData)}.
+   */
+  public ReentrantLock getCorePushLock(String collectionName, String shardName, String coreName) {
+    SharedCoreVersionMetadata coreVersionMetadata = getOrCreateCoreVersionMetadata(collectionName, shardName, coreName);
+    return coreVersionMetadata.getCorePushLock();
+  }
+
+  /**
+   * Returns {@link SharedCoreVersionMetadata} representing shared store version the core is at or ahead of (core has
+   * to sometimes be ahead of shared store given indexing first happens locally before being propagated to shared store).
+   */
+  public SharedCoreVersionMetadata getCoreVersionMetadata(String collectionName, String shardName, String coreName) {
+    return getOrCreateCoreVersionMetadata(collectionName, shardName, coreName);
+  }
+
+  /**
+   * Updates {@link SharedCoreVersionMetadata} for the core with passed in
+   * {@link SharedShardVersionMetadata} and {@link BlobCoreMetadata}
+   */
+  public void updateCoreVersionMetadata(String collectionName, String shardName, String coreName,
+                                        SharedShardVersionMetadata shardVersionMetadata, BlobCoreMetadata blobCoreMetadata) {
+    updateCoreVersionMetadata(collectionName, shardName, coreName, shardVersionMetadata, blobCoreMetadata, false);
+  }
+
+  /**
+   * Updates {@link SharedCoreVersionMetadata} for the core with passed in
+   * {@link SharedShardVersionMetadata}, {@link BlobCoreMetadata} and {@code softGuaranteeOfEquality}
+   */
+  public void updateCoreVersionMetadata(String collectionName, String shardName, String coreName,
+                                        SharedShardVersionMetadata shardVersionMetadata, BlobCoreMetadata blobCoreMetadata,
+                                        boolean softGuaranteeOfEquality) {
+    SharedCoreVersionMetadata currentMetadata = getOrCreateCoreVersionMetadata(collectionName, shardName, coreName);
+    SharedCoreVersionMetadata updatedMetadata = currentMetadata.updatedOf(shardVersionMetadata.getVersion(), shardVersionMetadata.getMetadataSuffix(),
+        blobCoreMetadata, softGuaranteeOfEquality);
+    updateCoreVersionMetadata(collectionName, shardName, coreName, currentMetadata, updatedMetadata);
+  }
+
+  /**
+   * Updates {@link SharedCoreVersionMetadata} for the core with passed in
+   * {@code softGuaranteeOfEquality}
+   */
+  public void updateCoreVersionMetadata(String collectionName, String shardName, String coreName, boolean softGuaranteeOfEquality) {
+    SharedCoreVersionMetadata currentMetadata = getOrCreateCoreVersionMetadata(collectionName, shardName, coreName);
+    SharedCoreVersionMetadata updatedMetadata = currentMetadata.updatedOf(softGuaranteeOfEquality);
+    updateCoreVersionMetadata(collectionName, shardName, coreName, currentMetadata, updatedMetadata);
+  }
+
+  private void updateCoreVersionMetadata(String collectionName, String shardName, String coreName, SharedCoreVersionMetadata currentMetadata, SharedCoreVersionMetadata updatedMetadata) {
+    log.info(String.format("updateCoreVersionMetadata: collection=%s shard=%s core=%s  current={%s} updated={%s}",
+        collectionName, shardName, coreName, currentMetadata.toString(), updatedMetadata.toString()));
+    coresVersionMetadata.put(coreName, updatedMetadata);
+  }
+
+  private SharedCoreVersionMetadata getOrCreateCoreVersionMetadata(String collectionName, String shardName, String coreName) {
+    SharedCoreVersionMetadata coreVersionMetadata = coresVersionMetadata.get(coreName);
+    if (coreVersionMetadata != null) {
+      // already present
+      return coreVersionMetadata;
+    }
+    return initializeCoreVersionMetadata(collectionName, shardName, coreName);
+  }
+
+  private SharedCoreVersionMetadata initializeCoreVersionMetadata(String collectionName, String shardName, String coreName) {
+    // computeIfAbsent to ensure we only do single initialization
+    return coresVersionMetadata.computeIfAbsent(coreName, k -> {
+      ensureShardVersionMetadataNodeExists(collectionName, shardName);
+      // a value not to be found as a zk node version
+      int version = -1;
+      String metadataSuffix = null;
+      BlobCoreMetadata blobCoreMetadata = null;
+      boolean softGuaranteeOfEquality = false;
+      /** Should only be created once at initialization time, subsequent updates should reuse same lock instance
+       *  see {@link SharedCoreVersionMetadata#updatedOf(int, String, BlobCoreMetadata, boolean)} and other overload.
+       *
+       *  We don't need fair ordering policy for this lock, which normally has lower throughput.
+       *  We rely on softGuaranteeOfEquality 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. see  {@link CorePullerThread#run()} for details.
+       *
+       *  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 shared 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.
+       *  */
+      ReentrantReadWriteLock corePullLock = new ReentrantReadWriteLock();
+      /** Should only be created once at initialization time, subsequent updates should reuse same lock instance
+       *  see {@link SharedCoreVersionMetadata#updatedOf(int, String, BlobCoreMetadata, boolean)} and other overload
+       *
+       *  We don't need fair ordering policy for this lock, which normally has lower throughput.
+       *  see {@link CorePusher#pushCoreToBlob(PushPullData)} for details */
+      ReentrantLock corePushLock = new ReentrantLock();
+      return new SharedCoreVersionMetadata(version, metadataSuffix, blobCoreMetadata, softGuaranteeOfEquality, corePullLock, corePushLock);
+    });
+  }
+
+  @VisibleForTesting
+  protected void ensureShardVersionMetadataNodeExists(String collectionName, String shardName) {
+    SharedShardMetadataController metadataController = cores.getSharedStoreManager().getSharedShardMetadataController();
+    try {
+      // creates the metadata node if it doesn't exist
+      metadataController.ensureMetadataNodeExists(collectionName, shardName);
+    } catch (IOException ioe) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          String.format("Unable to ensure metadata for collection=%s shard=%s", collectionName, shardName), ioe);
+    }
+  }
+
+  /**
+   * This represents metadata that need to be cached for a core of a shared collection {@link DocCollection#getSharedIndex()}
+   * so that it can be properly synchronized for concurrent indexing, pushes and pulls.
+   * <p>
+   * The locks only need to be initialized once and can potentially be kept in a separate structure from version information. For that
+   * we would need to pay cost for another map in {@link SharedCoreConcurrencyController}. To avoid another map we are keeping locks here
+   * and making sure they are not re-initialized.
+   */
+  public static class SharedCoreVersionMetadata {
+    /**
+     * Value originating from a ZooKeeper node used to handle conditionally and safely update the
+     * core.metadata file written to the shared store.
+     */
+    private final int version;
+    /**
+     * Unique value representing the state of shared store with which the core is at least sync with.
+     */
+    private final String metadataSuffix;
+    /**
+     * {@link BlobCoreMetadata} representing the state corresponding to {@link #metadataSuffix}
+     */
+    private final BlobCoreMetadata blobCoreMetadata;
+    /**
+     * Whether there is a soft guarantee of being in sync with {@link SharedShardVersionMetadata} of the shard.
+     * In steady state this guarantee is provided for leader cores when they push
+     * {@link CorePusher#pushCoreToBlob(PushPullData)}
+     * and pull
+     * {@link BlobStoreUtils#syncLocalCoreWithSharedStore(String, String, String, CoreContainer, SharedShardVersionMetadata, boolean)}
+     * {@link CorePullTask#pullCoreFromBlob()}
+     * since followers cannot index. In presence of this guarantee we can skip consulting zookeeper before processing an indexing batch.
+     */
+    private final boolean softGuaranteeOfEquality;
+    /**
+     * See comments on {@link #getCorePullLock(String, String, String)}
+     */
+    private final ReentrantReadWriteLock corePullLock;
+    /**
+     * See comments on {@link #getCorePushLock(String, String, String)}
+     */
+    private final ReentrantLock corePushLock;
+
+    private SharedCoreVersionMetadata(int version, String metadataSuffix, BlobCoreMetadata blobCoreMetadata,
+                                      boolean softGuaranteeOfEquality, ReentrantReadWriteLock corePullLock, ReentrantLock corePushLock) {
+      this.version = version;
+      this.metadataSuffix = metadataSuffix;
+      this.blobCoreMetadata = blobCoreMetadata;
+      this.softGuaranteeOfEquality = softGuaranteeOfEquality;
+      this.corePullLock = corePullLock;
+      this.corePushLock = corePushLock;
+    }
+
+    public int getVersion() {
+      return version;
+    }
+
+    public BlobCoreMetadata getBlobCoreMetadata() {
+      return blobCoreMetadata;
+    }
+
+    public String getMetadataSuffix() {
+      return metadataSuffix;
+    }
+
+    public boolean isSoftGuaranteeOfEquality() {
+      return softGuaranteeOfEquality;
+    }
+
+    private ReentrantReadWriteLock getCorePullLock() {
+      return corePullLock;
+    }
+
+    private ReentrantLock getCorePushLock() {
+      return corePushLock;
+    }
+
+    private SharedCoreVersionMetadata updatedOf(int version, String metadataSuffix, BlobCoreMetadata blobCoreMetadata, boolean softGuaranteeOfEquality) {
+      return new SharedCoreVersionMetadata(version, metadataSuffix, blobCoreMetadata, softGuaranteeOfEquality, corePullLock, corePushLock);
+    }
+
+    private SharedCoreVersionMetadata updatedOf(boolean softGuaranteeOfEquality) {
+      return new SharedCoreVersionMetadata(version, metadataSuffix, blobCoreMetadata, softGuaranteeOfEquality, corePullLock, corePushLock);
+    }
+
+    @Override
+    public String toString() {
+      return String.format("version=%s  metadataSuffix=%s softGuaranteeOfEquality=%s", version, metadataSuffix, softGuaranteeOfEquality);
+    }
+  }
+
+  /**
+   * Various stages a core of a shared collection {@link DocCollection#getSharedIndex()} might go through during indexing and querying.
+   */
+  public enum SharedCoreStage {
+    /**
+     * Necessary locks have been acquired and we have started to pull from the shared store.
+     */
+    BlobPullStarted,
+    /**
+     * Pull(either successful or failed) has ended and we are about to release the necessary locks.
+     */
+    BlobPullFinished,
+    /**
+     * We have received an indexing batch but necessary locks have not been acquired yet.
+     */
+    IndexingBatchReceived,
+    /**
+     * We are passed the shared store pull stage(if applicable) and are in sync with shared store.
+     * Now we will proceed with local indexing.
+     */
+    LocalIndexingStarted,
+    /**
+     * Local indexing finished but not have been pushed to shared store.
+     */
+    LocalIndexingFinished,
+    /**
+     * Necessary locks have been acquired and push to shared store has started.
+     */
+    BlobPushStarted,
+    /**
+     * Files have been pushed to blob.
+     */
+    BlobPushed,
+    /**
+     * Zookeeper has been successfully updated with new metadata.
+     */
+    ZkUpdateFinished,
+    /**
+     * Local cache {@link #coresVersionMetadata} has been successfully updated with new metadata.
+     */
+    LocalCacheUpdateFinished,
+    /**
+     * Push(either successful or failed) has ended and we are about to release the necessary locks.
+     */
+    BlobPushFinished
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
index 8a17f2b..16a66b7 100644
--- a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
+++ b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
@@ -16,6 +16,7 @@
  */
 package org.apache.solr.store.shared;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.store.blob.metadata.BlobCoreSyncer;
 import org.apache.solr.store.blob.process.BlobDeleteManager;
@@ -24,8 +25,6 @@ import org.apache.solr.store.blob.process.CorePullTracker;
 import org.apache.solr.store.blob.provider.BlobStorageProvider;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Provides access to Shared Store processes. Note that this class is meant to be 
  * more generic in the future and provide a cleaner API but for now we'll expose
@@ -40,12 +39,14 @@ public class SharedStoreManager {
   private BlobProcessUtil blobProcessUtil;
   private CorePullTracker corePullTracker;
   private BlobCoreSyncer blobCoreSyncer;
-  
+  private SharedCoreConcurrencyController sharedCoreConcurrencyController;
+
   public SharedStoreManager(ZkController controller) {
     zkController = controller;
     // initialize BlobProcessUtil with the SharedStoreManager for background processes to be ready
     blobProcessUtil = new BlobProcessUtil(zkController.getCoreContainer());
     blobCoreSyncer = new BlobCoreSyncer();
+    sharedCoreConcurrencyController = new SharedCoreConcurrencyController(zkController.getCoreContainer());
   }
   
   @VisibleForTesting
@@ -113,4 +114,14 @@ public class SharedStoreManager {
   public BlobCoreSyncer getBlobCoreSyncer() {
     return blobCoreSyncer;
   }
+
+  public SharedCoreConcurrencyController getSharedCoreConcurrencyController() {
+    return sharedCoreConcurrencyController;
+  }
+
+  @VisibleForTesting
+  public void initConcurrencyController(SharedCoreConcurrencyController concurrencyController) {
+    this.sharedCoreConcurrencyController = concurrencyController;
+  }
+
 }
\ No newline at end of file
diff --git a/solr/core/src/java/org/apache/solr/store/shared/metadata/SharedShardMetadataController.java b/solr/core/src/java/org/apache/solr/store/shared/metadata/SharedShardMetadataController.java
index 9629e54..0b48a52 100644
--- a/solr/core/src/java/org/apache/solr/store/shared/metadata/SharedShardMetadataController.java
+++ b/solr/core/src/java/org/apache/solr/store/shared/metadata/SharedShardMetadataController.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
@@ -35,8 +34,6 @@ import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 
-import com.google.common.annotations.VisibleForTesting;
-
 /**
  * Class that manages metadata for shared index-based collections in Solr Cloud and 
  * ZooKeeper.
@@ -48,23 +45,10 @@ public class SharedShardMetadataController {
   
   private SolrCloudManager cloudManager;
   private DistribStateManager stateManager;
-  /* 
-   * Naive in-memory cache without any cache eviction logic used to cache zk version values.
-   * TODO - convert to a more memory efficient and intelligent caching strategy. 
-   */ 
-  private ConcurrentHashMap<String, VersionedData> cache;
-  
+
   public SharedShardMetadataController(SolrCloudManager cloudManager) {
     this.cloudManager = cloudManager;
     this.stateManager = cloudManager.getDistribStateManager();
-    cache = new ConcurrentHashMap<>();
-  }
-  
-  @VisibleForTesting
-  public SharedShardMetadataController(SolrCloudManager cloudManager, ConcurrentHashMap<String, VersionedData> cache) {
-    this.cloudManager = cloudManager;
-    this.stateManager = cloudManager.getDistribStateManager();
-    this.cache = cache;
   }
   
   /**
@@ -90,10 +74,8 @@ public class SharedShardMetadataController {
   }
 
   /**
-   * If the update is successful, the VersionedData will contain the new version as well as the 
-   * value of the data just written. Successful updates will cache the new VersionedData while 
-   * unsuccesful ones will invalidate any existing entries for the corresponding collectionName,
-   * shardName combination.
+   * If the update is successful, the returned {@link SharedShardVersionMetadata} will contain the new version as well as the 
+   * value of the data just written. 
    * 
    * Specify version to be -1 to skip the node version check before update. 
    * 
@@ -102,25 +84,23 @@ public class SharedShardMetadataController {
    * @param value the value to be written to ZooKeeper
    * @param version the ZooKeeper node version to conditionally update on
    */
-  public VersionedData updateMetadataValueWithVersion(String collectionName, String shardName, String value, int version) {
+  public SharedShardVersionMetadata 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 = stateManager.setAndGetResult(metadataPath, Utils.toJSON(nodeProps), version);
-      cache.put(getCacheKey(collectionName, shardName), data);
-      return data;
+      Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
+      String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
+      return new SharedShardVersionMetadata(data.getVersion(), metadataSuffix);
     } catch (BadVersionException e) {
-      cache.remove(getCacheKey(collectionName, shardName));
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error updating path: " + metadataPath
           + " due to mismatching versions", e);
     } catch (IOException | NoSuchElementException | KeeperException e) {
-      cache.remove(getCacheKey(collectionName, shardName));
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error updating path: " + metadataPath
           + " in ZooKeeper", e);
     } catch (InterruptedException e) {
-      cache.remove(getCacheKey(collectionName, shardName));
       Thread.currentThread().interrupt();
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error updating path: " + metadataPath
           + " in ZooKeeper due to interruption", e);
@@ -128,37 +108,18 @@ public class SharedShardMetadataController {
   }
   
   /**
-   * The returned VersionedData will contain the version of the node as well as the contents of the node.
-   * The data content of VersionedData will be a byte array that needs to be converted into a {@link Map}.
-   * 
-   * If readFromCache is true, we'll attempt to read from an in-memory cache the VersionedData based on 
-   * the collectionName and shardName and return that if it exists. There is no gaurantee this cache
-   * entry is not stale.
-   * 
-   * @param collectionName name of the collection being updated
-   * @param shardName name of the shard that owns the metadataSuffix node
-   */
-  public VersionedData readMetadataValue(String collectionName, String shardName, boolean readfromCache) throws SolrException {
-    if (readfromCache) {
-      VersionedData cachedEntry = cache.get(getCacheKey(collectionName, shardName));
-      if (cachedEntry != null) {
-        return cachedEntry;
-      }
-    }
-    return readMetadataValue(collectionName, shardName);
-  }
-  
-  /**
-   * The returned VersionedData will contain the version of the node as well as the contents of the node.
-   * The data content of VersionedData will be a byte array that needs to be converted into a {@link Map}.
+   * Reads the {@link SharedShardVersionMetadata} for the shard from zookeeper. 
    * 
-   * @param collectionName name of the collection being updated
+   * @param collectionName name of the shared collection
    * @param shardName name of the shard that owns the metadataSuffix node
    */
-  public VersionedData readMetadataValue(String collectionName, String shardName) {
+  public SharedShardVersionMetadata readMetadataValue(String collectionName, String shardName) {
     String metadataPath = getMetadataBasePath(collectionName, shardName) + "/" + SUFFIX_NODE_NAME;
     try {
-      return stateManager.getData(metadataPath, null);      
+      VersionedData data = stateManager.getData(metadataPath, null);
+      Map<String, String> nodeUserData = (Map<String, String>) Utils.fromJSON(data.getData());
+      String metadataSuffix = nodeUserData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
+      return  new SharedShardVersionMetadata(data.getVersion(), metadataSuffix);
     } catch (IOException | NoSuchElementException | KeeperException e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error reading data from path: " + metadataPath
           + " in ZooKeeper", e);
@@ -182,21 +143,9 @@ public class SharedShardMetadataController {
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error deleting path " + 
           metadataPath + " in Zookeeper", e);
-    } finally {
-      cache.clear();
     }
   }
-  
-  /**
-   * Clears any cached version value if it exists for the corresponding collection and shard
-   * 
-   * @param collectionName name of the collection being updated
-   * @param shardName name of the shard that owns the metadataSuffix node
-   */
-  public void clearCachedVersion(String collectionName, String shardName) {
-    cache.remove(getCacheKey(collectionName, shardName));
-  }
-  
+
   private void createPersistentNodeIfNonExistent(String path, byte[] data) {
     try {
       if (!stateManager.hasData(path)) {
@@ -219,13 +168,31 @@ public class SharedShardMetadataController {
   protected String getMetadataBasePath(String collectionName, String shardName) {
     return ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName + "/" + ZkStateReader.SHARD_LEADERS_ZKNODE + "/" + shardName;
   }
-  
-  protected String getCacheKey(String collectionName, String shardName) {
-    return collectionName + "_" + shardName;
-  }
-  
-  @VisibleForTesting
-  protected ConcurrentHashMap<String, VersionedData> getVersionedDataCache() {
-    return cache;
+
+  /**
+   * This represents correctness metadata for a shard of a shared collection {@link DocCollection#getSharedIndex()}
+   */
+  public static class SharedShardVersionMetadata {
+    /**
+     * version of zookeeper node maintaining the metadata
+     */
+    private final int version;
+    /**
+     * Unique value of the metadataSuffix for the last persisted shard index in the shared store.
+     */
+    private final String metadataSuffix;
+
+    public SharedShardVersionMetadata(int version, String metadataSuffix) {
+      this.version = version;
+      this.metadataSuffix = metadataSuffix;
+    }
+
+    public int getVersion() {
+      return version;
+    }
+
+    public String getMetadataSuffix() {
+      return metadataSuffix;
+    }
   }
 }
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 9b44ef4..7a47936 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
@@ -17,8 +17,6 @@
 
 package org.apache.solr.update.processor;
 
-import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
-
 import java.io.IOException;
 import java.lang.invoke.MethodHandles;
 import java.util.ArrayList;
@@ -32,7 +30,9 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.cloud.CloudDescriptor;
 import org.apache.solr.cloud.Overseer;
@@ -60,8 +60,15 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.store.blob.metadata.PushPullData;
+import org.apache.solr.store.blob.process.CorePusher;
 import org.apache.solr.store.blob.process.CoreUpdateTracker;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreStage;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.DeleteUpdateCommand;
@@ -75,7 +82,7 @@ import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.annotations.VisibleForTesting;
+import static org.apache.solr.update.processor.DistributingUpdateProcessorFactory.DISTRIB_UPDATE_PARAM;
 
 public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
 
@@ -87,6 +94,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
   private final String collection;
   private boolean readOnlyCollection = false;
   private CoreUpdateTracker sharedCoreTracker;
+  private ReentrantReadWriteLock corePullLock;
 
   // The cached immutable clusterState for the update... usually refreshed for each individual update.
   // Different parts of this class used to request current clusterState views, which lead to subtle bugs and race conditions
@@ -209,9 +217,6 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     } else {
       // zk
       ModifiableSolrParams params = new ModifiableSolrParams(filterParams(req.getParams()));
-      if (replicaType.equals(Replica.Type.SHARED)) {
-        readFromSharedStoreIfNecessary();
-      }
 
       List<SolrCmdDistributor.Node> useNodes = null;
       if (req.getParams().get(COMMIT_END_POINT) == null) {
@@ -1055,6 +1060,10 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
 
   @Override
   protected void doClose() {
+    if (corePullLock != null) {
+      // release read lock
+      corePullLock.readLock().unlock();
+    }
     if (cmdDistrib != null) {
       cmdDistrib.close();
     }
@@ -1081,30 +1090,131 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
     super.processRollback(cmd);
   }
 
-  private void writeToShareStore() throws SolrException {
-    log.info("Attempting to initiate index update write to shared store for collection=" + cloudDesc.getCollectionName() +
-        " and shard=" + cloudDesc.getShardId() + " using core=" + req.getCore().getName());
+
+  private void writeToSharedStore() {
+    String collectionName = cloudDesc.getCollectionName();
+    String shardName = cloudDesc.getShardId();
+    String coreName = req.getCore().getName();
+    SharedCoreConcurrencyController concurrencyController = req.getCore().getCoreContainer().getSharedStoreManager().getSharedCoreConcurrencyController();
+    concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.LocalIndexingFinished);
+
+    log.info("Attempting to initiate index update write to shared store for collection=" + collectionName +
+        " and shard=" + shardName + " using core=" + coreName);
 
     sharedCoreTracker.persistShardIndexToSharedStore(zkController.zkStateReader.getClusterState(),
-        cloudDesc.getCollectionName(),
-        cloudDesc.getShardId(),
-        req.getCore().getName());
+        collectionName,
+        shardName,
+        coreName);
   }
 
-  private void readFromSharedStoreIfNecessary() throws SolrException {
-    String coreName = req.getCore().getName();
-    String shardName = cloudDesc.getShardId();
+  private void readFromSharedStoreIfNecessary() {
     String collectionName = cloudDesc.getCollectionName();
+    String shardName = cloudDesc.getShardId();
+    String coreName = req.getCore().getName();
     assert Replica.Type.SHARED.equals(replicaType);
     // Peers and subShardLeaders should only forward the update request to leader replica,
     // hence not need to sync with the blob store at this point.
     if (!isLeader || isSubShardLeader) {
       return;
     }
-    BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName,coreName,shardName,req.getCore().getCoreContainer());
-  }
 
+    // this lock acquire/release logic is built on the assumption that one particular instance of this processor
+    // will solely be consumed by a single thread.
+    // Following pull logic should only run once before the first document of indexing batch(add/delete) is processed by this processor
+    if (corePullLock != null) {
+      // we already have a lock i.e. we have already read from the shared store (if needed)
+      return;
+    }
 
+    CoreContainer coreContainer = req.getCore().getCoreContainer();
+    SharedCoreConcurrencyController concurrencyController = coreContainer.getSharedStoreManager().getSharedCoreConcurrencyController();
+    concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.IndexingBatchReceived);
+    corePullLock = concurrencyController.getCorePullLock(collectionName, shardName, coreName);
+    // acquire lock for the whole duration of update
+    // it will be release in close method
+    corePullLock.readLock().lock();
+    // from this point on wards we should always exit this method with read lock (no matter failure or what)
+
+    SharedCoreVersionMetadata coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, coreName);
+    /**
+     * we only need to sync if there is no soft guarantee of being in sync.
+     * if there is one we will rely on that, and if we turned out to be wrong indexing will fail at push time
+     * and will remove this guarantee in {@link CorePusher#pushCoreToBlob(PushPullData)}
+     */
+    if (!coreVersionMetadata.isSoftGuaranteeOfEquality()) {
+      SharedShardMetadataController metadataController = coreContainer.getSharedStoreManager().getSharedShardMetadataController();
+      SharedShardVersionMetadata shardVersionMetadata = metadataController.readMetadataValue(collectionName, shardName);
+      if (!concurrencyController.areVersionsEqual(coreVersionMetadata, shardVersionMetadata)) {
+        // we need to pull before indexing therefore we need to upgrade to write lock
+        // we have to release read lock before we can acquire write lock
+        corePullLock.readLock().unlock();
+        boolean reacquireReadLock = true;
+        try {
+          // There is a likelihood that many indexing requests came at once and realized we are out of sync.
+          // They all would try to acquire write lock. One of them makes progress to pull from shared store.
+          // After that regular indexing will see soft guarantee of equality and moves straight to indexing
+          // under read lock. Now it is possible that new indexing keeps coming in and read lock is never free.
+          // In that case the poor guys that came in earlier and wanted to pull will still be struggling(starving) to
+          // acquire write lock. Since we know that write lock is only needed by one to do the work, we will
+          // try time boxed acquisition and in case of failed acquisition we will see if some one else has already completed the pull.
+          // We will make few attempts before we bail out. Ideally bail out scenario should never happen.
+          // If it does then either we are too slow in pulling and can tune following parameters or something else is wrong.
+          int attempt = 1;
+          while (true) {
+            try {
+              // try acquiring write lock
+              if (corePullLock.writeLock().tryLock(SharedCoreConcurrencyController.SECONDS_TO_WAIT_INDEXING_PULL_WRITE_LOCK, TimeUnit.SECONDS)) {
+                try {
+                  // in between upgrading locks things might have updated, should reestablish if pull is still needed
+                  coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, coreName);
+                  if (!coreVersionMetadata.isSoftGuaranteeOfEquality()) {
+                    shardVersionMetadata = metadataController.readMetadataValue(collectionName, shardName);
+                    if (!concurrencyController.areVersionsEqual(coreVersionMetadata, shardVersionMetadata)) {
+                      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, coreName, shardName, coreContainer, shardVersionMetadata, /* isLeaderSyncing */true);
+                    }
+                  }
+                  // reacquire read lock for the remainder of indexing before releasing write lock that was acquired for pull part
+                  corePullLock.readLock().lock();
+                  reacquireReadLock = false;
+                } finally {
+                  corePullLock.writeLock().unlock();
+                }
+                // write lock acquisition was successful and we are in sync with shared store
+                break;
+              } else {
+                // we could not acquire write lock but see if some other thread has already done the pulling
+                coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, coreName);
+                if (coreVersionMetadata.isSoftGuaranteeOfEquality()) {
+                  log.info(String.format("Indexing thread waited to acquire to write lock and could not. " +
+                          "But someone else has done the pulling so we are good. attempt=%s collection=%s shard=%s core=%s",
+                      attempt, collectionName, shardName, coreName));
+                  break;
+                }
+                // no one else has pulled yet either, lets make another attempt ourselves
+                attempt++;
+                if (attempt > SharedCoreConcurrencyController.MAX_ATTEMPTS_INDEXING_PULL_WRITE_LOCK) {
+                  throw new SolrException(ErrorCode.SERVER_ERROR, String.format("Indexing thread failed to acquire write lock for pull in %s seconds. " +
+                          "And no one else either has done the pull during that time. collection=%s shard=%s core=%s",
+                      Integer.toString(SharedCoreConcurrencyController.SECONDS_TO_WAIT_INDEXING_PULL_WRITE_LOCK * SharedCoreConcurrencyController.MAX_ATTEMPTS_INDEXING_PULL_WRITE_LOCK),
+                      collectionName, shardName, coreName));
+                }
+              }
+            } catch (InterruptedException ie) {
+              Thread.currentThread().interrupt();
+              throw new SolrException(ErrorCode.SERVER_ERROR, String.format("Indexing thread interrupted while trying to acquire pull write lock." +
+                  " collection=%s shard=%s core=%s", collectionName, shardName, coreName), ie);
+            }
+          }
+        } finally {
+          // we should always leave with read lock acquired(failure or success), since it is the job of close method to release it
+          if (reacquireReadLock) {
+            corePullLock.readLock().lock();
+          }
+        }
+      }
+    }
+    concurrencyController.recordState(collectionName, shardName, coreName, SharedCoreStage.LocalIndexingStarted);
+  }
 
   // TODO: optionally fail if n replicas are not reached...
   protected void doDistribFinish() {
@@ -1138,7 +1248,7 @@ public class DistributedZkUpdateProcessor extends DistributedUpdateProcessor {
        * what the DUP is concerned about so we may want to consider moving this code somewhere more appropriate
        * in the future (deeper in the stack)
        */
-      writeToShareStore();
+      writeToSharedStore();
     }
 
     // TODO: if not a forward and replication req is not specified, we could
diff --git a/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java b/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
index e29bad9..9a58ebd 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
@@ -147,7 +147,7 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
           // ensure we count down for all replicas per slice.
           String sharedShardName = (String) slice.getProperties().get(ZkStateReader.SHARED_SHARD_NAME);
           solrProcessesTaskTracker.get(replica.getNodeName())
-            .put(sharedShardName, cdl);
+            .put(replica.getCoreName(), cdl);
           SolrClient replicaClient = getHttpSolrClient(replica.getBaseUrl() + "/" + replica.getCoreName());
           try {
             replicaClient.query(params("q", "*:* priming pull", "distrib", "false"));
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
index 9ccbf39..de31b62 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/CorePushPullTest.java
@@ -24,6 +24,7 @@ import java.util.stream.Collectors;
 import org.apache.commons.io.FileUtils;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.cloud.api.collections.Assign;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
@@ -33,6 +34,10 @@ import org.apache.solr.store.blob.client.LocalStorageClient;
 import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetadataResolutionResult;
 import org.apache.solr.store.blob.process.BlobDeleteManager;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -95,7 +100,7 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
     
     // verify an exception is thrown
     try {
-      pushPull.pushToBlobStore();
+      pushPull.pushToBlobStore(null, null);
       fail("pushToBlobStore should have thrown an exception");
     } catch (Exception ex) {
       // core missing from core container should throw exception
@@ -262,8 +267,20 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
   }
 
   private BlobCoreMetadata doPush(SolrCore core) throws Exception {
+    String sharedBlobName = Assign.buildSharedShardName(collectionName, shardName);
+    // initialize metadata info to match initial zk version
+    SharedCoreConcurrencyController concurrencyController =  new SharedCoreConcurrencyController(null){
+      @Override
+      protected void ensureShardVersionMetadataNodeExists(String collectionName, String shardName) {
+        
+      }
+    };
+    concurrencyController.updateCoreVersionMetadata(collectionName, shardName, core.getName(), 
+        new SharedShardVersionMetadata(0, SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE),
+        BlobCoreMetadataBuilder.buildEmptyCoreMetadata(sharedBlobName));
+
     // build the require metadata
-    ServerSideMetadata solrServerMetadata = new ServerSideMetadata(core.getName(), h.getCoreContainer());
+    ServerSideMetadata solrServerMetadata = new ServerSideMetadata(core.getName(), h.getCoreContainer(), /* takeSnapshot */ true);
     
     // empty bcm means we should push everything we have locally 
     BlobCoreMetadata bcm = BlobCoreMetadataBuilder.buildEmptyCoreMetadata(sharedBlobName);
@@ -274,9 +291,6 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
         .setShardName(shardName)
         .setCoreName(core.getName())
         .setSharedStoreName(sharedBlobName)
-        .setLastReadMetadataSuffix(metadataSuffix)
-        .setNewMetadataSuffix(randomSuffix)
-        .setZkVersion(1)
         .build();
     SharedMetadataResolutionResult resResult = SharedStoreResolutionUtil.resolveMetadata(solrServerMetadata, bcm);
     
@@ -288,22 +302,19 @@ public class CorePushPullTest extends SolrTestCaseJ4 {
         return;
       }
     };
-    return pushPull.pushToBlobStore();
+    SharedCoreVersionMetadata coreVersionMetadata = concurrencyController.getCoreVersionMetadata(collectionName, shardName, core.getName());
+    return pushPull.pushToBlobStore(coreVersionMetadata.getMetadataSuffix(), randomSuffix);
   }
   
   private SharedMetadataResolutionResult doPull(SolrCore core, BlobCoreMetadata bcm) throws Exception {
     // build the require metadata
     ServerSideMetadata solrServerMetadata = new ServerSideMetadata(core.getName(), h.getCoreContainer());
     
-    String randomSuffix = BlobStoreUtils.generateMetadataSuffix();
     PushPullData ppd = new PushPullData.Builder()
         .setCollectionName(collectionName)
         .setShardName(shardName)
         .setCoreName(core.getName())
         .setSharedStoreName(sharedBlobName)
-        .setLastReadMetadataSuffix(metadataSuffix)
-        .setNewMetadataSuffix(randomSuffix)
-        .setZkVersion(1)
         .build();
     SharedMetadataResolutionResult resResult = SharedStoreResolutionUtil.resolveMetadata(solrServerMetadata, bcm);
     
diff --git a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
index d145d7c..c4b69d9 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/metadata/SharedStoreResolutionUtilTest.java
@@ -16,15 +16,13 @@
  */
 package org.apache.solr.store.blob.metadata;
 
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Set;
 import java.util.stream.Collectors;
 
+import com.google.common.collect.ImmutableSet;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
@@ -35,7 +33,8 @@ import org.apache.solr.store.blob.metadata.SharedStoreResolutionUtil.SharedMetad
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.ImmutableSet;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 /**
  * Unit tests for {@link SharedStoreResolutionUtil}.
@@ -71,7 +70,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
       
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Collections.emptySet(), false);
+        SharedMetadataResolutionResult(Collections.emptySet(), Collections.emptySet(), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(null, blobMetadata);
@@ -94,7 +93,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
       
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(serverMetadata.getLatestCommitFiles(), Collections.emptySet(), false);
+        SharedMetadataResolutionResult(serverMetadata.getLatestCommitFiles(), Collections.emptySet(), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, null);
@@ -117,7 +116,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
 
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), false);
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(null, blobMetadata);
@@ -182,7 +181,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Collections.emptySet(), false);
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Collections.emptySet(), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -212,7 +211,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(expectedFileToPull), false);
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(expectedFileToPull), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -245,7 +244,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
     
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new 
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(expectedFileToPull), false);
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(expectedFileToPull), Collections.emptySet(), false);
     
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -276,7 +275,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
 
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), true);
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), Collections.emptySet(), true);
 
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -303,7 +302,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
 
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new
-        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), true);
+        SharedMetadataResolutionResult(Collections.emptySet(), Arrays.asList(blobMetadata.getBlobFiles()), Collections.emptySet(), true);
 
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
@@ -335,7 +334,7 @@ public class SharedStoreResolutionUtilTest extends SolrTestCaseJ4 {
 
     // expected resolution
     SharedMetadataResolutionResult expectedResult = new
-        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), true);
+        SharedMetadataResolutionResult(Arrays.asList(expectedFileToPush), Arrays.asList(blobMetadata.getBlobFiles()), Collections.emptySet(), true);
 
     // do resolution
     SharedMetadataResolutionResult actual = SharedStoreResolutionUtil.resolveMetadata(serverMetadata, blobMetadata);
diff --git a/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java b/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
index 9f6d129..1bfcb55 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
@@ -16,10 +16,7 @@
  */
 package org.apache.solr.store.blob.process;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
+import com.google.common.collect.Sets;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.process.CorePullTask.PullCoreCallback;
 import org.apache.solr.store.blob.process.CorePullTask.PullTaskMerger;
@@ -28,8 +25,8 @@ import org.apache.solr.store.blob.process.CorePullerFeeder.PullCoreInfoMerger;
 import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.junit.Test;
 
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
 
 /**
  * Tests for {@link DeduplicatingList} using {@link PullTaskMerger} and
@@ -37,9 +34,6 @@ import com.google.common.collect.Sets;
  */
 public class PullMergeDeduplicationTest {
   
-  // not relevant in pulls but value required
-  private static final String NEW_METADATA_SUFFIX = "undefined";
-  
   private final String COLLECTION_NAME_1 = "collection1";
   private final String COLLECTION_NAME_2 = "collection2";
   private final String SHARD_NAME = "_shard_";
@@ -55,69 +49,28 @@ public class PullMergeDeduplicationTest {
     PullCoreInfoMerger merger = new CorePullerFeeder.PullCoreInfoMerger();
     
     // merging two exactly the same PCI should return exactly the same
-    String lastReadMetadataSuffix = "randomString";
-    PullCoreInfo pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
-    PullCoreInfo pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
+    PullCoreInfo pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, true, true);
+    PullCoreInfo pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, true, true);
     
     PullCoreInfo mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
     assertPullCoreInfo(mergedPullCoreInfo, pullCoreInfo2); 
     
     // boolean flags on merge flip
-    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, true);
-    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, false, true);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, true, false);
     
     mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
     assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
+        getTestPullCoreInfo(COLLECTION_NAME_1, true, true), 
         mergedPullCoreInfo);
     
     // boolean flags on merge flip
-    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
-    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, true);
-    
-    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
-    assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
-        mergedPullCoreInfo);
-    
-    // ensure the booleans are merged correctly when versions are equivalent 
-    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
-    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
-    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
-    assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false), 
-        mergedPullCoreInfo);
-    
-    // somehow if the versions are the same but lastReadMetadataSuffix are not, we fail 
-    // due to programming error
-    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        "notRandomString", NEW_METADATA_SUFFIX, 5, false, false);
-    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, false );
-    try {
-      merger.merge(pullCoreInfo1, pullCoreInfo2);
-      fail();
-    } catch (Throwable e) {
-      // success
-    }
-    
-    // higher version wins 
-    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        "differentSuffix", NEW_METADATA_SUFFIX, 6, true, true);
-    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, false);
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, true, false);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, false, true);
     
     mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
     assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, "differentSuffix", NEW_METADATA_SUFFIX, 6, true, true), 
+        getTestPullCoreInfo(COLLECTION_NAME_1, true, true), 
         mergedPullCoreInfo);
   }
   
@@ -130,9 +83,7 @@ public class PullMergeDeduplicationTest {
     DeduplicatingList<String, PullCoreInfo> dedupList = 
         new DeduplicatingList<>(10, new CorePullerFeeder.PullCoreInfoMerger());
     
-    String lastReadMetadataSuffix = "randomString";
-    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
+    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, true, true);
     dedupList.addDeduplicated(pullCoreInfo, false);
     assertEquals(1, dedupList.size());
     
@@ -141,31 +92,29 @@ public class PullMergeDeduplicationTest {
     assertEquals(1, dedupList.size());
     
     assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
+        getTestPullCoreInfo(COLLECTION_NAME_1, true, true), 
         dedupList.removeFirst());
     
     // add a different PCI that should still be merged
     dedupList.addDeduplicated(pullCoreInfo, false);
     
-    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, 
-        "differentString", NEW_METADATA_SUFFIX, 6, false, false);
+    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, false, false);
     dedupList.addDeduplicated(pullCoreInfo, false);
     
     assertEquals(1, dedupList.size());
     assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, "differentString", NEW_METADATA_SUFFIX, 6, false, false), 
+        getTestPullCoreInfo(COLLECTION_NAME_1, true, true), 
         dedupList.removeFirst());
     
     dedupList.addDeduplicated(pullCoreInfo, false);
-    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_2, 
-        "differentString", NEW_METADATA_SUFFIX, 6, false, false);
+    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_2, false, false);
     // add a different PCI that should not be merged
     dedupList.addDeduplicated(pullCoreInfo, false);
     assertEquals(2, dedupList.size());
     
     // check the integrity of the first 
     assertPullCoreInfo(
-        getTestPullCoreInfo(COLLECTION_NAME_1, "differentString", NEW_METADATA_SUFFIX, 6, false, false), 
+        getTestPullCoreInfo(COLLECTION_NAME_1, false, false), 
         dedupList.removeFirst());
   }
   
@@ -240,21 +189,18 @@ public class PullMergeDeduplicationTest {
   private CorePullTask getTestCorePullTask(String dedupKey, long queuedTimeMs, int attempts, long lastAttemptTimestamp, PullCoreCallback callback) {
     // assume PullCoreInfo merging works correctly, just create the same static one per CorePullTask
     // with the given dedup key
-    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(dedupKey, 
-        "randomString", NEW_METADATA_SUFFIX, 5, true, true);
+    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(dedupKey, true, true);
     
     CorePullTask pullTask = new CorePullTask(/* coreContainer */ null, pullCoreInfo, queuedTimeMs,
-        attempts, lastAttemptTimestamp, /* callback */ callback, Maps.newHashMap(), Sets.newHashSet());
+        attempts, lastAttemptTimestamp, /* callback */ callback, Sets.newHashSet());
     return pullTask;
   }
   
-  private PullCoreInfo getTestPullCoreInfo(String collectionName, String lastReadMetadataSuffix,
-      String newMetadataSuffix, int version, boolean createCoreIfAbsent, boolean waitForSearcher) {
+  private PullCoreInfo getTestPullCoreInfo(String collectionName, boolean createCoreIfAbsent, boolean waitForSearcher) {
     return new PullCoreInfo(collectionName, 
         collectionName + SHARD_NAME,
         collectionName + CORE_NAME,
         collectionName + SHARED_NAME,
-        lastReadMetadataSuffix, newMetadataSuffix, version, 
         createCoreIfAbsent, waitForSearcher);
   }
   
@@ -271,8 +217,6 @@ public class PullMergeDeduplicationTest {
     assertEquals("PullCoreInfos core names do not match", expected.getCoreName(), actual.getCoreName());
     assertEquals("PullCoreInfos shard names do not match", expected.getShardName(), actual.getShardName());
     assertEquals("PullCoreInfos sharedStore names do not match", expected.getSharedStoreName(), actual.getSharedStoreName());
-    assertEquals("PullCoreInfos lastReadMetadataSuffix do not match", expected.getLastReadMetadataSuffix(), actual.getLastReadMetadataSuffix());
-    assertEquals("PullCoreInfos zkVersions do not match", expected.getZkVersion(), actual.getZkVersion());
     assertEquals("PullCoreInfos shouldCreateCoreifAbsent flag do not match", expected.shouldCreateCoreIfAbsent(), actual.shouldCreateCoreIfAbsent());
     assertEquals("PullCoreInfos shouldWaitForSearcher flag do not match", expected.shouldWaitForSearcher(), actual.shouldWaitForSearcher());
   }
diff --git a/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java b/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
index 3cb3c86..33fa342 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/util/BlobStoreUtilsTest.java
@@ -17,10 +17,6 @@
 
 package org.apache.solr.store.blob.util;
 
-import static org.mockito.ArgumentMatchers.anyString;
-import static org.mockito.Mockito.never;
-import static org.mockito.Mockito.verify;
-
 import java.nio.file.Path;
 import java.util.UUID;
 
@@ -38,11 +34,16 @@ import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.shared.SolrCloudSharedStoreTestCase;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.verify;
+
 /**
  * Unit tests for {@link BlobStoreUtils}
  */
@@ -91,7 +92,8 @@ public class BlobStoreUtilsTest extends SolrCloudSharedStoreTestCase {
     
     CoreStorageClient blobClientSpy = Mockito.spy(storageClient);    
     try {
-      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc);
+      SharedShardVersionMetadata shardVersionMetadata = new SharedShardVersionMetadata(0, SharedShardMetadataController.METADATA_NODE_DEFAULT_VALUE);
+      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc, shardVersionMetadata, true);
       verify(blobClientSpy, never()).pullCoreMetadata(anyString(), anyString());
     } catch (Exception ex){
       fail("syncLocalCoreWithSharedStore failed with exception: " + ex.getMessage());
@@ -116,11 +118,9 @@ public class BlobStoreUtilsTest extends SolrCloudSharedStoreTestCase {
     newReplica = collection.getReplicas().get(0);
     cc = getCoreContainer(newReplica.getNodeName());
     
-    SharedShardMetadataController sharedMetadataController = cc.getSharedStoreManager().getSharedShardMetadataController();
-    sharedMetadataController.ensureMetadataNodeExists(collectionName, shardName);
-    sharedMetadataController.updateMetadataValueWithVersion(collectionName, shardName, UUID.randomUUID().toString(), -1);
     try {
-      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc);
+      SharedShardVersionMetadata shardVersionMetadata = new SharedShardVersionMetadata(0, UUID.randomUUID().toString());
+      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc, shardVersionMetadata, true);
       fail("syncLocalCoreWithSharedStore should throw exception if shared store doesn't have the core.metadata file.");
     } catch (Exception ex){
       String expectedException = "cannot get core.metadata file from shared store";
@@ -156,8 +156,10 @@ public class BlobStoreUtilsTest extends SolrCloudSharedStoreTestCase {
     req.add(doc);
     req.commit(cloudClient, collectionName);
     try {
+      SharedShardMetadataController metadataController = cc.getSharedStoreManager().getSharedShardMetadataController();
+      SharedShardVersionMetadata shardVersionMetadata = metadataController.readMetadataValue(collectionName, shardName);
       // we push and already have the latest updates so we should not pull here
-      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc);
+      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, newReplica.getCoreName(), shardName, cc, shardVersionMetadata, true);
       verify(blobClientSpy, never()).pullCoreMetadata(anyString(), anyString());
     } catch (Exception ex) { 
       fail("syncLocalCoreWithSharedStore failed with exception: " + ex.getMessage());
@@ -209,8 +211,10 @@ public class BlobStoreUtilsTest extends SolrCloudSharedStoreTestCase {
     assertEquals(1, core.getDeletionPolicy().getLatestCommit().getFileNames().size());
     
     try {
+      SharedShardMetadataController metadataController = cc.getSharedStoreManager().getSharedShardMetadataController();
+      SharedShardVersionMetadata shardVersionMetadata = metadataController.readMetadataValue(collectionName, shardName);
       // we pushed on the leader, try sync on the follower
-      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, follower.getCoreName(), shardName, cc);
+      BlobStoreUtils.syncLocalCoreWithSharedStore(collectionName, follower.getCoreName(), shardName, cc, shardVersionMetadata, true);
       
       // did we pull?
       assertTrue(core.getDeletionPolicy().getLatestCommit().getFileNames().size() > 1);
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SharedCoreConcurrencyTest.java b/solr/core/src/test/org/apache/solr/store/shared/SharedCoreConcurrencyTest.java
new file mode 100644
index 0000000..3bce31e
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/store/shared/SharedCoreConcurrencyTest.java
@@ -0,0 +1,488 @@
+/*
+ * 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.store.shared;
+
+import java.lang.invoke.MethodHandles;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.client.CoreStorageClient;
+import org.apache.solr.store.blob.process.BlobProcessUtil;
+import org.apache.solr.store.blob.process.CorePullTask;
+import org.apache.solr.store.blob.process.CorePullerFeeder;
+import org.apache.solr.store.blob.process.CoreSyncStatus;
+import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreStage;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests around synchronization of concurrent indexing, pushes and pulls
+ * happening on a core of a shared collection {@link DocCollection#getSharedIndex()}
+ * todo: add tests for failover scenarios and involve query pulls 
+ */
+public class SharedCoreConcurrencyTest extends SolrCloudSharedStoreTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static Path sharedStoreRootPath;
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    sharedStoreRootPath = createTempDir("tempDir");
+  }
+
+  /**
+   * Tests issuing random number of concurrent indexing requests in a given range for a shared core and making sure they all succeed.
+   */
+  @Test
+  public void testHighConcurrentIndexing() throws Exception {
+    int maxIndexingThreads = 100;
+    int maxDocsPerThread = 100;
+    testConcurrentIndexing(maxIndexingThreads, maxDocsPerThread);
+  }
+
+  /**
+   * Test ensuring two indexing requests interleave in desired ways and succeed.
+   */
+  //  @Test todo: leaking threads, test only issue
+  public void TODOtestPossibleInterleaving() throws Exception {
+
+    // completely serialized
+    testConcurrentIndexing(
+        new IndexingThreadInterleaver(
+            SharedCoreStage.BlobPushFinished,
+            SharedCoreStage.LocalIndexingStarted,
+            true));
+
+    // second pushes for first one too
+    testConcurrentIndexing(
+        new IndexingThreadInterleaver(
+            SharedCoreStage.LocalIndexingFinished, 
+            SharedCoreStage.BlobPushFinished,
+            true));
+  }
+
+  /**
+   * Test ensuring two indexing requests do not interleave in impossible ways even when forced and still succeed.
+   */
+  //  @Test todo: leaking threads, test only issue
+  public void TODOtestImpossibleInterleaving() throws Exception {
+    // push critical section
+    testConcurrentIndexing(
+        new IndexingThreadInterleaver(
+            SharedCoreStage.ZkUpdateFinished,
+            SharedCoreStage.ZkUpdateFinished,
+            false));
+
+    // another push critical section
+    testConcurrentIndexing(
+        new IndexingThreadInterleaver(
+            SharedCoreStage.ZkUpdateFinished,
+            SharedCoreStage.LocalCacheUpdateFinished,
+            false));
+  }
+
+  private void testConcurrentIndexing(int maxIndexingThreads, int maxDocsPerThread) throws Exception {
+    int numIndexingThreads = new Random().nextInt(maxIndexingThreads) + 1;;
+    testConcurrentIndexing(numIndexingThreads, maxDocsPerThread, null);
+  }
+
+  private void testConcurrentIndexing(IndexingThreadInterleaver interleaver) throws Exception {
+    testConcurrentIndexing(2, 10, interleaver);
+  }
+
+  /**
+   * Start desired number of concurrent indexing threads with each indexing random number of docs between 1 and maxDocsPerThread.
+   *
+   * At the end it verifies everything got indexed successfully on leader. No critical section got breached.
+   * Also verify the integrity of shared store contents by pulling them on a follower replica.
+   */
+  private void testConcurrentIndexing(int numIndexingThreads, int maxDocsPerThread, IndexingThreadInterleaver interleaver) throws Exception {
+    setupCluster(2);
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+
+    // this map tracks the async pull queues per solr process
+    Map<String, Map<String, CountDownLatch>> solrProcessesTaskTracker = new HashMap<>();
+
+    JettySolrRunner solrProcess1 = cluster.getJettySolrRunner(0);
+    CoreStorageClient storageClient1 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
+    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient1), solrProcess1);
+    Map<String, CountDownLatch> asyncPullLatches1 = configureTestBlobProcessForNode(solrProcess1);
+
+    JettySolrRunner solrProcess2 = cluster.getJettySolrRunner(1);
+    CoreStorageClient storageClient2 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
+    setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient2), solrProcess2);
+    Map<String, CountDownLatch> asyncPullLatches2 = configureTestBlobProcessForNode(solrProcess2);
+
+    solrProcessesTaskTracker.put(solrProcess1.getNodeName(), asyncPullLatches1);
+    solrProcessesTaskTracker.put(solrProcess2.getNodeName(), asyncPullLatches2);
+
+    String collectionName = "sharedCollection";
+    int maxShardsPerNode = 1;
+    int numReplicas = 2;
+    // specify a comma-delimited string of shard names for multiple shards when using
+    // an implicit router
+    String shardNames = "shard1";
+    setupSharedCollectionWithShardNames(collectionName, maxShardsPerNode, numReplicas, shardNames);
+
+    // get the leader replica and follower replicas
+    DocCollection collection = cloudClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    Replica shardLeaderReplica = collection.getLeader("shard1");
+    Replica followerReplica = null;
+    for (Replica repl : collection.getSlice("shard1").getReplicas()) {
+      if (repl.getName() != shardLeaderReplica.getName()) {
+        followerReplica = repl;
+        break;
+      }
+    }
+
+    
+   JettySolrRunner shardLeaderSolrRunner = null;
+   for (JettySolrRunner solrRunner : cluster.getJettySolrRunners()) {
+     if(solrRunner.getNodeName().equals(shardLeaderReplica.getNodeName())){
+       shardLeaderSolrRunner = solrRunner;
+       break;
+     }
+   }
+   List<String> progress = new ArrayList<>();
+
+   if(interleaver != null) {
+     configureTestSharedConcurrencyControllerForNode(shardLeaderSolrRunner, progress, interleaver);
+   } else {
+     configureTestSharedConcurrencyControllerForNode(shardLeaderSolrRunner, progress);
+   }
+
+    AtomicInteger totalDocs= new AtomicInteger(0);
+    log.info("numIndexingThreads=" + numIndexingThreads);
+    Thread[] indexingThreads = new Thread[numIndexingThreads];
+    ConcurrentLinkedQueue<String> indexingErrors = new ConcurrentLinkedQueue<>();
+    for (int i = 0; i < numIndexingThreads; i++) {
+      indexingThreads[i] = new Thread(() -> {
+        try {
+          // index between 1 to maxDocsPerThread docs
+          int numDocs = new Random().nextInt(maxDocsPerThread) + 1;
+          log.info("numDocs=" + numDocs);
+          UpdateRequest updateReq = new UpdateRequest();
+          for (int j = 0; j < numDocs; j++) {
+            int docId = totalDocs.incrementAndGet();
+            updateReq.add("id", Integer.toString(docId));
+          }
+          updateReq.commit(cloudClient, collectionName);
+        } catch (Exception ex) {
+          indexingErrors.add(ex.getMessage());
+        }
+      });
+    }
+
+    for (int i = 0; i < numIndexingThreads; i++) {
+      indexingThreads[i].start();
+    }
+
+    for (int i = 0; i < numIndexingThreads; i++) {
+      indexingThreads[i].join();
+    }
+
+    log.info("totalDocs=" + totalDocs.intValue());
+
+    assertTrue(indexingErrors.toString(), indexingErrors.isEmpty());
+
+    if(interleaver != null) {
+      assertNull(interleaver.error, interleaver.error);
+    }
+
+    assertFalse("no progress recorded", progress.isEmpty());
+
+    log.info(progress.toString());
+
+    assertCriticalSections(progress);
+
+    // verify the update wasn't forwarded to the follower and it didn't commit by checking the core
+    // this gives us confidence that the subsequent query we do triggers the pull
+    CoreContainer replicaCC = getCoreContainer(followerReplica.getNodeName());
+    SolrCore core = null;
+    SolrClient followerDirectClient = null;
+    SolrClient leaderDirectClient = null;
+    try {
+      core = replicaCC.getCore(followerReplica.getCoreName());
+      // the follower should only have the default segments file
+      assertEquals(1, core.getDeletionPolicy().getLatestCommit().getFileNames().size());
+
+      // query the leader directly to verify it should have the document
+      leaderDirectClient = getHttpSolrClient(shardLeaderReplica.getBaseUrl() + "/" + shardLeaderReplica.getCoreName());
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params
+          .set("q", "*:*")
+          .set("distrib", "false");
+      QueryResponse resp = leaderDirectClient.query(params);
+      assertEquals(totalDocs.intValue(), resp.getResults().getNumFound());
+
+      // we want to wait until the pull completes so set up a count down latch for the follower's
+      // core that we'll wait until pull finishes for
+      CountDownLatch latch = new CountDownLatch(1);
+      Map<String, CountDownLatch> asyncPullTasks = solrProcessesTaskTracker.get(followerReplica.getNodeName());
+      asyncPullTasks.put(followerReplica.getCoreName(), latch);
+
+      // query the follower directly to trigger the pull, this query should yield no results
+      // as it returns immediately 
+      followerDirectClient = getHttpSolrClient(followerReplica.getBaseUrl() + "/" + followerReplica.getCoreName());
+      resp = followerDirectClient.query(params);
+      assertEquals(0, resp.getResults().getNumFound());
+
+      // wait until pull is finished
+      assertTrue(latch.await(120, TimeUnit.SECONDS));
+
+      // do another query to verify we've pulled everything
+      resp = followerDirectClient.query(params);
+
+      // verify we pulled
+      assertTrue(core.getDeletionPolicy().getLatestCommit().getFileNames().size() > 1);
+
+      // verify the document is present
+      assertEquals(totalDocs.intValue(), resp.getResults().getNumFound());
+    } finally {
+      if (leaderDirectClient != null) {
+        leaderDirectClient.close();
+      }
+      if (followerDirectClient != null) {
+        followerDirectClient.close();
+      }
+      if (core != null) {
+        core.close();
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+      // clean up the shared store. The temp dir should clean up itself after the test class finishes
+      FileUtils.cleanDirectory(sharedStoreRootPath.toFile());
+    }
+  }
+
+  private void assertCriticalSections(List<String> progress) {
+    String currentThreadId = null;
+    SharedCoreStage currentStage = null;
+    String prevThreadId = null;
+    SharedCoreStage prevStage = null;
+    int activeIndexers = 0; // number of threads that have started indexing and not finished 
+    int activeBlobPushers = 0; // number of threads that are actively pushing at any given time
+    for (String p : progress) {
+      String[] parts = p.split("\\.");
+      currentThreadId = parts[0];
+      currentStage = SharedCoreStage.valueOf(parts[1]);
+      if (currentStage == SharedCoreStage.LocalIndexingStarted) {
+        activeIndexers++;
+      } else if (currentStage == SharedCoreStage.BlobPushStarted) {
+        activeBlobPushers++;
+      } else if (currentStage == SharedCoreStage.BlobPushFinished) {
+        // both blob pushing and indexing finish at this stage 
+        activeBlobPushers--;
+        activeIndexers--;
+      }
+
+      // making sure no other activity takes place during pull
+      if (prevStage == SharedCoreStage.BlobPullStarted) {
+        assertTrue("Pull critical section breached, currentStage=" + p, prevThreadId==currentThreadId && currentStage == SharedCoreStage.BlobPullFinished);
+      }
+
+      // making sure indexing is not disrupted by a pull from blob
+      assertFalse("Indexing breached by a pull, currentStage=" + p,
+          activeIndexers > 0 &&
+              (currentStage == SharedCoreStage.BlobPullStarted || currentStage == SharedCoreStage.BlobPullFinished));
+
+      // making sure push to blob are not disrupted by another push to blob
+      assertFalse("Blob push breached by another blob push,  currentStage=" + p, activeBlobPushers > 1);
+
+      prevThreadId = currentThreadId;
+      prevStage = currentStage;
+    }
+  }
+
+  private Map<String, CountDownLatch> configureTestBlobProcessForNode(JettySolrRunner runner) {
+    Map<String, CountDownLatch> asyncPullTracker = new HashMap<>();
+
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {
+      @Override
+      protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
+        return new CorePullTask.PullCoreCallback() {
+          @Override
+          public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+                                   String message) throws InterruptedException {
+            CountDownLatch latch = asyncPullTracker.get(pullTask.getPullCoreInfo().getCoreName());
+            if(latch != null) {
+              latch.countDown();
+            }
+          }
+        };
+      }
+    };
+
+    BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
+    setupTestBlobProcessUtilForNode(testUtil, runner);
+    return asyncPullTracker;
+  }
+
+  private void configureTestSharedConcurrencyControllerForNode(JettySolrRunner runner, List<String> progress) {
+    SharedCoreConcurrencyController concurrencyController = new SharedCoreConcurrencyController(runner.getCoreContainer()) {
+      Object recorderLock = new Object();
+      @Override
+      public void recordState(String collectionName, String shardName, String coreName, SharedCoreStage stage) {
+        super.recordState(collectionName, shardName, coreName, stage);
+
+        synchronized (recorderLock) {
+          progress.add(Thread.currentThread().getId() + "." + stage.name());
+        }
+
+      }
+    };
+    setupTestSharedConcurrencyControllerForNode(concurrencyController, runner);
+  }
+
+  private void configureTestSharedConcurrencyControllerForNode(JettySolrRunner runner, List<String> progress, IndexingThreadInterleaver interleaver) {
+    SharedCoreConcurrencyController concurrencyController = new SharedCoreConcurrencyController(runner.getCoreContainer()) {
+      Object recorderLock = new Object();
+      long firstThreadId = -1;
+      long secondThreadId = -1;
+      boolean firstAchieved = false;
+      boolean secondAchieved = false;
+      boolean secondAlreadyWaitingToStart = false;
+      CountDownLatch secondWaitingToStart = new CountDownLatch(1);
+      CountDownLatch firstWaitingForSecondToCompleteItsDesiredState = new CountDownLatch(1);
+      int timeoutSeconds = 10;
+      @Override
+      public void recordState(String collectionName, String shardName, String coreName, SharedCoreStage stage) {
+        super.recordState(collectionName, shardName, coreName, stage);
+
+        CountDownLatch latchToWait = null;
+        boolean latchShouldSucceed = true;
+        String succeedError = null;
+        String failureError = null;
+        String interruptionError = null;
+        // compute stalling decision under synchronized block and then stall(if needed) outside of synchronized block
+        synchronized (recorderLock) {
+          progress.add(Thread.currentThread().getId() + "." + stage.name());
+
+          if (interleaver.error != null) {
+            // already errored out
+            return;
+          }
+
+          long currentThreadId = Thread.currentThread().getId();
+          if (stage == SharedCoreStage.IndexingBatchReceived) {
+            // identify first and second thread and initialize their thread ids
+            // first is whichever comes first
+            if (firstThreadId == -1) {
+              firstThreadId = currentThreadId;
+            } else {
+              if(secondThreadId != -1) {
+                interleaver.error = "why there is a third indexing thread?";
+                return;
+              }
+              secondThreadId = currentThreadId;
+            }
+          }
+
+          if (firstAchieved && secondAchieved) {
+            // nothing left
+            return;
+          } else if (!secondAlreadyWaitingToStart && currentThreadId == secondThreadId) {
+            secondAlreadyWaitingToStart = true;
+            latchToWait = secondWaitingToStart;
+            latchShouldSucceed = true;
+            failureError = "second failed to wait for first to reach at desired state";
+            interruptionError = "second was interrupted while waiting for first to reach at desired state";
+          } else if (currentThreadId == firstThreadId && stage == interleaver.firstStage) {
+            // first reached to desired stage, now release second
+            secondWaitingToStart.countDown();
+            latchToWait = firstWaitingForSecondToCompleteItsDesiredState;
+            latchShouldSucceed = interleaver.isPossible;
+            succeedError = "first was able to wait for second to reach at desired state, even when it was not possible";
+            failureError = "first failed to wait for second to reach at desired state";
+            interruptionError = "first was interrupted while waiting for second to reach at desired state";
+            firstAchieved = true;
+          } else if (currentThreadId == secondThreadId && stage == interleaver.secondStage) {
+            // second also reached desired state, we are done
+            if (!firstAchieved) {
+              interleaver.error = "how come second reached its desired without first reaching its";
+            }
+            firstWaitingForSecondToCompleteItsDesiredState.countDown();
+            secondAchieved = true;
+          }
+        }
+
+        if (latchToWait != null) {
+          // need to stall this stage on the provided latch
+          try {
+            boolean successfulWait = latchToWait.await(timeoutSeconds, TimeUnit.SECONDS);
+            if (successfulWait && !latchShouldSucceed) {
+              interleaver.error = succeedError;
+            } else if (!successfulWait && latchShouldSucceed) {
+              interleaver.error = failureError;
+            }
+          } catch (InterruptedException iex) {
+            interleaver.error = interruptionError;
+          }
+        }
+      }
+    };
+    setupTestSharedConcurrencyControllerForNode(concurrencyController, runner);
+  }
+
+  /**
+   * 1. Second thread is stalled after {@link SharedCoreStage#IndexingBatchReceived} 
+   * 2. First thread is run up to firstStage then stalled
+   * 3. Second thread is resumed and run up to secondStage
+   * 4. First thread is resumed
+   */
+  private static class IndexingThreadInterleaver {
+    private final SharedCoreStage firstStage;
+    /**
+     * Let second indexing thread run up to this stage before resuming stalled first thread
+     */
+    private final SharedCoreStage secondStage;
+    private final boolean isPossible;
+    private String error = null;
+
+    private IndexingThreadInterleaver(SharedCoreStage firstStage,
+                                      SharedCoreStage secondStage,
+                                      boolean isPossible) {
+      this.firstStage = firstStage;
+      this.secondStage = secondStage;
+      this.isPossible = isPossible;
+    }
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
index fcd8459..e735315 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
@@ -169,21 +169,23 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
   
   private Map<String, CountDownLatch> configureTestBlobProcessForNode(JettySolrRunner runner) {
     Map<String, CountDownLatch> asyncPullTracker = new HashMap<>();
-    
-    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {  
+
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {
       @Override
       protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
         return new PullCoreCallback() {
           @Override
           public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
-              String message) throws InterruptedException {
+                                   String message) throws InterruptedException {
             CountDownLatch latch = asyncPullTracker.get(pullTask.getPullCoreInfo().getCoreName());
-            latch.countDown();
+            if (latch != null) {
+              latch.countDown();
+            }
           }
         };
       }
     };
-    
+
     BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
     setupTestBlobProcessUtilForNode(testUtil, runner);
     return asyncPullTracker;
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPushTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPushTest.java
index eb04529..8e44125 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPushTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPushTest.java
@@ -21,7 +21,6 @@ import java.util.Map;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.SolrQuery;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
@@ -29,13 +28,13 @@ import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.CoreStorageClient;
 import org.apache.solr.store.blob.util.BlobStoreUtils;
 import org.apache.solr.store.shared.metadata.SharedShardMetadataController;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -100,13 +99,11 @@ public class SimpleSharedStoreEndToEndPushTest extends SolrCloudSharedStoreTestC
     CoreContainer leaderCC = getCoreContainer(shardLeaderReplica.getNodeName());
     SharedShardMetadataController metadataController = leaderCC.getSharedStoreManager().getSharedShardMetadataController();
     try (SolrCore leaderCore = leaderCC.getCore(shardLeaderReplica.getCoreName())) {
-      VersionedData data = metadataController.readMetadataValue(collectionName, "shard1", false);
-      Map<String, Object> readData = (Map<String, Object>) Utils.fromJSON(data.getData());
-      String metadataSuffix = (String) readData.get(SharedShardMetadataController.SUFFIX_NODE_NAME);
+      SharedShardVersionMetadata shardMetadata = metadataController.readMetadataValue(collectionName, "shard1");
       Map<String, Object> props = collection.getSlice("shard1").getProperties();
       
       String sharedShardName = (String) props.get(ZkStateReader.SHARED_SHARD_NAME);
-      String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(metadataSuffix);
+      String blobCoreMetadataName = BlobStoreUtils.buildBlobStoreMetadataName(shardMetadata.getMetadataSuffix());
       
       // verify that we pushed the core to blob
       assertTrue(storageClient.coreMetadataExists(sharedShardName, blobCoreMetadataName));
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java b/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
index 738ff3f..5a40270 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SolrCloudSharedStoreTestCase.java
@@ -84,7 +84,15 @@ public class SolrCloudSharedStoreTestCase extends SolrCloudTestCase {
     SharedStoreManager manager = solrRunner.getCoreContainer().getSharedStoreManager();
     manager.initBlobProcessUtil(testBlobProcessUtil);
   }
-  
+
+  /**
+   * Configures the Solr process with the given {@link SharedCoreConcurrencyController}
+   */
+  protected static void setupTestSharedConcurrencyControllerForNode(SharedCoreConcurrencyController concurrencyController, JettySolrRunner solrRunner) {
+    SharedStoreManager manager = solrRunner.getCoreContainer().getSharedStoreManager();
+    manager.initConcurrencyController(concurrencyController);
+  }
+
   /**
    * Return a new CoreStorageClient that writes to the specified sharedStoreRootPath and blobDirectoryName
    * The sharedStoreRootPath should already exist when passed to this method
diff --git a/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java b/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
index 845e780..6e67b5c 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/metadata/SharedShardMetadataControllerTest.java
@@ -16,25 +16,20 @@
  */
 package org.apache.solr.store.shared.metadata;
 
-import java.util.Arrays;
 import java.util.Map;
 import java.util.NoSuchElementException;
-import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.solr.client.solrj.cloud.DistribStateManager;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.BadVersionException;
-import org.apache.solr.client.solrj.cloud.autoscaling.VersionedData;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.cloud.SolrCloudTestCase;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.Utils;
-import org.apache.zookeeper.CreateMode;
+import org.apache.solr.store.shared.metadata.SharedShardMetadataController.SharedShardVersionMetadata;
 import org.apache.zookeeper.data.Stat;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mockito.Mockito;
 
 /**
  * Tests for {@link SharedShardMetadataController}
@@ -152,18 +147,17 @@ public class SharedShardMetadataControllerTest extends SolrCloudTestCase {
     // try a conditional update that should pass and return a VersionedData instance with
     // the right written value and incremented version
     testMetadataValue = "testValue2";
-    VersionedData versionedData = shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
+    SharedShardVersionMetadata shardMetadata = shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
         testMetadataValue, version);
     
     
     // the version monotonically increases, increments on updates. We should expect only one update
-    readData = (Map<String, Object>) Utils.fromJSON(versionedData.getData());
-    assertEquals(testMetadataValue, readData.get(SharedShardMetadataController.SUFFIX_NODE_NAME));
-    assertEquals(version + 1, versionedData.getVersion());
+    assertEquals(testMetadataValue, shardMetadata.getMetadataSuffix());
+    assertEquals(version + 1, shardMetadata.getVersion());
     
     // try a conditional update that fails with the wrong version number
     try {
-      versionedData = shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
+      shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME,
           testMetadataValue, 100);
       fail();
     } catch (SolrException ex) {
@@ -187,10 +181,9 @@ public class SharedShardMetadataControllerTest extends SolrCloudTestCase {
     shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
         testMetadataValue, -1);
     
-    VersionedData versionedData = shardMetadataController.readMetadataValue(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
+    SharedShardVersionMetadata shardMetadata = shardMetadataController.readMetadataValue(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
     
-    Map<String, Object> readData = (Map<String, Object>) Utils.fromJSON(versionedData.getData());
-    assertEquals(testMetadataValue, readData.get(SharedShardMetadataController.SUFFIX_NODE_NAME));
+    assertEquals(testMetadataValue, shardMetadata.getMetadataSuffix());
   }
   
   /**
@@ -218,142 +211,4 @@ public class SharedShardMetadataControllerTest extends SolrCloudTestCase {
       assertTrue(t instanceof NoSuchElementException);
     }
   }
-  
-  /**
-   * Test that successful conditional updates caches in VersionedData in memory
-   */
-  @Test
-  public void testSuccessfulUpdateCaches() throws Exception {
-    // reset cache 
-    shardMetadataController.getVersionedDataCache().clear();
-    
-    shardMetadataController.ensureMetadataNodeExists(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    assertTrue(cluster.getZkClient().exists(metadataNodePath, false));
-    
-    String testMetadataValue = "testValue1";
-    
-    // setup with an initial value by writing
-    VersionedData newData = shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
-        testMetadataValue, -1);
-    String cacheKey = shardMetadataController.getCacheKey(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    VersionedData cachedData = shardMetadataController.getVersionedDataCache().get(cacheKey);
-    // verify the VersionedData returned is the same as the one cached
-    assertNotNull(cachedData);
-    assertEquals(newData.getVersion(), cachedData.getVersion());
-    assertTrue(Arrays.equals(newData.getData(), cachedData.getData()));
-  }
-  
-  /**
-   * Test that if an entry a VersionedData exists for a given collection and shard, and a conditional
-   * update fails, we remove the existing cached entry
-   */
-  @Test
-  public void testUnsuccessfulUpdateRemovesCacheEntry() throws Exception {
-    // reset cache 
-    shardMetadataController.getVersionedDataCache().clear();
-    
-    shardMetadataController.ensureMetadataNodeExists(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    assertTrue(cluster.getZkClient().exists(metadataNodePath, false));
-    
-    String testMetadataValue = "testValue1";
-    
-    // setup with an initial value by writing
-    shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
-        testMetadataValue, -1);
-    String cacheKey = shardMetadataController.getCacheKey(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    // verify our entry is cached now
-    VersionedData cachedData = shardMetadataController.getVersionedDataCache().get(cacheKey);
-    assertNotNull(cachedData);
-    
-    // write a value that should fail with a bad version value
-    try {
-      shardMetadataController.updateMetadataValueWithVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME, 
-          testMetadataValue, 100);
-      fail("Updating metadata value with the wrong version should have failed");
-    } catch (Exception ex) {
-      
-    }
-    
-    // verify the cached value was removed
-    cachedData = shardMetadataController.getVersionedDataCache().get(cacheKey);
-    assertNull(cachedData);
-  }
-  
-  /**
-   * Test that if an entry a VersionedData exists for a given collection and shard, it is returned
-   * on read if specified so.
-   */
-  @Test
-  public void testReadReturnsCachedEntry() throws Exception {
-    // set up mocks to verify we read from the cache
-    ConcurrentHashMap<String, VersionedData> cacheSpy = Mockito.spy(new ConcurrentHashMap<String, VersionedData>());
-    SolrCloudManager cloudManagerMock = Mockito.mock(SolrCloudManager.class);
-    DistribStateManager distribStateManagerMock = Mockito.mock(DistribStateManager.class);
-    Mockito.when(cloudManagerMock.getDistribStateManager()).thenReturn(distribStateManagerMock);
-    
-    SharedShardMetadataController shardMetadataControllerWithMock = 
-        new SharedShardMetadataController(cloudManagerMock, cacheSpy); 
-    
-    // set up some fake data
-    String cacheKey = shardMetadataControllerWithMock.getCacheKey(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    VersionedData mockData = new VersionedData(1, new byte[] {}, CreateMode.EPHEMERAL, "test");
-    shardMetadataControllerWithMock.getVersionedDataCache().put(cacheKey, mockData);
-    
-    // read and verify we pull from the cache if the value exists
-    shardMetadataControllerWithMock.readMetadataValue(TEST_COLLECTION_NAME, TEST_SHARD_NAME, true);
-    
-    // verify we pull from cache and not zookeeper
-    Mockito.verify(cacheSpy).get(shardMetadataControllerWithMock.getCacheKey(TEST_COLLECTION_NAME, TEST_SHARD_NAME));
-    Mockito.verifyZeroInteractions(distribStateManagerMock);
-  }
-  
-  /**
-   * Test that if readFromCache is specified when reading the metadata node and an entry in the
-   * cache for the given collection and shard doesn't exist, we read directly from zookeeper
-   */
-  @Test
-  public void testReadRetrievesFromZooKeeper() throws Exception {
-    ConcurrentHashMap<String, VersionedData> cacheSpy = Mockito.spy(new ConcurrentHashMap<String, VersionedData>());
-    SolrCloudManager cloudManagerMock = Mockito.mock(SolrCloudManager.class);
-    DistribStateManager distribStateManagerMock = Mockito.mock(DistribStateManager.class);
-    
-    Mockito.when(cloudManagerMock.getDistribStateManager()).thenReturn(distribStateManagerMock);
-    Mockito.when(distribStateManagerMock.getData(Mockito.any(), Mockito.any())).thenReturn(null);
-    
-    SharedShardMetadataController shardMetadataControllerWithMock = 
-        new SharedShardMetadataController(cloudManagerMock, cacheSpy); 
-    
-    // read and verify we pull from the cache if the value exists
-    shardMetadataControllerWithMock.readMetadataValue(TEST_COLLECTION_NAME, TEST_SHARD_NAME, true);
-    
-    // verify we're reading from zookeeper and not cache (even though there's no real data there)
-    Mockito.verify(distribStateManagerMock, Mockito.times(1)).getData(Mockito.any(), Mockito.any());
-  }
-  
-  /**
-   * Verify we can clear the cached version for some collection and shard
-   */
-  public void testClearCachedVersion() throws Exception {
-    // set up mocks to verify we read from the cache
-    ConcurrentHashMap<String, VersionedData> cacheSpy = Mockito.spy(new ConcurrentHashMap<String, VersionedData>());
-    SolrCloudManager cloudManagerMock = Mockito.mock(SolrCloudManager.class);
-    DistribStateManager distribStateManagerMock = Mockito.mock(DistribStateManager.class);
-    
-    Mockito.when(cloudManagerMock.getDistribStateManager()).thenReturn(distribStateManagerMock);
-    Mockito.when(distribStateManagerMock.getData(Mockito.any(), Mockito.any())).thenReturn(null);
-    
-    SharedShardMetadataController shardMetadataControllerWithMock = 
-        new SharedShardMetadataController(cloudManagerMock, cacheSpy);
-    
-    // set up some fake data
-    String cacheKey = shardMetadataControllerWithMock.getCacheKey(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    VersionedData mockData = new VersionedData(1, new byte[] {}, CreateMode.EPHEMERAL, "test");
-    shardMetadataControllerWithMock.getVersionedDataCache().put(cacheKey, mockData);
-    
-    assertEquals(1, shardMetadataControllerWithMock.getVersionedDataCache().size());
-    // try to clean and verify 
-    shardMetadataControllerWithMock.clearCachedVersion(TEST_COLLECTION_NAME, TEST_SHARD_NAME);
-    Mockito.verify(cacheSpy).remove(Mockito.any());
-    assertEquals(0, shardMetadataControllerWithMock.getVersionedDataCache().size());
-  }
 }
\ No newline at end of file


[lucene-solr] 11/11: @W-6782433 Shared replica skip recovery (#409)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 7c9063b6c5c05ecf196969ed6f88314e2ffe92a8
Author: Megan Carey <me...@salesforce.com>
AuthorDate: Tue Oct 29 15:09:23 2019 -0700

    @W-6782433 Shared replica skip recovery (#409)
    
    * Adding back joda-time dependency for S3
    
    * Skip sync and recovery for SHARED replicas prior to leader election
---
 .../org/apache/solr/cloud/ElectionContext.java     | 157 +++++++++++----------
 1 file changed, 81 insertions(+), 76 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
index ba2f3a5..d179ca0 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
@@ -362,92 +362,97 @@ 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);
-            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");
-        
-        // 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(ErrorCode.SERVICE_UNAVAILABLE, e);
-          }
-        }
-
-        PeerSync.PeerSyncResult result = null;
-        boolean success = false;
-        try {
-          result = syncStrategy.sync(zkController, core, leaderProps, weAreReplacement);
-          success = result.isSuccess();
-        } catch (Exception e) {
-          SolrException.log(log, "Exception while trying to sync", e);
-          result = PeerSync.PeerSyncResult.failure();
-        }
-        
-        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 SHARED replica, skip sync and recovery stages. a SHARED replica that is not up-to-date can
+        // still become leader; it will sync the latest from blobstore with the next request.
+        if (replicaType != Replica.Type.SHARED) {
+          // 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;
+          }
           
-          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;
+          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();
+          
+          if (weAreReplacement) {
+            // wait a moment for any floating updates to finish
+            try {
+              Thread.sleep(2500);
+            } catch (InterruptedException e) {
+              Thread.currentThread().interrupt();
+              throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, e);
             }
           }
-        }
-        
-        // solrcloud_debug
-        if (log.isDebugEnabled()) {
+  
+          PeerSync.PeerSyncResult result = null;
+          boolean success = false;
           try {
-            RefCounted<SolrIndexSearcher> searchHolder = core.getNewestSearcher(false);
-            SolrIndexSearcher searcher = searchHolder.get();
+            result = syncStrategy.sync(zkController, core, leaderProps, weAreReplacement);
+            success = result.isSuccess();
+          } catch (Exception e) {
+            SolrException.log(log, "Exception while trying to sync", e);
+            result = PeerSync.PeerSyncResult.failure();
+          }
+          
+          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 (!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 {
-              log.debug(core.getCoreContainer().getZkController().getNodeName() + " synched "
-                  + searcher.count(new MatchAllDocsQuery()));
-            } finally {
-              searchHolder.decref();
+              RefCounted<SolrIndexSearcher> searchHolder = core.getNewestSearcher(false);
+              SolrIndexSearcher searcher = searchHolder.get();
+              try {
+                log.debug(core.getCoreContainer().getZkController().getNodeName() + " synched "
+                    + searcher.count(new MatchAllDocsQuery()));
+              } 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;
+          }
         }
         
       }


[lucene-solr] 07/11: @W-6684857 Don't call shutdown multiple times (#399)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit bf100b6838f8ec14ce6f2ab707701c8bfb5716b9
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Thu Oct 10 13:20:06 2019 -0400

    @W-6684857 Don't call shutdown multiple times (#399)
    
    * Don't call shutdown multiple times when releasing blob pull resources
    * fix logging
---
 .../solr/store/blob/process/CorePullTask.java      |  4 ++--
 .../solr/store/blob/process/CorePullerThread.java  |  4 ++--
 .../solr/store/blob/process/CoreSyncFeeder.java    | 23 ++++++++++------------
 3 files changed, 14 insertions(+), 17 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index 32be1b0..0e1260d 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -296,11 +296,11 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
 
       // The following call can fail if blob is corrupt (in non trivial ways, trivial ways are identified by other cases)
       // pull was successful
-      if (CorePullerFeeder.isEmptyCoreAwaitingPull(coreContainer, pullCoreInfo.getCoreName())) {
+      //if (CorePullerFeeder.isEmptyCoreAwaitingPull(coreContainer, pullCoreInfo.getCoreName())) {
         // the javadoc for pulledBlob suggests that it is only meant to be called if we pulled from scratch
         // therefore only limiting this call when we created the local core for this pull ourselves
         // BlobTransientLog.get().getCorruptCoreTracker().pulledBlob(pullCoreInfo.coreName, blobMetadata);
-      }
+      //}
     } catch (InterruptedException e) {
       throw e;
     } catch (Exception e) {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
index 4e90ed9..ce4003e 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
@@ -52,9 +52,9 @@ public class CorePullerThread implements Runnable {
 
       } catch (InterruptedException ie) {
         log.info("Puller thread " + Thread.currentThread().getName()
-            + " got interrupted. Shutting down Blob CorePullerFeeder.");
+            + " got interrupted. Shutting down Blob CorePullerFeeder if not already.");
 
-        // Stop the puller feeder that will close the other threads and reinterrupt ourselves
+        // Stop the puller feeder that will close the other threads and re-interrupt ourselves
         pullerFeeder.close();
         Thread.currentThread().interrupt();
         break;
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
index c9cef36..b92693e 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
@@ -20,8 +20,8 @@ import java.io.Closeable;
 import java.lang.invoke.MethodHandles;
 import java.util.HashSet;
 import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
 
+import org.apache.commons.lang3.exception.ExceptionUtils;
 //import com.force.commons.util.concurrent.NamedThreadFactory; difference?
 import org.apache.lucene.util.NamedThreadFactory;
 import org.apache.solr.core.CoreContainer;
@@ -63,8 +63,6 @@ public abstract class CoreSyncFeeder implements Runnable, Closeable {
     private volatile Thread executionThread;
     private volatile boolean closed = false;
 
-    private final AtomicBoolean shouldContinue = new AtomicBoolean(true);
-
     protected CoreSyncFeeder(CoreContainer cores, int numSyncThreads) {
         this.numSyncThreads = numSyncThreads;
         this.cores = cores;
@@ -100,7 +98,8 @@ public abstract class CoreSyncFeeder implements Runnable, Closeable {
                 feedTheMonsters();
             } catch (Throwable e) {
                 if (!closed) {
-                  // TODO send warning message here
+                  log.error("CoreSyncFeeder thread encountered an error and is exiting "
+                      + "while close() was not called. " + ExceptionUtils.getStackTrace(e));
                 }
             } finally {
                 // If we stop, have our syncer "thread pool" stop as well since there's not much they can do anyway
@@ -116,24 +115,22 @@ public abstract class CoreSyncFeeder implements Runnable, Closeable {
     }
 
     boolean shouldContinueRunning() {
-        // Theoretically shouldContinue should get set to false before the CoreContainer is shutdown,
-        // but just in case we'll check both.
-        return shouldContinue.get() && !this.cores.isShutDown();
+      return !this.cores.isShutDown();
     }
 
     @Override
     public void close() {
+      if (!closed) {
         closed = true;
-
         Thread thread = this.executionThread;
         if (thread != null) {
-            this.executionThread = null; // race to set to null but ok to try to interrupt twice
-            log.info(String.format("Closing CoreSyncFeeder; interrupting execution thread %s.", thread.getName()));
-            thread.interrupt();
-            
+          this.executionThread = null; // race to set to null but ok to try to interrupt twice
+          log.info(String.format("Closing CoreSyncFeeder; interrupting execution thread %s.", thread.getName()));
+          thread.interrupt();
         } else {
-            log.warn("Closing CoreSyncFeeder before any syncer thread was started. Weird.");
+          log.warn("Closing CoreSyncFeeder before any syncer thread was started. Weird.");
         }
+      }
     }
 
     abstract String getMonsterThreadName();


[lucene-solr] 10/11: @W-6587412 (#406)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1bc8d34422c369d430cfe08425613ac99149a896
Author: Bilal Waheed <mw...@salesforce.com>
AuthorDate: Tue Oct 22 18:16:09 2019 -0400

    @W-6587412 (#406)
    
    * @W-6587412
    -CorePullerFeeder cyclic initialization stop gap measure:
     --honor interrupts
    -CorePullerThread: isLeader stop gap measure:
     --check for core existence
     --in case of failure, proceed as we are not leader
    
    * fix space.
---
 .../solr/store/blob/process/CorePullerFeeder.java  |  3 +
 .../solr/store/blob/process/CorePullerThread.java  | 66 ++++++++++++++--------
 2 files changed, 47 insertions(+), 22 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index 5fdfa6b..0db2c1b 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -97,6 +97,9 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       // todo: Fix cyclic initialization sequence
       // if thread starts early it will be killed since the initialization of sharedStoreManager has triggered the
       // creation of this thread and following line will throw NPE.
+      if (Thread.interrupted()) {
+        throw new InterruptedException();
+      }
     }
     CorePullTracker tracker = cores.getSharedStoreManager().getCorePullTracker();
     final long minMsBetweenLogs = 15000;
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
index d1ad641..8d7c418 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerThread.java
@@ -26,6 +26,7 @@ import org.apache.solr.cloud.ZkController;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.CoreDescriptor;
+import org.apache.solr.core.SolrCore;
 import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.apache.solr.store.shared.SharedCoreConcurrencyController;
 import org.apache.solr.store.shared.SharedCoreConcurrencyController.SharedCoreVersionMetadata;
@@ -74,7 +75,7 @@ public class CorePullerThread implements Runnable {
         //       - we need not to enqueue pull requests to begin with
         //       - this isLeader computation is not complete, it does not handle cases when core is absent
         //       - this isLeader computation might not be performant and efficient. I don't know if caching is involved here or not.
-        boolean isLeaderPulling = isLeader(task.getCoreContainer(), coreName);
+        boolean isLeaderPulling = isLeader(task.getCoreContainer(), collectionName, shardName, coreName);
         if (coreVersionMetadata.isSoftGuaranteeOfEquality() && isLeaderPulling) {
           // already in sync
           task.finishedPull(coreVersionMetadata.getBlobCoreMetadata(), CoreSyncStatus.SUCCESS_EQUIVALENT, null);
@@ -123,27 +124,48 @@ public class CorePullerThread implements Runnable {
   }
 
   // TODO: This is temporary, see detailed note where it is consumed above.
-  private boolean isLeader(CoreContainer coreContainer, String coreName) throws InterruptedException {
-    if (!coreContainer.isZooKeeperAware()) {
-      // not solr cloud
-      return false;
-    }
-    CoreDescriptor coreDescriptor = coreContainer.getCoreDescriptor(coreName);
-    if (coreDescriptor == null) {
-      // core does not exist
-      return false;
-    }
-    CloudDescriptor cd = coreDescriptor.getCloudDescriptor();
-    if (cd == null || cd.getReplicaType() != Replica.Type.SHARED) {
-      // not a shared replica
-      return false;
-    }
-    ZkController zkController = coreContainer.getZkController();
-    Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(cd.getCollectionName(), cd.getShardId());
-    if (leaderReplica == null || !cd.getCoreNodeName().equals(leaderReplica.getName())) {
-      // not a leader replica
-      return false;
+  private boolean isLeader(CoreContainer coreContainer, String collectionName, String shardName, String coreName) throws InterruptedException {
+    try {
+      if (!coreContainer.isZooKeeperAware()) {
+        // not solr cloud
+        return false;
+      }
+
+      CoreDescriptor coreDescriptor = coreContainer.getCoreDescriptor(coreName);
+      if (coreDescriptor == null) {
+        // core descriptor does not exist
+        return false;
+      }
+
+      SolrCore core = coreContainer.getCore(coreName);
+      if (core != null) {
+        core.close();
+      } else {
+        // core does not exist
+        return false;
+      }
+
+      CloudDescriptor cd = coreDescriptor.getCloudDescriptor();
+      if (cd == null || cd.getReplicaType() != Replica.Type.SHARED) {
+        // not a shared replica
+        return false;
+      }
+
+      ZkController zkController = coreContainer.getZkController();
+      Replica leaderReplica = zkController.getZkStateReader().getLeaderRetry(cd.getCollectionName(), cd.getShardId());
+      if (leaderReplica == null || !cd.getCoreNodeName().equals(leaderReplica.getName())) {
+        // not a leader replica
+        return false;
+      }
+
+      return true;
+    } catch (InterruptedException ie) {
+      throw ie;
+    } catch (Exception ex) {
+      log.warn(String.format("Could not establish if current replica is leader for the given core, collection=%s shard=%s core=%s",
+          collectionName, shardName, coreName), ex);
+      // we will proceed further as we are not a leader
     }
-    return true;
+    return false;
   }
 }


[lucene-solr] 06/11: @W-6684857 Refactor HttpSolrCall to call missing pull correctly and refactor asy… (#398)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit d0c13cf6b9240f369c04f08c07783a9b42e1b115
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Wed Oct 9 17:21:56 2019 -0400

    @W-6684857 Refactor HttpSolrCall to call missing pull correctly and refactor asy… (#398)
    
    * Refactor HttpSolrCall to call missing pull correctly and refactor async test
---
 .../java/org/apache/solr/servlet/HttpSolrCall.java | 142 +++++++++----------
 .../solr/store/blob/metadata/BlobCoreSyncer.java   |   2 +-
 .../solr/store/blob/SharedStorageSplitTest.java    | 152 +++++++++------------
 .../shared/SimpleSharedStoreEndToEndPullTest.java  |   1 -
 .../SimpleSharedStoreMissingCorePullTest.java      |  51 ++++++-
 5 files changed, 182 insertions(+), 166 deletions(-)

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 56235fa..c03a743 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -285,52 +285,50 @@ public class HttpSolrCall {
         String collectionName = collectionsList.isEmpty() ? null : collectionsList.get(0); // route to 1st
         //TODO try the other collections if can't find a local replica of the first?   (and do to V2HttpSolrCall)
 
-        boolean isPreferLeader = (path.endsWith("/update") || path.contains("/update/"));
+        boolean isPreferLeader = doesPathContainUpdate();
 
-        DocCollection collection = getCollection(collectionName);
         core = getCoreByCollection(collectionName, isPreferLeader); // find a local replica/core for the collection
         if (core != null) {
           if (idx > 0) {
             path = path.substring(idx);
           }
-
         } else {
-          // check pull from blob
-          if (idx > 0) {
-            // if the core belongs to a replica of a shared collection and if core is not 
-            // present locally but ZK expects replica on this node, enqueue pull
-            Replica replica = null;
-            if (collection != null && collection.getSharedIndex()) {
-              replica = getReplicaFromCurrentNode(collectionName);
-              if (replica != null) {
-                String coreName = replica.getCoreName();
-                String shardName = getShardName(collectionName, coreName);
-                BlobCoreSyncer syncer = cores.getSharedStoreManager().getBlobCoreSyncer();
-                syncer.pull(coreName, shardName, collectionName, cores, true, false);
-                core = cores.getCore(coreName);
-                if (!retry) {
-                  action = RETRY;
-                  return;
-                }
-              }
-            } 
-            
-            if (replica == null) {
-              // if we couldn't find it locally, look on other nodes
+
+          // At this point both collectionName may be null, and origCorename may reference either a core, 
+          // collection, or be null. So check if we can get a collection from either variable.
+          // TODO we don't support lists of collections yet and need to explore how this feature works...
+          DocCollection collection = getCollection(collectionName);
+          collection = (collection != null) ? collection : getCollection(origCorename);
+
+          // the core is missing locally but we check if its present in our cluster state, if it is then
+          // we trigger a missing core pull
+          if (collection != null && collection.getSharedIndex()) {
+            Replica replica = getReplicaFromCurrentNode(collection);
+            String coreName = replica.getCoreName();
+            String shardName = collection.getShardId(replica.getNodeName(), coreName);
+            BlobCoreSyncer syncer = cores.getSharedStoreManager().getBlobCoreSyncer();
+            syncer.pull(coreName, shardName, collectionName, cores, true, false);
+            core = cores.getCore(coreName);
+            if (idx > 0) {
+              path = path.substring(idx);
+            }
+          } else {
+            // if we couldn't find it locally, look on other nodes
+            if (idx > 0) {
               extractRemotePath(collectionName, origCorename);
               if (action == REMOTEQUERY) {
                 path = path.substring(idx);
                 return;
               }
             }
+            // core is not available locally or remotely
           }
-          //core is not available locally or remotely
+
           autoCreateSystemColl(collectionName);
           if (action != null) return;
-        } 
+        }
       }
     }
-    
 
     // With a valid core...
     if (core != null) {
@@ -350,15 +348,15 @@ public class HttpSolrCall {
         if (solrReq == null) {
           solrReq = parser.parse(core, path, req);
         }
-        
-        if (cores.isZooKeeperAware()) {
-          // collectionlist should be assigned in the solr cloud code above
-          String collectionName = collectionsList.isEmpty() ? null : collectionsList.get(0); // route to 1st
+
+        // don't enqueue a pull on updates as those will already trigger their own synchronous pulls
+        if (cores.isZooKeeperAware() && !doesPathContainUpdate()) {
+          String collectionName = core.getCoreDescriptor().getCloudDescriptor().getCollectionName();
           DocCollection collection = getCollection(collectionName);
-          // TODO: limit the number of pulls we do
-          if (collection != null && collection.getSharedIndex()) {
-            CorePullTracker corePullTracker = cores.getSharedStoreManager().getCorePullTracker();
-            corePullTracker.enqueueForPullIfNecessary(path, core, collectionName, cores);
+          boolean belongsToSharedCollection =
+              (collection != null) ? collection.getSharedIndex() : false;
+          if (belongsToSharedCollection) {
+            enqueuePullFromSharedStore(core);
           }
         }
 
@@ -375,6 +373,41 @@ public class HttpSolrCall {
     action = PASSTHROUGH;
   }
 
+  /**
+   * Attempt to initiate a pull from the shared store. It's the client responsibility to ensure
+   * only Shared replicas use this method.
+   */
+  protected void enqueuePullFromSharedStore(SolrCore core) throws SolrException, IOException {
+    CorePullTracker pull = cores.getSharedStoreManager().getCorePullTracker();
+    String collectionName = core.getCoreDescriptor().getCollectionName();
+    pull.enqueueForPullIfNecessary(path, core, collectionName, cores);
+  }
+
+  /**
+   * Given a collection using SHARED replicas, checks if the given collection
+   * contains a Replica that should live on the current node
+   */
+  protected Replica getReplicaFromCurrentNode(DocCollection collection) {
+    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
+    ClusterState clusterState = zkStateReader.getClusterState();
+    Set<String> liveNodes = clusterState.getLiveNodes();
+    List<Replica> replicas = collection.getReplicas(cores.getZkController().getNodeName());
+    if (replicas != null) {
+      RandomIterator<Replica> it = new RandomIterator<>(random, replicas);
+      while (it.hasNext()) {
+        Replica replica = it.next();
+        if (liveNodes.contains(replica.getNodeName())) {
+          return replica;
+        }
+      }
+    }
+    return null;
+  }
+
+  protected boolean doesPathContainUpdate() {
+    return path.endsWith("/update") || path.contains("/update/");
+  }
+
   protected void autoCreateSystemColl(String corename) throws Exception {
     if (core == null &&
         SYSTEM_COLL.equals(corename) &&
@@ -958,20 +991,13 @@ public class HttpSolrCall {
     }
     return result;
   }
-  
-  protected DocCollection getCollection(String collectionName) {
-    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
 
-    ClusterState clusterState = zkStateReader.getClusterState();
-    return clusterState.getCollectionOrNull(collectionName, true);
-  }
-  
-  protected String getShardName(String collectionName, String coreName) {
+  protected DocCollection getCollection(String collectionName) {
     ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
 
     ClusterState clusterState = zkStateReader.getClusterState();
     DocCollection collection = clusterState.getCollectionOrNull(collectionName, true);
-    return collection.getShardId(cores.getZkController().getNodeName(), coreName);
+    return collection;
   }
 
   protected SolrCore getCoreByCollection(String collectionName, boolean isPreferLeader) {
@@ -994,32 +1020,6 @@ public class HttpSolrCall {
     List<Replica> replicas = collection.getReplicas(cores.getZkController().getNodeName());
     return randomlyGetSolrCore(liveNodes, replicas);
   }
-  
-  private Replica getReplicaFromCurrentNode(String collectionName) {
-    ZkStateReader zkStateReader = cores.getZkController().getZkStateReader();
-
-    ClusterState clusterState = zkStateReader.getClusterState();
-    DocCollection collection = clusterState.getCollectionOrNull(collectionName, true);
-    
-    if (collection == null) {
-      return null;
-    }
-    
-    Set<String> liveNodes = clusterState.getLiveNodes();
-    List<Replica> replicas = collection.getReplicas(cores.getZkController().getNodeName());
-    
-    if (replicas != null) {
-      RandomIterator<Replica> it = new RandomIterator<>(random, replicas);
-      while (it.hasNext()) {
-        Replica replica = it.next();
-        if (liveNodes.contains(replica.getNodeName())) {
-          return replica;
-        }
-      }
-    }
-    
-    return null;
-  }
 
   private SolrCore randomlyGetSolrCore(Set<String> liveNodes, List<Replica> replicas) {
     if (replicas != null) {
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
index b281b27..fceb569 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
@@ -161,7 +161,7 @@ public class BlobCoreSyncer {
           pull(pushPullData, waitForSearcher, emptyCoreAwaitingPull, cores);
         } catch (Exception ex) {
           // wrap every thrown exception in a solr exception
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error trying to push to blob store", ex);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error trying to pull from blob store", ex);
         }
       }
     }
diff --git a/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java b/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
index d69766f..e29bad9 100644
--- a/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
+++ b/solr/core/src/test/org/apache/solr/store/blob/SharedStorageSplitTest.java
@@ -18,10 +18,13 @@ package org.apache.solr.store.blob;
 
 import java.nio.file.Path;
 import java.util.Collection;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
-import junit.framework.TestCase;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -32,17 +35,25 @@ import org.apache.solr.client.solrj.response.UpdateResponse;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.CoreStorageClient;
+import org.apache.solr.store.blob.process.BlobProcessUtil;
+import org.apache.solr.store.blob.process.CorePullTask;
+import org.apache.solr.store.blob.process.CorePullTask.PullCoreCallback;
+import org.apache.solr.store.blob.process.CorePullerFeeder;
+import org.apache.solr.store.blob.process.CoreSyncStatus;
 import org.apache.solr.store.shared.SolrCloudSharedStoreTestCase;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**
  * Tests for shard splitting in conjunction with shared storage
  */
 public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
+
+  static Map<String, Map<String, CountDownLatch>> solrProcessesTaskTracker = new HashMap<>();
   
   @BeforeClass
   public static void setupCluster() throws Exception {    
@@ -56,6 +67,7 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
     // configure same client for each runner, this isn't a concurrency test so this is fine
     for (JettySolrRunner runner : cluster.getJettySolrRunners()) {
       setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient), runner);
+      solrProcessesTaskTracker.put(runner.getNodeName(), configureTestBlobProcessForNode(runner));
     }
   }
   
@@ -65,7 +77,8 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
     shutdownCluster();
   }
 
-  CloudSolrClient createCollection(String collectionName, boolean sharedStorage, int repFactor) throws Exception {
+  void doSplitShard(String collectionName, boolean sharedStorage, int repFactor, int nPrefixes, int nDocsPerPrefix) throws Exception {
+
     if (sharedStorage) {
       CollectionAdminRequest
           .createCollection(collectionName, "conf", 1, 0, 0, 0)
@@ -84,11 +97,6 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
 
     CloudSolrClient client = cluster.getSolrClient();
     client.setDefaultCollection(collectionName);
-    return client;
-  }
-
-  void doSplitShard(String collectionName, boolean sharedStorage, int repFactor, int nPrefixes, int nDocsPerPrefix) throws Exception {
-    CloudSolrClient client = createCollection(collectionName, sharedStorage, repFactor);
 
     if (random().nextBoolean()) {
       for (int i = 0; i < nPrefixes; i++) {
@@ -129,8 +137,17 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
     Collection<Slice> slices = collection.getSlices();
 
     if (repFactor > 1) {
+      // set up count down latches to wait for pulls to complete
+      List<CountDownLatch> latches = new LinkedList<>();
+
       for (Slice slice : slices) {
+        CountDownLatch cdl = new CountDownLatch(slice.getReplicas().size());
+        latches.add(cdl);
         for (Replica replica : slice.getReplicas()) {
+          // ensure we count down for all replicas per slice.
+          String sharedShardName = (String) slice.getProperties().get(ZkStateReader.SHARED_SHARD_NAME);
+          solrProcessesTaskTracker.get(replica.getNodeName())
+            .put(sharedShardName, cdl);
           SolrClient replicaClient = getHttpSolrClient(replica.getBaseUrl() + "/" + replica.getCoreName());
           try {
             replicaClient.query(params("q", "*:* priming pull", "distrib", "false"));
@@ -140,29 +157,30 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
         }
       }
 
-      // TODO super ugly and inappropriate but the pull shouldn't take long. At some point we'll
-      // make our end-to-end async testing nicer by supporting test listeners for the async tasks
-      Thread.sleep(5000);
+      for (CountDownLatch latch : latches) {
+        assertTrue(latch.await(60, TimeUnit.SECONDS));
+      }
+
     }
 
     long totCount = 0;
-      for (Slice slice : slices) {
-        if (!slice.getState().equals(Slice.State.ACTIVE)) continue;
-        long lastReplicaCount = -1;
-        for (Replica replica : slice.getReplicas()) {
-          SolrClient replicaClient = getHttpSolrClient(replica.getBaseUrl() + "/" + replica.getCoreName());
-          long numFound = 0;
-          try {
-            numFound = replicaClient.query(params("q", "*:*", "distrib", "false")).getResults().getNumFound();
-          } finally {
-            replicaClient.close();
-          }
-          if (lastReplicaCount >= 0) {
-            assertEquals("Replica doc count for " + replica, lastReplicaCount, numFound);
-          }
-          lastReplicaCount = numFound;
+    for (Slice slice : slices) {
+      if (!slice.getState().equals(Slice.State.ACTIVE)) continue;
+      long lastReplicaCount = -1;
+      for (Replica replica : slice.getReplicas()) {
+        SolrClient replicaClient = getHttpSolrClient(replica.getBaseUrl() + "/" + replica.getCoreName());
+        long numFound = 0;
+        try {
+          numFound = replicaClient.query(params("q", "*:*", "distrib", "false")).getResults().getNumFound();
+        } finally {
+          replicaClient.close();
         }
-        totCount += lastReplicaCount;
+        if (lastReplicaCount >= 0) {
+          assertEquals("Replica doc count for " + replica, lastReplicaCount, numFound);
+        }
+        lastReplicaCount = numFound;
+      }
+      totCount += lastReplicaCount;
     }
 
     assertEquals(numExpected, totCount);
@@ -177,68 +195,28 @@ public class SharedStorageSplitTest extends SolrCloudSharedStoreTestCase  {
     doSplitShard("c2", true, 2, 2, 2);
   }
 
-
-  void doLiveSplitShard(String collectionName, boolean sharedStorage, int repFactor) throws Exception {
-    final CloudSolrClient client = createCollection(collectionName, sharedStorage, repFactor);
-
-    final AtomicBoolean doIndex = new AtomicBoolean(true);
-    final AtomicInteger docsIndexed = new AtomicInteger();
-    Thread indexThread = null;
-    try {
-      // start indexing client before we initiate a shard split
-      indexThread = new Thread(() -> {
-        while (doIndex.get()) {
-          try {
-            Thread.sleep(10);  // cap indexing rate at 100 docs per second...
-            int currDoc = docsIndexed.get();
-
-            // Try all docs in the same update request
-            UpdateRequest updateReq = new UpdateRequest();
-            updateReq.add(sdoc("id", "doc_" + currDoc));
-            UpdateResponse ursp = updateReq.commit(client, collectionName);
-            assertEquals(0, ursp.getStatus());  // for now, don't accept any failures
-            if (ursp.getStatus() == 0) {
-              docsIndexed.incrementAndGet();
+  private static Map<String, CountDownLatch> configureTestBlobProcessForNode(JettySolrRunner runner) {
+    Map<String, CountDownLatch> asyncPullTracker = new HashMap<>();
+
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {  
+      @Override
+      protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
+        return new PullCoreCallback() {
+          @Override
+          public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+              String message) throws InterruptedException {
+            CountDownLatch latch = asyncPullTracker.get(pullTask.getPullCoreInfo().getDedupeKey());
+            if (latch != null) {
+              latch.countDown();
             }
-          } catch (Exception e) {
-            TestCase.fail(e.getMessage());
-            break;
           }
-        }
-      });
-      indexThread.start();
-
-      Thread.sleep(100);  // wait for a few docs to be indexed before invoking split
-      int docCount = docsIndexed.get();
-
-      CollectionAdminRequest.SplitShard splitShard = CollectionAdminRequest.splitShard(collectionName)
-          .setShardName("shard1");
-      splitShard.process(client);
-      waitForState("Timed out waiting for sub shards to be active.",
-          collectionName, activeClusterShape(2, 3*repFactor));  // 2 repFactor for the new split shards, 1 repFactor for old replicas
-
-      // make sure that docs were able to be indexed during the split
-      assertTrue(docsIndexed.get() > docCount);
-
-      Thread.sleep(100);  // wait for a few more docs to be indexed after split
-
-    } finally {
-      // shut down the indexer
-      doIndex.set(false);
-      if (indexThread != null) {
-        indexThread.join();
+        };
       }
-    }
+    };
 
-    assertTrue(docsIndexed.get() > 0);
-
-    checkExpectedDocs(client, repFactor, docsIndexed.get());
-  }
-
-  @Test
-  @Ignore // need future fixes for this
-  public void testLiveSplit() throws Exception {
-    doLiveSplitShard("livesplit1", true, 1);
+    BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
+    setupTestBlobProcessUtilForNode(testUtil, runner);
+    return asyncPullTracker;
   }
 
 }
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
index 2c1bd7a..fcd8459 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
@@ -149,7 +149,6 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
       assertEquals(0, resp.getResults().getNumFound());
       
       // wait until pull is finished
-      //CountDownLatch latch = asyncPullTasks.get(followerReplica.getCoreName());
       assertTrue(latch.await(120, TimeUnit.SECONDS));
       
       // do another query to verify we've pulled everything
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreMissingCorePullTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreMissingCorePullTest.java
index 61ca4aa..0dbd11e 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreMissingCorePullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreMissingCorePullTest.java
@@ -18,6 +18,10 @@ package org.apache.solr.store.shared;
 
 import java.io.File;
 import java.nio.file.Path;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
@@ -29,7 +33,13 @@ import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
 import org.apache.solr.store.blob.client.CoreStorageClient;
+import org.apache.solr.store.blob.process.BlobProcessUtil;
+import org.apache.solr.store.blob.process.CorePullTask;
+import org.apache.solr.store.blob.process.CorePullTask.PullCoreCallback;
+import org.apache.solr.store.blob.process.CorePullerFeeder;
+import org.apache.solr.store.blob.process.CoreSyncStatus;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -114,6 +124,7 @@ public class SimpleSharedStoreMissingCorePullTest extends SolrCloudSharedStoreTe
       runner = cluster.startJettySolrRunner(runner, true);
       cluster.waitForNode(runner, /* seconds */ 30);
       setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient), cluster.getJettySolrRunner(0));
+      Map<String, CountDownLatch> asyncPullLatches = configureTestBlobProcessForNode(cluster.getJettySolrRunner(0));
       
       collection = cloudClient.getZkStateReader().getClusterState().getCollection(collectionName);
       shardLeaderReplica = collection.getLeader("shard1");
@@ -123,8 +134,15 @@ public class SimpleSharedStoreMissingCorePullTest extends SolrCloudSharedStoreTe
       core = cc.getCore(shardLeaderReplica.getCoreName());
       
       assertNull(core);
-      
-      // do a query to trigger missing core pulls
+
+      // do a query to trigger missing core pulls - this client request will wait for 5 seconds
+      // before the request fails with pull in progress. This is okay as the syncer is meant
+      // to support this.
+      // We want to wait until the pull completes so set up a count down latch for the follower's
+      // core that we'll wait until pull finishes before verifying
+      CountDownLatch latch = new CountDownLatch(1);
+      asyncPullLatches.put(shardLeaderReplica.getCoreName(), latch);
+
       try {
         resp = cloudClient.query(collectionName, params);
         assertEquals(1, resp.getResults().getNumFound());
@@ -134,10 +152,9 @@ public class SimpleSharedStoreMissingCorePullTest extends SolrCloudSharedStoreTe
         // so we can accept that in this test
       }
       
-      // TODO super ugly and inappropriate but the pull shouldn't take long. At some point we'll
-      // make our end-to-end async testing nicer by supporting test listeners for the async tasks 
-      Thread.sleep(5000);
-      
+      // wait until pull is finished
+      assertTrue(latch.await(120, TimeUnit.SECONDS));
+
       core = cc.getCore(shardLeaderReplica.getCoreName());
       // verify we pulled
       assertTrue(core.getDeletionPolicy().getLatestCommit().getFileNames().size() > 1);
@@ -152,4 +169,26 @@ public class SimpleSharedStoreMissingCorePullTest extends SolrCloudSharedStoreTe
       }
     }
   }
+
+  private Map<String, CountDownLatch> configureTestBlobProcessForNode(JettySolrRunner runner) {
+    Map<String, CountDownLatch> asyncPullTracker = new HashMap<>();
+
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {
+      @Override
+      protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
+        return new PullCoreCallback() {
+          @Override
+          public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+              String message) throws InterruptedException {
+            CountDownLatch latch = asyncPullTracker.get(pullTask.getPullCoreInfo().getCoreName());
+            latch.countDown();
+          }
+        };
+      }
+    };
+
+    BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
+    setupTestBlobProcessUtilForNode(testUtil, runner);
+    return asyncPullTracker;
+  }
 }


[lucene-solr] 04/11: Adding S3 dependencies back (#396)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 648d01e913a6c6dcd0ae2726e62da04bf3efd5bc
Author: Megan Carey <me...@salesforce.com>
AuthorDate: Tue Oct 1 19:02:04 2019 -0400

    Adding S3 dependencies back (#396)
---
 lucene/ivy-versions.properties | 1 +
 solr/core/ivy.xml              | 1 +
 2 files changed, 2 insertions(+)

diff --git a/lucene/ivy-versions.properties b/lucene/ivy-versions.properties
index c61284f..40a375d 100644
--- a/lucene/ivy-versions.properties
+++ b/lucene/ivy-versions.properties
@@ -100,6 +100,7 @@ io.prometheus.version = 0.2.0
 /javax.activation/activation = 1.1.1
 /javax.servlet/javax.servlet-api = 3.1.0
 /junit/junit = 4.12
+/joda-time/joda-time = 2.10
 
 /mecab/mecab-ipadic = 2.7.0-20070801
 /mecab/mecab-ko-dic = 2.0.3-20170922
diff --git a/solr/core/ivy.xml b/solr/core/ivy.xml
index cdf4dcc..a09196c 100644
--- a/solr/core/ivy.xml
+++ b/solr/core/ivy.xml
@@ -149,6 +149,7 @@
     <dependency org="com.amazonaws" name="aws-java-sdk-s3" rev="${/com.amazonaws/aws-java-sdk-s3}" conf="compile"/>
     <dependency org="com.amazonaws" name="aws-java-sdk-kms" rev="${/com.amazonaws/aws-java-sdk-kms}" conf="compile"/>
     <dependency org="com.amazonaws" name="jmespath-java" rev="${/com.amazonaws/jmespath-java}" conf="compile"/>
+    <dependency org="joda-time" name="joda-time" rev="${/joda-time/joda-time}" conf="compile"/>
     <dependency org="com.fasterxml.jackson.dataformat" name="jackson-dataformat-cbor" rev="${/com.fasterxml.jackson.dataformat/jackson-dataformat-cbor}" conf="compile"/>
     <dependency org="software.amazon.ion" name="ion-java" rev="${/software.amazon.ion/ion-java}" conf="compile"/>
     <dependency org="com.google.code.gson" name="gson" rev="${/com.google/gson}" conf="compile"/>


[lucene-solr] 05/11: @W-6650036 Fix pull data object mergers for dedup lists and add tests (#391)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2b70913a6cc0fc807c649b9f1734f473d53a7c91
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Thu Oct 3 18:41:50 2019 -0400

    @W-6650036 Fix pull data object mergers for dedup lists and add tests (#391)
    
    * Fix pull data object mergers for dedup lists and add tests
---
 .../solr/store/blob/process/CorePullTask.java      |  17 +-
 .../solr/store/blob/process/CorePullerFeeder.java  |  66 ++++-
 .../blob/process/PullMergeDeduplicationTest.java   | 279 +++++++++++++++++++++
 3 files changed, 344 insertions(+), 18 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index 6450135..32be1b0 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -42,6 +42,7 @@ import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Throwables;
 
 /**
@@ -81,7 +82,8 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
         callback, pullsInFlight, coresCreatedNotPulledYet);
   }
 
-  private CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, long queuedTimeMs, int attempts,
+  @VisibleForTesting
+  CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, long queuedTimeMs, int attempts,
       long lastAttemptTimestamp, PullCoreCallback callback, HashMap<String, Long> pullsInFlight, 
       Set<String> coresCreatedNotPulledYet) {
     this.coreContainer = coreContainer;
@@ -107,7 +109,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
    */
   static class PullTaskMerger implements DeduplicatingList.Merger<String, CorePullTask> {
     /**
-     * Given two tasks (that have not yet started executing!) that target the same core (and would basically do the
+     * 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.
@@ -123,10 +125,9 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       int mergedAttempts;
       long mergedLatAttemptsTimestamp;
 
-      // Synchronizing on the tasks separately to not risk deadlock (even though in practice there's only one
-      // concurrent
-      // call to this method anyway since it's called from DeduplicatingList.addDeduplicated() and we syncrhonize
-      // on the
+      // Synchronizing on the tasks separately to not risk deadlock (even though in 
+      // practice there's only one concurrent call to this method anyway since it's 
+      // called from DeduplicatingList.addDeduplicated() and we synchronize on the
       // list there).
       synchronized (task1) {
         mergedAttempts = task1.attempts;
@@ -179,6 +180,10 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
   public PullCoreInfo getPullCoreInfo() {
     return pullCoreInfo;
   }
+  
+  public long getQueuedTimeMs() {
+    return this.queuedTimeMs;
+  }
 
   /**
    * Pulls the local core updates from the Blob store then calls the task callback to notify the
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index 50bdff2..4f5a223 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -156,9 +156,10 @@ public class CorePullerFeeder extends CoreSyncFeeder {
     }
 
     PullCoreInfo(String collectionName, String shardName, String coreName, String sharedStoreName,
+        String lastReadMetadataSuffix, String newMetadataSuffix, int zkVersion,
         boolean createCoreIfAbsent, boolean waitForSearcher) {
-      // TODO - just super() to avoid compile errors
-      super();
+      super(collectionName, shardName, coreName, sharedStoreName, lastReadMetadataSuffix,
+          newMetadataSuffix, zkVersion);
       this.waitForSearcher = waitForSearcher;
       this.createCoreIfAbsent = createCoreIfAbsent;
     }
@@ -178,7 +179,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
   }
 
   /**
-   * We only want one entry in the list for each core, so when a second entry arrives, we merge them on 
+   * We only want one entry in the list for each shard, so when a second entry arrives, we merge them on 
    * their shared store name
    */
   static class PullCoreInfoMerger implements DeduplicatingList.Merger<String, PullCoreInfo> {
@@ -189,16 +190,57 @@ public class CorePullerFeeder extends CoreSyncFeeder {
 
     static PullCoreInfo mergePullCoreInfos(PullCoreInfo v1, PullCoreInfo v2) {
       assert v1.getSharedStoreName().equals(v2.getSharedStoreName());
-
-      // if one needs to wait then merged will have to wait as well 
-      final boolean waitForSearcher = v1.waitForSearcher || v2.waitForSearcher;
-
-      // if one wants to create core if absent then merged will have to create as well 
-      final boolean createCoreIfAbsent = v1.createCoreIfAbsent || v2.createCoreIfAbsent;
-
-
+      assert v1.getDedupeKey().equals(v2.getDedupeKey());
+      assert v1.getCoreName().equals(v2.getCoreName());
+      
+      // Merging the version number here implies an ordering on the pull operation
+      // enqueued as we want higher version-ed operations to be what the pulling
+      // mechanisms pull off of due to presence of metadataSuffix information
+      // Therefore these flags are dependent on which version in either PullCoreInfos 
+      // is higher EXCEPT in the case where they are the same
+      boolean waitForSearcher = false;
+      boolean createCoreIfAbsent = false;
+
+      // TODO newMetadataSuffix isn't used in the pull pipeline but the argument is nevertheless
+      // required when enqueuing a new pull and propagated downward. We should refactor 
+      // PullCoreInfo and PushPullData to make these concerns only relevant where they are needed
+      String newMetadataSuffix = null;
+      String lastReadMetadataSuffix = null;
+      int version = -1;
+      
+      // if the versions are the same then the last read metadata suffix should be the same
+      if (v1.getZkVersion() == v2.getZkVersion()) {
+        assert v1.getLastReadMetadataSuffix().equals(v2.getLastReadMetadataSuffix());
+        lastReadMetadataSuffix = v1.getLastReadMetadataSuffix();
+        // this doesn't matter which structure it comes from
+        newMetadataSuffix = v1.getNewMetadataSuffix();
+        version = v1.getZkVersion();
+        
+        // if one needs to wait then merged will have to wait as well
+        waitForSearcher = v1.waitForSearcher || v2.waitForSearcher;
+        // if one wants to create core if absent then merged will have to create as well
+        createCoreIfAbsent = v1.createCoreIfAbsent || v2.createCoreIfAbsent;
+      } else if (v1.getZkVersion() > v2.getZkVersion()) {
+        // version number increments on updates so the higher version will result in a pull
+        // from the most up-to-date state on blob
+        lastReadMetadataSuffix = v1.getLastReadMetadataSuffix();
+        newMetadataSuffix = v1.getNewMetadataSuffix();
+        version = v1.getZkVersion();
+        
+        waitForSearcher = v1.waitForSearcher;
+        createCoreIfAbsent = v1.createCoreIfAbsent;
+      } else {
+        lastReadMetadataSuffix = v2.getLastReadMetadataSuffix();
+        newMetadataSuffix = v2.getNewMetadataSuffix();
+        version = v2.getZkVersion();
+        
+        waitForSearcher = v2.waitForSearcher;
+        createCoreIfAbsent = v2.createCoreIfAbsent;
+      }
+      
       return new PullCoreInfo(v1.getCollectionName(), v1.getShardName(), v1.getCoreName(), 
-          v1.getSharedStoreName(), createCoreIfAbsent, createCoreIfAbsent);
+          v1.getSharedStoreName(), lastReadMetadataSuffix, newMetadataSuffix, version, 
+          createCoreIfAbsent, waitForSearcher); 
     }
   }
 
diff --git a/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java b/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
new file mode 100644
index 0000000..9f6d129
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/store/blob/process/PullMergeDeduplicationTest.java
@@ -0,0 +1,279 @@
+/*
+ * 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.store.blob.process;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import org.apache.solr.store.blob.client.BlobCoreMetadata;
+import org.apache.solr.store.blob.process.CorePullTask.PullCoreCallback;
+import org.apache.solr.store.blob.process.CorePullTask.PullTaskMerger;
+import org.apache.solr.store.blob.process.CorePullerFeeder.PullCoreInfo;
+import org.apache.solr.store.blob.process.CorePullerFeeder.PullCoreInfoMerger;
+import org.apache.solr.store.blob.util.DeduplicatingList;
+import org.junit.Test;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+/**
+ * Tests for {@link DeduplicatingList} using {@link PullTaskMerger} and
+ * {@link PullCoreInfoMerger}.
+ */
+public class PullMergeDeduplicationTest {
+  
+  // not relevant in pulls but value required
+  private static final String NEW_METADATA_SUFFIX = "undefined";
+  
+  private final String COLLECTION_NAME_1 = "collection1";
+  private final String COLLECTION_NAME_2 = "collection2";
+  private final String SHARD_NAME = "_shard_";
+  private final String CORE_NAME = "_core_";
+  private final String SHARED_NAME = "_sharedName_";
+
+  /**
+   * Verifies that {@link PullCoreInfoMerger} correctly merges two PullCoreInfos
+   * together
+   */
+  @Test
+  public void testPullInfoMergerMergesSuccessfully() {
+    PullCoreInfoMerger merger = new CorePullerFeeder.PullCoreInfoMerger();
+    
+    // merging two exactly the same PCI should return exactly the same
+    String lastReadMetadataSuffix = "randomString";
+    PullCoreInfo pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
+    PullCoreInfo pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
+    
+    PullCoreInfo mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
+    assertPullCoreInfo(mergedPullCoreInfo, pullCoreInfo2); 
+    
+    // boolean flags on merge flip
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, true);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
+    
+    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
+        mergedPullCoreInfo);
+    
+    // boolean flags on merge flip
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, true);
+    
+    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
+        mergedPullCoreInfo);
+    
+    // ensure the booleans are merged correctly when versions are equivalent 
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false);
+    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, false), 
+        mergedPullCoreInfo);
+    
+    // somehow if the versions are the same but lastReadMetadataSuffix are not, we fail 
+    // due to programming error
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        "notRandomString", NEW_METADATA_SUFFIX, 5, false, false);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, false );
+    try {
+      merger.merge(pullCoreInfo1, pullCoreInfo2);
+      fail();
+    } catch (Throwable e) {
+      // success
+    }
+    
+    // higher version wins 
+    pullCoreInfo1 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        "differentSuffix", NEW_METADATA_SUFFIX, 6, true, true);
+    pullCoreInfo2 = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, false, false);
+    
+    mergedPullCoreInfo = merger.merge(pullCoreInfo1, pullCoreInfo2);
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, "differentSuffix", NEW_METADATA_SUFFIX, 6, true, true), 
+        mergedPullCoreInfo);
+  }
+  
+  /**
+   * Verifies that {@link PullCoreInfoMerger} used in a {@link DeduplicatingList} merges
+   * added data successfully
+   */
+  @Test
+  public void testDeduplicatingListWithPullCoreInfoMerge() throws Exception {
+    DeduplicatingList<String, PullCoreInfo> dedupList = 
+        new DeduplicatingList<>(10, new CorePullerFeeder.PullCoreInfoMerger());
+    
+    String lastReadMetadataSuffix = "randomString";
+    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true);
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    assertEquals(1, dedupList.size());
+    
+    // add the same PCI should merge it and only result in once instance
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    assertEquals(1, dedupList.size());
+    
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, lastReadMetadataSuffix, NEW_METADATA_SUFFIX, 5, true, true), 
+        dedupList.removeFirst());
+    
+    // add a different PCI that should still be merged
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    
+    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_1, 
+        "differentString", NEW_METADATA_SUFFIX, 6, false, false);
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    
+    assertEquals(1, dedupList.size());
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, "differentString", NEW_METADATA_SUFFIX, 6, false, false), 
+        dedupList.removeFirst());
+    
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    pullCoreInfo = getTestPullCoreInfo(COLLECTION_NAME_2, 
+        "differentString", NEW_METADATA_SUFFIX, 6, false, false);
+    // add a different PCI that should not be merged
+    dedupList.addDeduplicated(pullCoreInfo, false);
+    assertEquals(2, dedupList.size());
+    
+    // check the integrity of the first 
+    assertPullCoreInfo(
+        getTestPullCoreInfo(COLLECTION_NAME_1, "differentString", NEW_METADATA_SUFFIX, 6, false, false), 
+        dedupList.removeFirst());
+  }
+  
+  /**
+   * Verifies that {@link PullTaskMerger} correctly merges two {@link CorePullTask}
+   * together
+   */
+  @Test
+  public void testPullTaskMergerMergesSuccessfully() throws Exception {
+    PullTaskMerger merger = new CorePullTask.PullTaskMerger();
+
+    // verify that merging two of the same CorePullTask is no-op and the merge results in exactly
+    // the same tasks
+    CorePullTask expectedCallbackTask = getTestCorePullTask(COLLECTION_NAME_1, 
+        /* queuedTimeMs */ 0, /* attempts */ 0, /* lastAttemptTimestamp */ 0, null);
+    PullCoreCallback callback = getTestAssertingCallback(expectedCallbackTask);
+    CorePullTask task1 = getTestCorePullTask(COLLECTION_NAME_1, 0, 0, 0, callback);
+    CorePullTask task2 = getTestCorePullTask(COLLECTION_NAME_1, 0, 0, 0, callback);
+    assertCorePullTask(task1, merger.merge(task1, task2));
+    
+    // verify that merging keeps the larger LastAttemptTimestamp and smaller attempts and queued time
+    // expectedCallbackTask should always match the second task argument provided to merge
+    expectedCallbackTask = getTestCorePullTask(COLLECTION_NAME_1, 0, 3, 1, null);
+    callback = getTestAssertingCallback(expectedCallbackTask);
+    task1 = getTestCorePullTask(COLLECTION_NAME_1, 10, 5, 10, callback);
+    task2 = getTestCorePullTask(COLLECTION_NAME_1, 0, 3, 1, callback);
+    
+    CorePullTask expectedTask = getTestCorePullTask(COLLECTION_NAME_1, 0, 3, 10, null);
+    assertCorePullTask(expectedTask, merger.merge(task1, task2));
+  }
+  
+  /**
+   * Verifies that {@link PullTaskMerger} used in a {@link DeduplicatingList} merges
+   * added data successfully
+   */
+  @Test
+  public void testDeduplicatingListWithPullTaskMerger() throws Exception {
+    DeduplicatingList<String, CorePullTask> dedupList = 
+        new DeduplicatingList<>(10, new CorePullTask.PullTaskMerger());
+    
+    // verify that merging two of the same CorePullTask is no-op and the merge results in exactly
+    // the same tasks
+    CorePullTask expectedCallbackTask = getTestCorePullTask(COLLECTION_NAME_1, 0, 0, 0, null);
+    PullCoreCallback callback = getTestAssertingCallback(expectedCallbackTask);
+    CorePullTask task1 = getTestCorePullTask(COLLECTION_NAME_1, 0, 0, 0, callback);
+    CorePullTask task2 = getTestCorePullTask(COLLECTION_NAME_1, 0, 0, 0, callback);
+    dedupList.addDeduplicated(task1, false);
+    assertEquals(1, dedupList.size());
+    
+    // note the callback here will evaluate if the task2 matches the expectedCallbackTask;
+    // this order is defined in addDeduplicated.
+    dedupList.addDeduplicated(task2, false);
+    assertEquals(1, dedupList.size());
+    // the 'merged' should just equal the original added task
+    assertCorePullTask(task1, dedupList.removeFirst());
+  }
+  
+  // return a callback and verify that the callback's propagated merged task matches the provided taskToBeMerged
+  private PullCoreCallback getTestAssertingCallback(CorePullTask expectedCallbackTask) {
+    return new PullCoreCallback() {
+      @Override
+      public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+          String message) throws InterruptedException {
+        assertNull(blobMetadata);
+        assertCorePullTask(expectedCallbackTask, pullTask);
+        assertEquals(CoreSyncStatus.TASK_MERGED, status);
+        assertEquals("CorePullTask merged with duplicate task in queue.", message);
+      }
+    };
+  }
+  
+  private CorePullTask getTestCorePullTask(String dedupKey, long queuedTimeMs, int attempts, long lastAttemptTimestamp, PullCoreCallback callback) {
+    // assume PullCoreInfo merging works correctly, just create the same static one per CorePullTask
+    // with the given dedup key
+    PullCoreInfo pullCoreInfo = getTestPullCoreInfo(dedupKey, 
+        "randomString", NEW_METADATA_SUFFIX, 5, true, true);
+    
+    CorePullTask pullTask = new CorePullTask(/* coreContainer */ null, pullCoreInfo, queuedTimeMs,
+        attempts, lastAttemptTimestamp, /* callback */ callback, Maps.newHashMap(), Sets.newHashSet());
+    return pullTask;
+  }
+  
+  private PullCoreInfo getTestPullCoreInfo(String collectionName, String lastReadMetadataSuffix,
+      String newMetadataSuffix, int version, boolean createCoreIfAbsent, boolean waitForSearcher) {
+    return new PullCoreInfo(collectionName, 
+        collectionName + SHARD_NAME,
+        collectionName + CORE_NAME,
+        collectionName + SHARED_NAME,
+        lastReadMetadataSuffix, newMetadataSuffix, version, 
+        createCoreIfAbsent, waitForSearcher);
+  }
+  
+  private void assertCorePullTask(CorePullTask expected, CorePullTask actual) {
+    assertEquals("CorePullTask attempts do not match", expected.getAttempts(), actual.getAttempts());
+    assertEquals("CorePullTask lastAttempTimestamps do not match", expected.getLastAttemptTimestamp(), actual.getLastAttemptTimestamp());
+    assertEquals("CorePullTask dedupKeys do not match", expected.getDedupeKey(), actual.getDedupeKey());
+    assertEquals("CorePullTask queuedTimed do not match", expected.getQueuedTimeMs(), actual.getQueuedTimeMs());
+    assertPullCoreInfo(expected.getPullCoreInfo(), actual.getPullCoreInfo());
+  }
+  
+  private void assertPullCoreInfo(PullCoreInfo expected, PullCoreInfo actual) {
+    assertEquals("PullCoreInfos collection names do not match", expected.getCollectionName(), actual.getCollectionName());
+    assertEquals("PullCoreInfos core names do not match", expected.getCoreName(), actual.getCoreName());
+    assertEquals("PullCoreInfos shard names do not match", expected.getShardName(), actual.getShardName());
+    assertEquals("PullCoreInfos sharedStore names do not match", expected.getSharedStoreName(), actual.getSharedStoreName());
+    assertEquals("PullCoreInfos lastReadMetadataSuffix do not match", expected.getLastReadMetadataSuffix(), actual.getLastReadMetadataSuffix());
+    assertEquals("PullCoreInfos zkVersions do not match", expected.getZkVersion(), actual.getZkVersion());
+    assertEquals("PullCoreInfos shouldCreateCoreifAbsent flag do not match", expected.shouldCreateCoreIfAbsent(), actual.shouldCreateCoreIfAbsent());
+    assertEquals("PullCoreInfos shouldWaitForSearcher flag do not match", expected.shouldWaitForSearcher(), actual.shouldWaitForSearcher());
+  }
+}


[lucene-solr] 02/11: @W-6635251 Refactor async pull code to avoid static initialized data structures from colliding in unit tests (#390)

Posted by yo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ce12e737e17225fed8d623a18fdf5ab52df825fe
Author: Andy Vuong <a....@salesforce.com>
AuthorDate: Thu Sep 26 11:58:12 2019 -0400

    @W-6635251 Refactor async pull code to avoid static initialized data structures from colliding in unit tests (#390)
    
    * Refactor async pulls to use non-static initialized pulling data structures
    * Update jenkin file to run tests on blob
---
 Jenkinsfile                                        |  0
 .../java/org/apache/solr/servlet/HttpSolrCall.java |  3 +-
 .../solr/store/blob/metadata/BlobCoreSyncer.java   | 30 ++++++-----
 .../solr/store/blob/process/BlobProcessUtil.java   |  6 ++-
 .../solr/store/blob/process/CorePullTask.java      | 54 ++++++++------------
 .../solr/store/blob/process/CorePullerFeeder.java  | 59 ++++++++++++++++++++--
 .../solr/store/blob/process/CoreSyncFeeder.java    |  6 +--
 .../solr/store/shared/SharedStoreManager.java      |  7 +++
 .../shared/SimpleSharedStoreEndToEndPullTest.java  | 52 ++++++++++---------
 9 files changed, 136 insertions(+), 81 deletions(-)

diff --git a/Jenkinsfile b/Jenkinsfile
new file mode 100644
index 0000000..e69de29
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 adfbc5d..56235fa 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -305,7 +305,8 @@ public class HttpSolrCall {
               if (replica != null) {
                 String coreName = replica.getCoreName();
                 String shardName = getShardName(collectionName, coreName);
-                BlobCoreSyncer.pull(coreName, shardName, collectionName, cores, true, false);
+                BlobCoreSyncer syncer = cores.getSharedStoreManager().getBlobCoreSyncer();
+                syncer.pull(coreName, shardName, collectionName, cores, true, false);
                 core = cores.getCore(coreName);
                 if (!retry) {
                   action = RETRY;
diff --git a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
index 520a361..b281b27 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/metadata/BlobCoreSyncer.java
@@ -45,6 +45,8 @@ import java.lang.invoke.MethodHandles;
  */
 public class BlobCoreSyncer {
 
+    private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
     /**
      * Threads wait for at most this duration before giving up on async pull to finish and returning with a PullInProgressException.
      */
@@ -70,20 +72,20 @@ public class BlobCoreSyncer {
 
     private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
+    @GuardedBy("coreSyncsInFlight")
+    private int total_waiting_threads = 0;
+
     /** The shared store name for the core currently being pulled from blob. Value is collection of objects used for synchronization by all waiting threads.
      * If both the locks on this map and on a specific SyncOnPullWait in the map are needed, the lock on the map must be acquired first.
      */
     @GuardedBy("itself")
-    private static final Map<String, Collection<SyncOnPullWait>> coreSyncsInFlight = Maps.newHashMap();
-
-    @GuardedBy("coreSyncsInFlight")
-    private static int total_waiting_threads = 0;
+    private final Map<String, Collection<SyncOnPullWait>> coreSyncsInFlight = Maps.newHashMap();
 
     /**
      * @return Total number of threads across all cores waiting for their respective core to be pulled from blob store
      */
     @VisibleForTesting
-    protected static int getTotalWaitingThreads() {
+    protected int getTotalWaitingThreads() {
         synchronized (coreSyncsInFlight) {
             return total_waiting_threads;
         }
@@ -114,11 +116,11 @@ public class BlobCoreSyncer {
      * This is just a hint because as soon as the lock is released when the method returns, the status of the core could change.
      * Because of that, in method {@link #pull(PushPullData, boolean, boolean, CoreContainer)} we need to check again.
      */
-    public static boolean isEmptyCoreAwaitingPull(String coreName) {
-        return CorePullTask.isEmptyCoreAwaitingPull(coreName);
+    public boolean isEmptyCoreAwaitingPull(CoreContainer cores, String coreName) {
+      return CorePullerFeeder.isEmptyCoreAwaitingPull(cores, coreName);
     }
     
-    public static void pull(String coreName, String shardName, String collectionName, CoreContainer cores,
+    public void pull(String coreName, String shardName, String collectionName, CoreContainer cores,
         boolean waitForSearcher, boolean emptyCoreAwaitingPull) {
       // Initialize variables
       SharedShardMetadataController sharedShardMetadataController = cores.getSharedStoreManager().getSharedShardMetadataController();
@@ -183,7 +185,7 @@ public class BlobCoreSyncer {
      *
      * @throws PullInProgressException In case a thread does not wait or times out before the async pull is finished
      */
-    public static void pull(PushPullData pushPullData, boolean waitForSearcher, boolean emptyCoreAwaitingPull, CoreContainer cores) throws PullInProgressException {
+    public void pull(PushPullData pushPullData, boolean waitForSearcher, boolean emptyCoreAwaitingPull, CoreContainer cores) throws PullInProgressException {
         // Is there another thread already working on the async pull?
         final boolean pullAlreadyInProgress;
         // Indicates if thread waits for the pull to finish or too many waiters already
@@ -199,7 +201,7 @@ public class BlobCoreSyncer {
         // Only can have only one thread working on async pull of this core (and we do no logging while holding the lock)
         // Let's understand what our role and actions are while holding the global lock and then execute on them without the lock.
         synchronized (coreSyncsInFlight) {
-            if (emptyCoreAwaitingPull && !isEmptyCoreAwaitingPull(pushPullData.getCoreName())) {
+            if (emptyCoreAwaitingPull && !isEmptyCoreAwaitingPull(cores, pushPullData.getCoreName())) {
                 // Core was observed empty awaiting pull and is no longer awaiting pull. This means the pull happened.
                 return;
             }
@@ -336,7 +338,7 @@ public class BlobCoreSyncer {
      * This is called whenever core from {@link CorePullTracker} finish its async pull(successfully or unsuccessfully)
      * We use this to notify all waiting threads for a core that their wait has ended (if there are some waiting).
      */
-    public static void finishedPull(String sharedStoreName, CoreSyncStatus status, BlobCoreMetadata blobMetadata, String message) {
+    public void finishedPull(String sharedStoreName, CoreSyncStatus status, BlobCoreMetadata blobMetadata, String message) {
         Exception pullException = null;
         final boolean isPullSuccessful = (status.isSuccess() ||
                 // Following statuses are not considered success in strictest definition of pull but for BlobSyncer
@@ -351,7 +353,7 @@ public class BlobCoreSyncer {
         notifyEndOfPull(sharedStoreName, pullException);
     }
 
-    private static void throwPullInProgressException(String corename, String msgSuffix) throws PullInProgressException {
+    private void throwPullInProgressException(String corename, String msgSuffix) throws PullInProgressException {
         String msg = SKIPPING_PULLING_CORE + " " + corename + " from blob " + msgSuffix;
         log.info(msg);
         // Note that longer term, this is the place where we could decide that if the async
@@ -366,7 +368,7 @@ public class BlobCoreSyncer {
      * Also serves the purpose of being a memory barrier so that the waiting threads can check their SyncOnPullWait instances
      * for updates.
      */
-    private static void notifyEndOfPull(String sharedStoreName, Exception e) {
+    private void notifyEndOfPull(String sharedStoreName, Exception e) {
         final Collection<SyncOnPullWait> collectionOfWaiters = pullEnded(sharedStoreName);
         if (collectionOfWaiters != null) {
             for (SyncOnPullWait w : collectionOfWaiters) {
@@ -381,7 +383,7 @@ public class BlobCoreSyncer {
      * Cleans up the core from bookkeeping related to in-progress pulls and returns the collection of waiters for that core.
      * Collection of returned waiters could be null as well.
      */
-    private static Collection<SyncOnPullWait> pullEnded(String sharedStoreName) {
+    private Collection<SyncOnPullWait> pullEnded(String sharedStoreName) {
         final Collection<SyncOnPullWait> collectionOfWaiters;
         synchronized (coreSyncsInFlight) {
             // Note that threads waiting for the pull to finish have references on their individual SyncOnPullWait instances,
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java b/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
index 164818d..7df8aa4 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/BlobProcessUtil.java
@@ -44,6 +44,10 @@ public class BlobProcessUtil {
     runningFeeder = initializeCorePullerFeeder(cpf);
   }
   
+  public CorePullerFeeder getCorePullerFeeder() {
+    return runningFeeder;
+  }
+  
   /**
    * Shutdown background blob puller process
    */
@@ -71,10 +75,10 @@ public class BlobProcessUtil {
    */
   private void shutdownCorePullerFeeder() {
     final CoreSyncFeeder rf = runningFeeder;
-    runningFeeder = null;
     if (rf != null) {
       log.info("Shutting down CorePullerFeeder");
       rf.close();
     }
+    runningFeeder = null;
   }
 }
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
index e8af082..6450135 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullTask.java
@@ -43,8 +43,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Throwables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
 
 /**
  * Code for pulling updates on a specific core to the Blob store. see {@CorePushTask} for the push version of this.
@@ -60,52 +58,42 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
    */
   private static final long MIN_RETRY_DELAY_MS = 20000;
 
-  /** Cores currently being pulled and timestamp of pull start (to identify stuck ones in logs) */
-  private static final HashMap<String, Long> pullsInFlight = Maps.newHashMap();
-
-  /** Cores unknown locally that got created as part of the pull process but for which no data has been pulled yet
-   * from Blob store. If we ignore this transitory state, these cores can be accessed locally and simply look empty.
-   * We'd rather treat threads attempting to access such cores like threads attempting to access an unknown core and
-   * do a pull (or more likely wait for an ongoing pull to finish).<p>
-   *
-   * When this lock has to be taken as well as {@link #pullsInFlight}, then {@link #pullsInFlight} has to be taken first.
-   * Reading this set implies acquiring the monitor of the set (as if @GuardedBy("itself")), but writing to the set
-   * additionally implies holding the {@link #pullsInFlight}. This guarantees that while {@link #pullsInFlight}
-   * is held, no element in the set is changing.
-   */
-  private static final Set<String> coresCreatedNotPulledYet = Sets.newHashSet();
-
   private final CoreContainer coreContainer;
   private final PullCoreInfo pullCoreInfo;
+  
+  /**
+   * Data structures injected as dependencies that track the core pulls occurring
+   * in flight and the cores that have been created and not pulled. These should
+   * be passed in via a constructor from CorePullerFeeder where they are defined
+   * and are unique per CorePullerFeeder (itself a singleton).
+   */
+  private final HashMap<String, Long> pullsInFlight;
+  private final Set<String> coresCreatedNotPulledYet;
+  
   private final long queuedTimeMs;
   private int attempts;
   private long lastAttemptTimestamp;
   private final PullCoreCallback callback;
 
-  CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, PullCoreCallback callback) {
-    this(coreContainer, pullCoreInfo, System.currentTimeMillis(), 0, 0L, callback);
+  CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, PullCoreCallback callback,
+      HashMap<String, Long> pullsInFlight, Set<String> coresCreatedNotPulledYet) {
+    this(coreContainer, pullCoreInfo, System.currentTimeMillis(), 0, 0L, 
+        callback, pullsInFlight, coresCreatedNotPulledYet);
   }
 
   private CorePullTask(CoreContainer coreContainer, PullCoreInfo pullCoreInfo, long queuedTimeMs, int attempts,
-      long lastAttemptTimestamp, PullCoreCallback callback) {
+      long lastAttemptTimestamp, PullCoreCallback callback, HashMap<String, Long> pullsInFlight, 
+      Set<String> coresCreatedNotPulledYet) {
     this.coreContainer = coreContainer;
     this.pullCoreInfo = pullCoreInfo;
     this.queuedTimeMs = queuedTimeMs;
     this.attempts = attempts;
     this.lastAttemptTimestamp = lastAttemptTimestamp;
     this.callback = callback;
+    this.pullsInFlight = pullsInFlight;
+    this.coresCreatedNotPulledYet = coresCreatedNotPulledYet;
   }
-
-  /**
-   * Returns a _hint_ that the given core might be locally empty because it is awaiting pull from Blob store.
-   * This is just a hint because as soon as the lock is released when the method returns, the status of the core could change.
-   */
-  public static boolean isEmptyCoreAwaitingPull(String corename) {
-    synchronized (coresCreatedNotPulledYet) {
-      return coresCreatedNotPulledYet.contains(corename);
-    }
-  }
-
+  
   /**
    * Needed for the {@link CorePullTask} to be used in a {@link DeduplicatingList}.
    */
@@ -164,7 +152,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
       // We merge the tasks.
       return new CorePullTask(task1.coreContainer, mergedPullCoreInfo,
           Math.min(task1.queuedTimeMs, task2.queuedTimeMs), mergedAttempts, mergedLatAttemptsTimestamp,
-          task1.callback);
+          task1.callback, task1.pullsInFlight, task1.coresCreatedNotPulledYet);
     }
   }
 
@@ -303,7 +291,7 @@ public class CorePullTask implements DeduplicatingList.Deduplicatable<String> {
 
       // The following call can fail if blob is corrupt (in non trivial ways, trivial ways are identified by other cases)
       // pull was successful
-      if(isEmptyCoreAwaitingPull(pullCoreInfo.getCoreName())){
+      if (CorePullerFeeder.isEmptyCoreAwaitingPull(coreContainer, pullCoreInfo.getCoreName())) {
         // the javadoc for pulledBlob suggests that it is only meant to be called if we pulled from scratch
         // therefore only limiting this call when we created the local core for this pull ourselves
         // BlobTransientLog.get().getCorruptCoreTracker().pulledBlob(pullCoreInfo.coreName, blobMetadata);
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
index b950d67..50bdff2 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CorePullerFeeder.java
@@ -17,6 +17,8 @@
 package org.apache.solr.store.blob.process;
 
 import java.lang.invoke.MethodHandles;
+import java.util.HashMap;
+import java.util.Set;
 
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.store.blob.client.BlobCoreMetadata;
@@ -26,6 +28,9 @@ import org.apache.solr.store.blob.util.DeduplicatingList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
 /**
  * A pull version of {@link CoreSyncFeeder} then will continually ({@link #feedTheMonsters()}) to load up a work queue (
  * {@link #pullTaskQueue}) with such tasks {@link CorePullTask} to keep the created threads busy :) The tasks will be
@@ -35,19 +40,54 @@ import org.slf4j.LoggerFactory;
 public class CorePullerFeeder extends CoreSyncFeeder {
 
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  private final CorePullTask.PullCoreCallback callback;
 
-  protected final DeduplicatingList<String, CorePullTask> pullTaskQueue;
   protected static final String PULLER_THREAD_PREFIX = "puller";
 
   private static final int numPullerThreads = 5; // TODO : make configurable
 
+  private final CorePullTask.PullCoreCallback callback;
+
+  protected final DeduplicatingList<String, CorePullTask> pullTaskQueue;
+
+  /**
+   * Cores currently being pulled and timestamp of pull start (to identify stuck ones in logs)
+   *
+   * Note, it is the client's responsibility to synchronize accesses
+   */
+  private final HashMap<String, Long> pullsInFlight = Maps.newHashMap();
+
+  /** Cores unknown locally that got created as part of the pull process but for which no data has been pulled yet
+   * from Blob store. If we ignore this transitory state, these cores can be accessed locally and simply look empty.
+   * We'd rather treat threads attempting to access such cores like threads attempting to access an unknown core and
+   * do a pull (or more likely wait for an ongoing pull to finish).<p>
+   *
+   * When this lock has to be taken as well as {@link #pullsInFlight}, then {@link #pullsInFlight} has to be taken first.
+   * Reading this set implies acquiring the monitor of the set (as if @GuardedBy("itself")), but writing to the set
+   * additionally implies holding the {@link #pullsInFlight}. This guarantees that while {@link #pullsInFlight}
+   * is held, no element in the set is changing.
+   *
+   * Note, it is the client's responsibility to synchronize accesses
+   */
+  private final Set<String> coresCreatedNotPulledYet = Sets.newHashSet();
+
   protected CorePullerFeeder(CoreContainer cores) {
     super(cores, numPullerThreads);
     this.pullTaskQueue = new DeduplicatingList<>(ALMOST_MAX_WORKER_QUEUE_SIZE, new CorePullTask.PullTaskMerger());
     this.callback = new CorePullResult();
   }
 
+  /**
+   * Returns a _hint_ that the given core might be locally empty because it is awaiting pull from Blob store.
+   * This is just a hint because as soon as the lock is released when the method returns, the status of the core could change.
+   */
+  public static boolean isEmptyCoreAwaitingPull(CoreContainer cores, String corename) {
+    CorePullerFeeder cpf = cores.getSharedStoreManager().getBlobProcessManager().getCorePullerFeeder();
+    Set<String> coresCreatedNotPulledYet = cpf.getCoresCreatedNotPulledYet();
+    synchronized (coresCreatedNotPulledYet) {
+      return coresCreatedNotPulledYet.contains(corename);
+    }
+  }
+
   @Override
   public Runnable getSyncer() {
     return new CorePullerThread(this, pullTaskQueue);
@@ -62,6 +102,14 @@ public class CorePullerFeeder extends CoreSyncFeeder {
     return callback;
   }
 
+  protected HashMap<String, Long> getPullsInFlight() {
+    return pullsInFlight;
+  }
+
+  protected Set<String> getCoresCreatedNotPulledYet() {
+    return coresCreatedNotPulledYet;
+  }
+
   @Override
   void feedTheMonsters() throws InterruptedException {
     CorePullTracker tracker = cores.getSharedStoreManager().getCorePullTracker();
@@ -73,7 +121,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
       PullCoreInfo pci = tracker.getCoreToPull();
 
       // Add the core to the list consumed by the thread doing the actual work
-      CorePullTask pt = new CorePullTask(cores, pci, getCorePullTaskCallback());
+      CorePullTask pt = new CorePullTask(cores, pci, getCorePullTaskCallback(), pullsInFlight, coresCreatedNotPulledYet);
       pullTaskQueue.addDeduplicated(pt, /* isReenqueue */ false);
       syncsEnqueuedSinceLastLog++;
 
@@ -95,7 +143,7 @@ public class CorePullerFeeder extends CoreSyncFeeder {
    * deduplicated on core name (the same core requiring two pulls from Blob will only be recorded one if the first
    * pull has not been processed yet).
    */
-  static class PullCoreInfo extends PushPullData implements DeduplicatingList.Deduplicatable<String> {
+  public static class PullCoreInfo extends PushPullData implements DeduplicatingList.Deduplicatable<String> {
 
     private final boolean waitForSearcher;
     private final boolean createCoreIfAbsent;
@@ -182,7 +230,8 @@ public class CorePullerFeeder extends CoreSyncFeeder {
           log.warn(String.format("Pulling core %s failed. Giving up. Last status=%s attempts=%s . %s",
               pullCoreInfo.getSharedStoreName(), status, pullTask.getAttempts(), message == null ? "" : message));
         }
-        BlobCoreSyncer.finishedPull(pullCoreInfo.getSharedStoreName(), status, blobMetadata, message);
+        BlobCoreSyncer syncer = cores.getSharedStoreManager().getBlobCoreSyncer();
+        syncer.finishedPull(pullCoreInfo.getSharedStoreName(), status, blobMetadata, message);
       } catch (InterruptedException ie) {
         close();
         throw ie;
diff --git a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
index 820b4ee..c9cef36 100644
--- a/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
+++ b/solr/core/src/java/org/apache/solr/store/blob/process/CoreSyncFeeder.java
@@ -17,17 +17,16 @@
 package org.apache.solr.store.blob.process;
 
 import java.io.Closeable;
+import java.lang.invoke.MethodHandles;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.solr.core.CoreContainer;
-
 //import com.force.commons.util.concurrent.NamedThreadFactory; difference?
 import org.apache.lucene.util.NamedThreadFactory;
+import org.apache.solr.core.CoreContainer;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.lang.invoke.MethodHandles;
 
 /**
  * A {@link Runnable} that will start a set of threads {@link CorePullerThread} to process tasks
@@ -131,6 +130,7 @@ public abstract class CoreSyncFeeder implements Runnable, Closeable {
             this.executionThread = null; // race to set to null but ok to try to interrupt twice
             log.info(String.format("Closing CoreSyncFeeder; interrupting execution thread %s.", thread.getName()));
             thread.interrupt();
+            
         } else {
             log.warn("Closing CoreSyncFeeder before any syncer thread was started. Weird.");
         }
diff --git a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
index 3dd525a..8a17f2b 100644
--- a/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
+++ b/solr/core/src/java/org/apache/solr/store/shared/SharedStoreManager.java
@@ -17,6 +17,7 @@
 package org.apache.solr.store.shared;
 
 import org.apache.solr.cloud.ZkController;
+import org.apache.solr.store.blob.metadata.BlobCoreSyncer;
 import org.apache.solr.store.blob.process.BlobDeleteManager;
 import org.apache.solr.store.blob.process.BlobProcessUtil;
 import org.apache.solr.store.blob.process.CorePullTracker;
@@ -38,11 +39,13 @@ public class SharedStoreManager {
   private BlobDeleteManager blobDeleteManager;
   private BlobProcessUtil blobProcessUtil;
   private CorePullTracker corePullTracker;
+  private BlobCoreSyncer blobCoreSyncer;
   
   public SharedStoreManager(ZkController controller) {
     zkController = controller;
     // initialize BlobProcessUtil with the SharedStoreManager for background processes to be ready
     blobProcessUtil = new BlobProcessUtil(zkController.getCoreContainer());
+    blobCoreSyncer = new BlobCoreSyncer();
   }
   
   @VisibleForTesting
@@ -106,4 +109,8 @@ public class SharedStoreManager {
     corePullTracker = new CorePullTracker();
     return corePullTracker ;
   }
+  
+  public BlobCoreSyncer getBlobCoreSyncer() {
+    return blobCoreSyncer;
+  }
 }
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
index 4a7921e..2c1bd7a 100644
--- a/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
+++ b/solr/core/src/test/org/apache/solr/store/shared/SimpleSharedStoreEndToEndPullTest.java
@@ -75,23 +75,21 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
     setupCluster(2);
     CloudSolrClient cloudClient = cluster.getSolrClient();
     
-    // setup the test harnesses
-    Map<String, CountDownLatch> cdlMap = new HashMap<>();
+    // this map tracks the async pull queues per solr process
+    Map<String, Map<String, CountDownLatch>> solrProcessesTaskTracker = new HashMap<>();
     
-    CountDownLatch latch1 = new CountDownLatch(1);
     JettySolrRunner solrProcess1 = cluster.getJettySolrRunner(0);
     CoreStorageClient storageClient1 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
     setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient1), solrProcess1);
-    configureTestBlobProcessForNode(solrProcess1, setupCallback(latch1));
+    Map<String, CountDownLatch> asyncPullLatches1 = configureTestBlobProcessForNode(solrProcess1);
     
-    CountDownLatch latch2 = new CountDownLatch(1);
     JettySolrRunner solrProcess2 = cluster.getJettySolrRunner(1);
     CoreStorageClient storageClient2 = setupLocalBlobStoreClient(sharedStoreRootPath, DEFAULT_BLOB_DIR_NAME);
     setupTestSharedClientForNode(getBlobStorageProviderTestInstance(storageClient2), solrProcess2);
-    configureTestBlobProcessForNode(solrProcess2, setupCallback(latch2));
+    Map<String, CountDownLatch> asyncPullLatches2 = configureTestBlobProcessForNode(solrProcess2);
     
-    cdlMap.put(solrProcess1.getNodeName(), latch1);
-    cdlMap.put(solrProcess2.getNodeName(), latch2);
+    solrProcessesTaskTracker.put(solrProcess1.getNodeName(), asyncPullLatches1);
+    solrProcessesTaskTracker.put(solrProcess2.getNodeName(), asyncPullLatches2);
     
     String collectionName = "sharedCollection";
     int maxShardsPerNode = 1;
@@ -138,14 +136,20 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
       assertEquals(1, resp.getResults().getNumFound());
       assertEquals("1", (String) resp.getResults().get(0).getFieldValue("id"));
       
+      // we want to wait until the pull completes so set up a count down latch for the follower's
+      // core that we'll wait until pull finishes for
+      CountDownLatch latch = new CountDownLatch(1);
+      Map<String, CountDownLatch> asyncPullTasks = solrProcessesTaskTracker.get(followerReplica.getNodeName());
+      asyncPullTasks.put(followerReplica.getCoreName(), latch);
+      
       // query the follower directly to trigger the pull, this query should yield no results
-      // as we don't wait
+      // as it returns immediately 
       followerDirectClient = getHttpSolrClient(followerReplica.getBaseUrl() + "/" + followerReplica.getCoreName());
       resp = followerDirectClient.query(params);
       assertEquals(0, resp.getResults().getNumFound());
       
       // wait until pull is finished
-      CountDownLatch latch = cdlMap.get(followerReplica.getNodeName());
+      //CountDownLatch latch = asyncPullTasks.get(followerReplica.getCoreName());
       assertTrue(latch.await(120, TimeUnit.SECONDS));
       
       // do another query to verify we've pulled everything
@@ -164,26 +168,26 @@ public class SimpleSharedStoreEndToEndPullTest extends SolrCloudSharedStoreTestC
     }
   }
   
-  private BlobProcessUtil configureTestBlobProcessForNode(JettySolrRunner runner, PullCoreCallback callback) {
-    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {
+  private Map<String, CountDownLatch> configureTestBlobProcessForNode(JettySolrRunner runner) {
+    Map<String, CountDownLatch> asyncPullTracker = new HashMap<>();
+    
+    CorePullerFeeder cpf = new CorePullerFeeder(runner.getCoreContainer()) {  
       @Override
       protected CorePullTask.PullCoreCallback getCorePullTaskCallback() {
-        return callback;
+        return new PullCoreCallback() {
+          @Override
+          public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
+              String message) throws InterruptedException {
+            CountDownLatch latch = asyncPullTracker.get(pullTask.getPullCoreInfo().getCoreName());
+            latch.countDown();
+          }
+        };
       }
     };
+    
     BlobProcessUtil testUtil = new BlobProcessUtil(runner.getCoreContainer(), cpf);
     setupTestBlobProcessUtilForNode(testUtil, runner);
-    return testUtil;
+    return asyncPullTracker;
   }
   
-  private PullCoreCallback setupCallback(CountDownLatch latch) {
-    return new PullCoreCallback() {
-      @Override
-      public void finishedPull(CorePullTask pullTask, BlobCoreMetadata blobMetadata, CoreSyncStatus status,
-          String message) throws InterruptedException {
-        assertTrue(status.isSuccess());
-        latch.countDown();
-      }
-    };
-  }
 }