You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@solr.apache.org by GitBox <gi...@apache.org> on 2021/10/01 19:33:51 UTC

[GitHub] [solr] warperwolf opened a new pull request #324: SOLR-14660 - move HDFS to a contrib module

warperwolf opened a new pull request #324:
URL: https://github.com/apache/solr/pull/324


   https://issues.apache.org/jira/browse/SOLR-14660
   
   # Description
   
   This PR moves HDFS from Core to a Contrib module.
   
   Note: I kept separate commits to make it easier to review and to change / revert if necessary. They can be squashed once everything is polished and we have a green light.
   
   # Solution
   - A new contrib module has been created with its own build
   - HDFS classes have been moved to the new module
   - This PR does not yet take advantage of the Solr packaging system, this is just a separate module. Implementing as a package will be a next step. 
   - HDFS directory direct references have been removed from Solr core (for example the UpdateHandler had a direct reference to HdfsDirectoryFactory, removed it so it works the same way as other directory factories and can also be loaded as a plugin (core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);)
   - Solr Core still has some Hadoop-related (but non HDFS-related) classes (mostly Hadoop authentication), these are still left in Core , if they need to be separated, they will be handled in a separate jira
   - The block cache feature was only used by HDFS, moved to the contrib module)
   - The only change needed to use this module is to include its jar on the classpath (for example symlinking the jar to the web-inf/lib of the webapp), no changes to the solrconfig.xml are needed.
   - DirectoryFactory had DirectoryFactory.java: public final static String LOCK_TYPE_HDFS   = "hdfs";  , but this was only referenced from the tests and from HdfsDirectoryFactory, removed it.
   - Removed the deprecated flags from the code and the warning from the ref guide.
   - Changed reference in ref guide to the new location of HdfsDirectoryFactory.
   - Added a simple readme markdown doc 
    
   # Tests
   - Existing tests have been moved to the new contrib module where needed
   - Test have been refactored where a Hdfs test was extending a Core test class. For example, HdfsChaosMonkeyNothingIsSafeTest (hdfs test) extends ChaosMonkeyNothingIsSafeTest (core test). 
   Gradle does not allow dependencies between test projects of 2 different modules, so in such cases I introduced an abstract base class (here AbstractChaosMonkeyNothingIsSafeTest) which is extended by both. A good place to store these classes would be the test fixtures feature of gradle which is especially designed for this purpose, however some of the gradle plugins Solr uses fail because they are not compatible with fixtures (see https://issues.apache.org/jira/browse/SOLR-14660?focusedCommentId=17409690&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17409690 ), so I moved these classes to the Solr Test Framework project. 
   - Solr core had some helper classes which are core related but they are also referenced by the hdfs classes (for example the MockCoreContainer.java) - moved those to gradle test fixtures of the Core project, and later moved them to the test framework
   - I also created a Cloudera CDP test release and built a cluster with HDFS and Solr and tested there
   - TestBackupRepositoryFactory had some references to HdfsBackupRepository, but the test case is not hdfs related. Removed that part.
   - HDFSRecoveryZKTest was also failing before this change. Added an AwaitsFix annotation, needs to be fixed separately.
   
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/solr/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [x] I have given Solr maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   - [x] I have added documentation for the [Reference Guide](https://github.com/apache/solr/tree/main/solr/solr-ref-guide)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1024422694


   I went through these two and I'm not sure there is a good way to address them right now. I don't think it should hold up the merge of this PR.
   
   * render-javadoc needs a little more attention - basically shouldn't need so many changes
       * This is related to the bullet below. There were a few classes moved to `test-framework` which ended up in a few more split packages. This should be addressed for all of test-framework.
   * Abstract*Base test files moved to test-framework - not sure they are needed, but its a lot of the changes in this besides the move.
       * This seems to be necessary due to how gradle and test dependencies end up interacting. It matches the existing pattern of some test-framework classes. The commit log has details about why each file was created/moved in test-framework.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1018680697


   > I'm concerned about this missing changes to the Hadoop code from the endianness fix, the upgrade to 3.3.1, and whatever else we've had to patch up since this PR was originally opened. What is our strategy for keeping things in sync.
   
   @warperwolf and I have been recently talking about this. I've been keeping an eye on the changes that have been made to the Hadoop code and this PR so it shouldn't be terrible to keep up to date. We should get this merged as soon as reasonably ready to avoid having to keep it in sync any longer.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792130767



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r793134447



##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *InvalidBlockTag:*  Tag name `lucene.experimental` is unknown. If this is a commonly-used custom tag, please click 'not useful' and file a bug. [(details)](https://errorprone.info/bugpattern/InvalidBlockTag)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *InvalidBlockTag:*  Tag name `lucene.experimental` is unknown. If this is a commonly-used custom tag, please click 'not useful' and file a bug. [(details)](https://errorprone.info/bugpattern/InvalidBlockTag)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *MissingSummary:*  A summary line is required on public/protected Javadocs. [(details)](https://google.github.io/styleguide/javaguide.html#s7.2-summary-fragment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *MissingSummary:*  A summary line is required on public/protected Javadocs. [(details)](https://google.github.io/styleguide/javaguide.html#s7.2-summary-fragment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       *UnusedVariable:*  The field 'log' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794661450



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,12 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      // if the update log class is not defined in the plugin info / solrconfig.xml
+      // (like <updateLog class="${solr.ulog:solr.UpdateLog}"> )
+      // we fall back use the one which is the default for the given directory factory
+      ulog = ulogPluginInfo.className == null ? dirFactory.newDefaultUpdateLog() :

Review comment:
       @dsmiley  not sure I understand your comment here. The original logic:
   
   ```
   DirectoryFactory dirFactory = core.getDirectoryFactory();
         if (dirFactory instanceof HdfsDirectoryFactory) {
           ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
         } else {
           ulog = ulogPluginInfo.className == null ? new UpdateLog():
                   core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
         }
   ```
   
   If `ulogPluginInfo.className == null` then `new UpdateLog();`. So I don't think there is any change here to the logic for `UpdateLog()`. Line 132 has `ulog.init(ulogPluginInfo);` which I think makes sure the ulog implementation takes into account the `uLogPluginInfo).




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794173884



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,12 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      // if the update log class is not defined in the plugin info / solrconfig.xml
+      // (like <updateLog class="${solr.ulog:solr.UpdateLog}"> )
+      // we fall back use the one which is the default for the given directory factory
+      ulog = ulogPluginInfo.className == null ? dirFactory.newDefaultUpdateLog() :

Review comment:
       but then uLogPluginInfo would be ignored, which might have settings to be applied.  Either `newDefaultUpdateLog` should take the pluginInfo as a parameter, or just have it return the class (or class name) to be instantiated (and maybe tweak the name to be getDefaultUpdateLogClassName).  I think the latter is cleaner.

##########
File path: solr/modules/hdfs/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Module
+===============================
+
+Introduction
+------------
+This module implements the support for Hadoop Distributed File System in Apache Solr. 
+
+Building
+--------
+The HDFS module uses the same Gradle build as the core Solr components. 
+
+To build the module, you can use
+
+```
+./gradlew :solr:modules:hdfs:assemble
+```
+
+The resulting module will be placed to the libs directory, for example:
+`solr/modules/hdfs/build/libs/solr-hdfs-9.0.0-SNAPSHOT.jar`
+
+To execute the module tests:
+
+```
+./gradlew :solr:modules:hdfs:test
+```
+
+Usage
+-----
+To use the module, it needs to be placed to the Solr web application classpath (for example by symlinking it to the WEB-INF/lib directory.)

Review comment:
       IMO, unless there are unique constraints about HDFS's JARs, we can simply refer to the ref guide (`solr-plugins.adoc`)  on how plugins are installed.  There are a number of options.

##########
File path: solr/modules/hdfs/src/test/org/apache/solr/store/blockcache/BlockCacheTest.java
##########
@@ -257,7 +259,7 @@ public void testCacheConcurrent() throws Exception {
           return;
         }
       }
-      assertEquals("cache key differs from value's key", k, (Long) v.key);
+      Assert.assertEquals("cache key differs from value's key", k, (Long) v.key);

Review comment:
       minor: you could just add static imports for this stuff.  I'm not sure why these changes are happening.

##########
File path: solr/solr-ref-guide/src/solr-on-hdfs.adoc
##########
@@ -16,11 +16,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-WARNING: Storing indexes in HDFS is deprecated and may be be removed in 9.0.
-This functionality may be moved to a 3rd-party plugin in the future.
 
 Solr has support for writing and reading its index and transaction log files to the HDFS distributed filesystem.
 
+Starting from 9.0, this functionality has been moved from Solr Core to the HDFS contrib module. 

Review comment:
       Not "contrib"

##########
File path: gradle/validation/validate-source-patterns.gradle
##########
@@ -108,6 +108,11 @@ allprojects {
 configure(project(':solr:core')) {
   project.tasks.withType(ValidateSourcePatternsTask) {
     sourceFiles.exclude 'src/**/CheckLoggingConfiguration.java'
+  }
+}
+
+configure(project(':solr:modules:hdfs')) {
+  project.tasks.withType(ValidateSourcePatternsTask) {

Review comment:
       just an observation: this will go away with the Hadoop upgrade that will follow, if I recall




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r790347582



##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+    assertEquals(3, allDocs);
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void createCollection(String collection,
+                                List<SolrClient> collectionClients, String baseUrl, int num) {
+    createSolrCore(collection, collectionClients, baseUrl, num, null);
+  }
+  
+  private void createSolrCore(final String collection,
+      List<SolrClient> collectionClients, final String baseUrl, final int num,
+      final String shardId) {
+    Callable<Object> call = () -> {
+      try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
+        // client.setConnectionTimeout(15000);
+        Create createCmd = new Create();
+        createCmd.setRoles("none");
+        createCmd.setCoreName(collection + num);
+        createCmd.setCollection(collection);
+
+        if (random().nextBoolean()) {
+          // sometimes we use an explicit core node name
+          createCmd.setCoreNodeName("anode" + nodeCounter.incrementAndGet());
+        }
+
+        if (shardId == null) {
+          createCmd.setNumShards(2);
+        }
+        createCmd.setDataDir(getDataDir(createTempDir(collection).toFile().getAbsolutePath()));
+        if (shardId != null) {
+          createCmd.setShardId(shardId);
+        }
+        client.request(createCmd);
+      } catch (Exception e) {
+        e.printStackTrace();
+        //fail
+      }
+      return null;
+    };
+
+    pending.add(completionService.submit(call));
+
+
+    collectionClients.add(createNewSolrClient(collection + num, baseUrl));
+  }
+
+  private void testMultipleCollections() throws Exception {
+    log.info("### STARTING testMultipleCollections");
+    // create another 2 collections and search across them
+    createNewCollection("collection2");
+    createNewCollection("collection3");
+
+    while (pending != null && pending.size() > 0) {
+
+      Future<Object> future = completionService.take();
+      if (future == null) return;
+      pending.remove(future);
+    }
+
+    indexDoc("collection2", getDoc(id, "10000000"));
+    indexDoc("collection2", getDoc(id, "10000001"));
+    indexDoc("collection2", getDoc(id, "10000003"));
+    getCommonCloudSolrClient().setDefaultCollection("collection2");
+    getCommonCloudSolrClient().add(getDoc(id, "10000004"));
+    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    indexDoc("collection3", getDoc(id, "20000000"));
+    indexDoc("collection3", getDoc(id, "20000001"));
+    getCommonCloudSolrClient().setDefaultCollection("collection3");
+    getCommonCloudSolrClient().add(getDoc(id, "10000005"));
+    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    otherCollectionClients.get("collection2").get(0).commit();
+    otherCollectionClients.get("collection3").get(0).commit();
+
+    getCommonCloudSolrClient().setDefaultCollection("collection1");
+    long collection1Docs = getCommonCloudSolrClient().query(new SolrQuery("*:*")).getResults()
+        .getNumFound();
+
+    long collection2Docs = otherCollectionClients.get("collection2").get(0)
+        .query(new SolrQuery("*:*")).getResults().getNumFound();
+
+    long collection3Docs = otherCollectionClients.get("collection3").get(0)
+        .query(new SolrQuery("*:*")).getResults().getNumFound();
+
+
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("collection", "collection2,collection3");
+    long found = clients.get(0).query(query).getResults().getNumFound();
+    assertEquals(collection2Docs + collection3Docs, found);
+
+    query = new SolrQuery("*:*");
+    query.set("collection", "collection1,collection2,collection3");
+    found = clients.get(0).query(query).getResults().getNumFound();
+    assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+    // try to search multiple with cloud client
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+    query.set("collection", "collection2,collection3");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection2Docs + collection3Docs, found);
+
+    query.set("collection", "collection3");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection3Docs, found);
+
+    query.remove("collection");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection1Docs, found);
+
+    assertEquals(collection3Docs, collection2Docs - 1);
+  }
+
+  protected void indexDoc(String collection, SolrInputDocument doc) throws IOException, SolrServerException {
+    List<SolrClient> clients = otherCollectionClients.get(collection);
+    int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) % clients.size();
+    SolrClient client = clients.get(which);
+    client.add(doc);
+  }
+
+  @SuppressWarnings({"unchecked"})
+  private void createNewCollection(final String collection) throws InterruptedException {
+    try {
+      assertEquals(0, CollectionAdminRequest
+          .createCollection(collection, "conf1", 2, 1)
+          .setCreateNodeSet("")
+          .process(cloudClient).getStatus());
+    } catch (Exception e) {
+      e.printStackTrace();
+      //fails
+    }
+    final List<SolrClient> collectionClients = new ArrayList<>();
+    otherCollectionClients.put(collection, collectionClients);
+    int unique = 0 ;
+    for (final JettySolrRunner runner : jettys) {
+      unique++;
+      final int frozeUnique = unique;
+      Callable<Object> call = () -> {
+
+        try {
+          assertTrue(CollectionAdminRequest
+              .addReplicaToShard(collection, "shard"+ ((frozeUnique%2)+1))
+              .setNode(runner.getNodeName())
+              .process(cloudClient).isSuccess());
+        } catch (Exception e) {
+          e.printStackTrace();

Review comment:
       *CatchAndPrintStackTrace:*  Logging or rethrowing exceptions should usually be preferred to catching and calling printStackTrace [(details)](https://errorprone.info/bugpattern/CatchAndPrintStackTrace)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+    assertEquals(3, allDocs);
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void createCollection(String collection,
+                                List<SolrClient> collectionClients, String baseUrl, int num) {
+    createSolrCore(collection, collectionClients, baseUrl, num, null);
+  }
+  
+  private void createSolrCore(final String collection,
+      List<SolrClient> collectionClients, final String baseUrl, final int num,
+      final String shardId) {
+    Callable<Object> call = () -> {
+      try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
+        // client.setConnectionTimeout(15000);
+        Create createCmd = new Create();
+        createCmd.setRoles("none");
+        createCmd.setCoreName(collection + num);
+        createCmd.setCollection(collection);
+
+        if (random().nextBoolean()) {
+          // sometimes we use an explicit core node name
+          createCmd.setCoreNodeName("anode" + nodeCounter.incrementAndGet());
+        }
+
+        if (shardId == null) {
+          createCmd.setNumShards(2);
+        }
+        createCmd.setDataDir(getDataDir(createTempDir(collection).toFile().getAbsolutePath()));
+        if (shardId != null) {
+          createCmd.setShardId(shardId);
+        }
+        client.request(createCmd);
+      } catch (Exception e) {
+        e.printStackTrace();

Review comment:
       *CatchAndPrintStackTrace:*  Logging or rethrowing exceptions should usually be preferred to catching and calling printStackTrace [(details)](https://errorprone.info/bugpattern/CatchAndPrintStackTrace)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+    assertEquals(3, allDocs);
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void createCollection(String collection,
+                                List<SolrClient> collectionClients, String baseUrl, int num) {
+    createSolrCore(collection, collectionClients, baseUrl, num, null);
+  }
+  
+  private void createSolrCore(final String collection,
+      List<SolrClient> collectionClients, final String baseUrl, final int num,
+      final String shardId) {
+    Callable<Object> call = () -> {
+      try (HttpSolrClient client = getHttpSolrClient(baseUrl)) {
+        // client.setConnectionTimeout(15000);
+        Create createCmd = new Create();
+        createCmd.setRoles("none");
+        createCmd.setCoreName(collection + num);
+        createCmd.setCollection(collection);
+
+        if (random().nextBoolean()) {
+          // sometimes we use an explicit core node name
+          createCmd.setCoreNodeName("anode" + nodeCounter.incrementAndGet());
+        }
+
+        if (shardId == null) {
+          createCmd.setNumShards(2);
+        }
+        createCmd.setDataDir(getDataDir(createTempDir(collection).toFile().getAbsolutePath()));
+        if (shardId != null) {
+          createCmd.setShardId(shardId);
+        }
+        client.request(createCmd);
+      } catch (Exception e) {
+        e.printStackTrace();
+        //fail
+      }
+      return null;
+    };
+
+    pending.add(completionService.submit(call));
+
+
+    collectionClients.add(createNewSolrClient(collection + num, baseUrl));
+  }
+
+  private void testMultipleCollections() throws Exception {
+    log.info("### STARTING testMultipleCollections");
+    // create another 2 collections and search across them
+    createNewCollection("collection2");
+    createNewCollection("collection3");
+
+    while (pending != null && pending.size() > 0) {
+
+      Future<Object> future = completionService.take();
+      if (future == null) return;
+      pending.remove(future);
+    }
+
+    indexDoc("collection2", getDoc(id, "10000000"));
+    indexDoc("collection2", getDoc(id, "10000001"));
+    indexDoc("collection2", getDoc(id, "10000003"));
+    getCommonCloudSolrClient().setDefaultCollection("collection2");
+    getCommonCloudSolrClient().add(getDoc(id, "10000004"));
+    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    indexDoc("collection3", getDoc(id, "20000000"));
+    indexDoc("collection3", getDoc(id, "20000001"));
+    getCommonCloudSolrClient().setDefaultCollection("collection3");
+    getCommonCloudSolrClient().add(getDoc(id, "10000005"));
+    getCommonCloudSolrClient().setDefaultCollection(null);
+
+    otherCollectionClients.get("collection2").get(0).commit();
+    otherCollectionClients.get("collection3").get(0).commit();
+
+    getCommonCloudSolrClient().setDefaultCollection("collection1");
+    long collection1Docs = getCommonCloudSolrClient().query(new SolrQuery("*:*")).getResults()
+        .getNumFound();
+
+    long collection2Docs = otherCollectionClients.get("collection2").get(0)
+        .query(new SolrQuery("*:*")).getResults().getNumFound();
+
+    long collection3Docs = otherCollectionClients.get("collection3").get(0)
+        .query(new SolrQuery("*:*")).getResults().getNumFound();
+
+
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("collection", "collection2,collection3");
+    long found = clients.get(0).query(query).getResults().getNumFound();
+    assertEquals(collection2Docs + collection3Docs, found);
+
+    query = new SolrQuery("*:*");
+    query.set("collection", "collection1,collection2,collection3");
+    found = clients.get(0).query(query).getResults().getNumFound();
+    assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+    // try to search multiple with cloud client
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection1Docs + collection2Docs + collection3Docs, found);
+
+    query.set("collection", "collection2,collection3");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection2Docs + collection3Docs, found);
+
+    query.set("collection", "collection3");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection3Docs, found);
+
+    query.remove("collection");
+    found = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+    assertEquals(collection1Docs, found);
+
+    assertEquals(collection3Docs, collection2Docs - 1);
+  }
+
+  protected void indexDoc(String collection, SolrInputDocument doc) throws IOException, SolrServerException {
+    List<SolrClient> clients = otherCollectionClients.get(collection);
+    int which = (doc.getField(id).toString().hashCode() & 0x7fffffff) % clients.size();
+    SolrClient client = clients.get(which);
+    client.add(doc);
+  }
+
+  @SuppressWarnings({"unchecked"})
+  private void createNewCollection(final String collection) throws InterruptedException {
+    try {
+      assertEquals(0, CollectionAdminRequest
+          .createCollection(collection, "conf1", 2, 1)
+          .setCreateNodeSet("")
+          .process(cloudClient).getStatus());
+    } catch (Exception e) {
+      e.printStackTrace();

Review comment:
       *CatchAndPrintStackTrace:*  Logging or rethrowing exceptions should usually be preferred to catching and calling printStackTrace [(details)](https://errorprone.info/bugpattern/CatchAndPrintStackTrace)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeySafeLeaderTestBase.java
##########
@@ -0,0 +1,211 @@
+/*
+ * 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.cloud;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractChaosMonkeySafeLeaderTestBase extends AbstractFullDistribZkTestBase {
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+
+  @BeforeClass
+  public static void beforeSuperClass() {
+    schemaString = "schema15.xml";      // we need a string id
+    System.setProperty("solr.autoCommit.maxTime", "15000");
+    System.clearProperty("solr.httpclient.retries");
+    System.clearProperty("solr.retries.on.forward");
+    System.clearProperty("solr.retries.to.followers");
+    setErrorHook();
+  }
+
+  @AfterClass
+  public static void afterSuperClass() {
+    System.clearProperty("solr.autoCommit.maxTime");
+    clearErrorHook();
+  }
+
+  protected static final String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
+  protected static final RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
+
+  public String[] getFieldNames() {
+    return fieldNames;
+  }
+
+  public RandVal[] getRandValues() {
+    return randVals;
+  }
+
+  protected abstract String getDirectoryFactory();
+  @Override
+  public void distribSetUp() throws Exception {
+    useFactory(getDirectoryFactory());
+    super.distribSetUp();
+  }
+
+  public AbstractChaosMonkeySafeLeaderTestBase() {
+    super();
+    sliceCount = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.slicecount", "-1"));
+    if (sliceCount == -1) {
+      sliceCount = random().nextInt(TEST_NIGHTLY ? 5 : 3) + 1;
+    }
+
+    int numShards = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.shardcount", "-1"));
+    if (numShards == -1) {
+      // we make sure that there's at least one shard with more than one replica
+      // so that the ChaosMonkey has something to kill
+      numShards = sliceCount + random().nextInt(TEST_NIGHTLY ? 12 : 2) + 1;
+    }
+    fixShardCount(numShards);
+  }
+
+  @Test
+  public void test() throws Exception {
+
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    // randomly turn on 1 seconds 'soft' commit
+    randomlyEnableAutoSoftCommit();
+
+    tryDelete();
+
+    List<StoppableIndexingThread> threads = new ArrayList<>();
+    int threadCount = 2;
+    int batchSize = 1;
+    if (random().nextBoolean()) {
+      batchSize = random().nextInt(98) + 2;
+    }
+
+    boolean pauseBetweenUpdates = TEST_NIGHTLY ? random().nextBoolean() : true;
+    int maxUpdates = -1;
+    if (!pauseBetweenUpdates) {
+      maxUpdates = 1000 + random().nextInt(1000);
+    } else {
+      maxUpdates = 15000;
+    }
+
+    for (int i = 0; i < threadCount; i++) {
+      StoppableIndexingThread indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true, maxUpdates, batchSize, pauseBetweenUpdates); // random().nextInt(999) + 1
+      threads.add(indexThread);
+      indexThread.start();
+    }
+
+    chaosMonkey.startTheMonkey(false, 500);
+    try {
+      long runLength;
+      if (RUN_LENGTH != -1) {
+        runLength = RUN_LENGTH;
+      } else {
+        int[] runTimes;
+        if (TEST_NIGHTLY) {
+          runTimes = new int[] {5000, 6000, 10000, 15000, 25000, 30000,
+              30000, 45000, 90000, 120000};
+        } else {
+          runTimes = new int[] {5000, 7000, 15000};
+        }
+        runLength = runTimes[random().nextInt(runTimes.length - 1)];
+      }
+
+      Thread.sleep(runLength);
+    } finally {
+      chaosMonkey.stopTheMonkey();
+    }
+
+    for (StoppableIndexingThread indexThread : threads) {
+      indexThread.safeStop();
+    }
+
+    // wait for stop...
+    for (StoppableIndexingThread indexThread : threads) {
+      indexThread.join();
+    }
+
+    for (StoppableIndexingThread indexThread : threads) {
+      assertEquals(0, indexThread.getFailCount());
+    }
+
+    // try and wait for any replications and what not to finish...
+
+    Thread.sleep(2000);
+
+    waitForThingsToLevelOut(3, TimeUnit.MINUTES);
+
+    // even if things were leveled out, a jetty may have just been stopped or something
+    // we wait again and wait to level out again to make sure the system is not still in flux
+
+    Thread.sleep(3000);
+
+    waitForThingsToLevelOut(3, TimeUnit.MINUTES);
+
+    checkShardConsistency(batchSize == 1, true);
+
+    if (VERBOSE) System.out.println("control docs:" + controlClient.query(new SolrQuery("*:*")).getResults().getNumFound() + "\n\n");
+
+    // try and make a collection to make sure the overseer has survived the expiration and session loss
+
+    // sometimes we restart zookeeper as well
+    if (random().nextBoolean()) {
+      zkServer.shutdown();
+      zkServer = new ZkTestServer(zkServer.getZkDir(), zkServer.getPort());
+      zkServer.run(false);
+    }
+
+    try (CloudSolrClient client = createCloudClient("collection1")) {
+      createCollection(null, "testcollection", 1, 1, client, null, "conf1");
+
+    }
+    List<Integer> numShardsNumReplicas = new ArrayList<>(2);
+    numShardsNumReplicas.add(1);
+    numShardsNumReplicas.add(1);
+    checkForCollection("testcollection",numShardsNumReplicas, null);
+  }
+
+  private void tryDelete() throws Exception {
+    long start = System.nanoTime();
+    long timeout = start + TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
+    while (System.nanoTime() < timeout) {
+      try {
+        del("*:*");
+        break;
+      } catch (SolrServerException e) {
+        // cluster may not be up yet
+        e.printStackTrace();

Review comment:
       *CatchAndPrintStackTrace:*  Logging or rethrowing exceptions should usually be preferred to catching and calling printStackTrace [(details)](https://errorprone.info/bugpattern/CatchAndPrintStackTrace)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractSyncSliceTestBase.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.cloud;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test sync phase that occurs when Leader goes down and a new Leader is
+ * elected.
+ */
+public abstract class AbstractSyncSliceTestBase extends AbstractFullDistribZkTestBase {
+  private boolean success = false;
+
+  @Override
+  public void distribTearDown() throws Exception {
+    if (!success) {
+      printLayoutOnTearDown = true;
+    }
+    super.distribTearDown();
+  }
+
+  public AbstractSyncSliceTestBase() {
+    super();
+    sliceCount = 1;
+    fixShardCount(TEST_NIGHTLY ? 7 : 4);
+  }
+
+  @Test
+  public void test() throws Exception {
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+    
+    waitForThingsToLevelOut(30, TimeUnit.SECONDS);
+
+    del("*:*");
+    List<CloudJettyRunner> skipServers = new ArrayList<>();
+    int docId = 0;
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "to come to the aid of their country.");
+    
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "old haven was blue.");
+    
+    skipServers.add(shardToJetty.get("shard1").get(1));
+    
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "but the song was fancy.");
+    
+    skipServers.add(shardToJetty.get("shard1").get(2));
+    
+    indexDoc(skipServers, id,docId++, i1, 50, tlong, 50, t1,
+        "under the moon and over the lake");
+    
+    commit();
+    
+    waitForRecoveriesToFinish(false);
+
+    // shard should be inconsistent
+    String shardFailMessage = checkShardConsistency("shard1", true, false);
+    assertNotNull(shardFailMessage);
+    
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.SYNCSHARD.toString());
+    params.set("collection", "collection1");
+    params.set("shard", "shard1");
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    
+    String baseUrl = ((HttpSolrClient) shardToJetty.get("shard1").get(2).client.solrClient)
+        .getBaseURL();
+    baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
+    
+    // we only set the connect timeout, not so timeout
+    try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl, 30000)) {
+      baseClient.request(request);
+    }
+
+    waitForThingsToLevelOut(15, TimeUnit.SECONDS);
+    
+    checkShardConsistency(false, true);
+    
+    long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    assertEquals(4, cloudClientDocs);
+    
+    
+    // kill the leader - new leader could have all the docs or be missing one
+    CloudJettyRunner leaderJetty = shardToLeaderJetty.get("shard1");
+    
+    skipServers = getRandomOtherJetty(leaderJetty, null); // but not the leader
+    
+    // this doc won't be on one node
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "to come to the aid of their country.");
+    commit();
+    
+    
+    Set<CloudJettyRunner> jetties = new HashSet<>();
+    jetties.addAll(shardToJetty.get("shard1"));
+    jetties.remove(leaderJetty);
+    assertEquals(getShardCount() - 1, jetties.size());
+    
+    leaderJetty.jetty.stop();
+    
+    Thread.sleep(3000);
+    
+    waitForNoShardInconsistency();
+    
+    Thread.sleep(1000);
+    
+    checkShardConsistency(false, true);
+    
+    cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    assertEquals(5, cloudClientDocs);
+    
+    CloudJettyRunner deadJetty = leaderJetty;
+    
+    // let's get the latest leader
+    while (deadJetty == leaderJetty) {
+      updateMappingsFromZk(this.jettys, this.clients);
+      leaderJetty = shardToLeaderJetty.get("shard1");
+    }
+    
+    // bring back dead node
+    deadJetty.jetty.start(); // he is not the leader anymore
+    
+    waitTillAllNodesActive();
+    
+    skipServers = getRandomOtherJetty(leaderJetty, deadJetty);
+    skipServers.addAll( getRandomOtherJetty(leaderJetty, deadJetty));
+    // skip list should be 
+    
+//    System.out.println("leader:" + leaderJetty.url);
+//    System.out.println("dead:" + deadJetty.url);
+//    System.out.println("skip list:" + skipServers);
+    
+    // we are skipping  2 nodes
+    assertEquals(2, skipServers.size());
+    
+    // more docs than can peer sync
+    for (int i = 0; i < 300; i++) {
+      indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+          "to come to the aid of their country.");
+    }
+    
+    commit();
+    
+    Thread.sleep(1000);
+    
+    waitForRecoveriesToFinish(false);
+    
+    // shard should be inconsistent
+    shardFailMessage = waitTillInconsistent();
+    assertNotNull(
+        "Test Setup Failure: shard1 should have just been set up to be inconsistent - but it's still consistent. Leader:"
+            + leaderJetty.url + " Dead Guy:" + deadJetty.url + "skip list:" + skipServers, shardFailMessage);
+    
+    // good place to test compareResults
+    boolean shouldFail = CloudInspectUtil.compareResults(controlClient, cloudClient);
+    assertTrue("A test that compareResults is working correctly failed", shouldFail);
+    
+    jetties = new HashSet<>();
+    jetties.addAll(shardToJetty.get("shard1"));
+    jetties.remove(leaderJetty);
+    assertEquals(getShardCount() - 1, jetties.size());
+
+    
+    // kill the current leader
+    leaderJetty.jetty.stop();
+    
+    waitForNoShardInconsistency();
+
+    checkShardConsistency(true, true);
+    
+    success = true;
+  }
+
+  private void waitTillAllNodesActive() throws Exception {
+    for (int i = 0; i < 60; i++) { 
+      Thread.sleep(3000);
+      ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+      ClusterState clusterState = zkStateReader.getClusterState();
+      DocCollection collection1 = clusterState.getCollection("collection1");
+      Slice slice = collection1.getSlice("shard1");
+      Collection<Replica> replicas = slice.getReplicas();
+      boolean allActive = true;
+      for (Replica replica : replicas) {
+        if (!clusterState.liveNodesContain(replica.getNodeName()) || replica.getState() != Replica.State.ACTIVE) {
+          allActive = false;
+          break;
+        }
+      }
+      if (allActive) {
+        return;
+      }
+    }
+    printLayout();
+    fail("timeout waiting to see all nodes active");
+  }
+
+  private String waitTillInconsistent() throws Exception, InterruptedException {
+    String shardFailMessage = null;
+    
+    shardFailMessage = pollConsistency(shardFailMessage, 0);
+    shardFailMessage = pollConsistency(shardFailMessage, 3000);
+    shardFailMessage = pollConsistency(shardFailMessage, 5000);
+    shardFailMessage = pollConsistency(shardFailMessage, 15000);
+    
+    return shardFailMessage;
+  }
+
+  private String pollConsistency(String shardFailMessage, int sleep)
+      throws InterruptedException, Exception {
+    try {
+      commit();
+    } catch (Throwable t) {
+      t.printStackTrace();

Review comment:
       *CatchAndPrintStackTrace:*  Logging or rethrowing exceptions should usually be preferred to catching and calling printStackTrace [(details)](https://errorprone.info/bugpattern/CatchAndPrintStackTrace)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {

Review comment:
       *CatchFail:*  Ignoring exceptions and calling fail() is unnecessary, and makes test output less useful [(details)](https://errorprone.info/bugpattern/CatchFail)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";

Review comment:
       *HidingField:*  Hiding fields of superclasses may cause confusion and errors. This field is hiding a field of the same name in superclass: AbstractFullDistribZkTestBase [(details)](https://errorprone.info/bugpattern/HidingField)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";

Review comment:
       *HidingField:*  Hiding fields of superclasses may cause confusion and errors. This field is hiding a field of the same name in superclass: AbstractFullDistribZkTestBase [(details)](https://errorprone.info/bugpattern/HidingField)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";

Review comment:
       *HidingField:*  Hiding fields of superclasses may cause confusion and errors. This field is hiding a field of the same name in superclass: AbstractFullDistribZkTestBase [(details)](https://errorprone.info/bugpattern/HidingField)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";

Review comment:
       *HidingField:*  Hiding fields of superclasses may cause confusion and errors. This field is hiding a field of the same name in superclass: AbstractFullDistribZkTestBase [(details)](https://errorprone.info/bugpattern/HidingField)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";

Review comment:
       *HidingField:*  Hiding fields of superclasses may cause confusion and errors. This field is hiding a field of the same name in superclass: AbstractFullDistribZkTestBase [(details)](https://errorprone.info/bugpattern/HidingField)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *InvalidBlockTag:*  Tag name `lucene.experimental` is unknown. If this is a commonly-used custom tag, please click 'not useful' and file a bug. [(details)](https://errorprone.info/bugpattern/InvalidBlockTag)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *InvalidBlockTag:*  Tag name `lucene.experimental` is unknown. If this is a commonly-used custom tag, please click 'not useful' and file a bug. [(details)](https://errorprone.info/bugpattern/InvalidBlockTag)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  abstract String getConfigSet();
+
+  @Before
+  public void setupCluster() throws Exception {
+    // we don't want this test to have zk timeouts
+    System.setProperty("zkClientTimeout", "60000");
+    System.setProperty("createCollectionWaitTimeTillActive", "5");
+    TestInjection.randomDelayInCoreCreation = "true:5";
+    System.setProperty("validateAfterInactivity", "200");
+    System.setProperty("solr.allowPaths", "*");
+
+    configureCluster(4)
+        .addConfig("conf", configset(getConfigSet()))
+        .addConfig("conf2", configset(getConfigSet()))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+  }
+  
+  @After
+  public void tearDownCluster() throws Exception {
+    try {
+      shutdownCluster();
+    } finally {
+      System.clearProperty("createCollectionWaitTimeTillActive");
+      System.clearProperty("solr.allowPaths");
+      super.tearDown();
+    }
+  }
+
+  @Test
+  public void testCreationAndDeletion() throws Exception {
+    String collectionName = "created_and_deleted";
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+        .contains(collectionName));
+
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deleteCollectionRemovesStaleZkCollectionsNode() throws Exception {
+    String collectionName = "out_of_sync_collection";
+
+    // manually create a collections zknode
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+    
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deletePartiallyCreatedCollection() throws Exception {
+    final String collectionName = "halfdeletedcollection";
+
+    assertEquals(0, CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    String dataDir = createTempDir().toFile().getAbsolutePath();
+    // create a core that simulates something left over from a partially-deleted collection
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard(collectionName, "shard1")
+        .setDataDir(dataDir)
+        .process(cluster.getSolrClient()).isSuccess());
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void deleteCollectionOnlyInZk() throws Exception {
+    final String collectionName = "onlyinzk";
+
+    // create the collections node, but nothing else
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    // delete via API - should remove collections node
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+    
+    // now creating that collection should work
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void testBadActionNames() {
+    // try a bad action
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", "BADACTION");
+    String collectionName = "badactioncollection";
+    params.set("name", collectionName);
+    params.set("numShards", 2);
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingRequiredParameters() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("numShards", 2);
+    // missing required collection parameter
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingNumShards() {
+    // No numShards should fail
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testZeroNumShards() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("numShards", 0);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testCreateShouldFailOnExistingCore() throws Exception {
+    String nn1 = cluster.getJettySolrRunner(0).getNodeName();
+    String nn2 = cluster.getJettySolrRunner(1).getNodeName();
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker", "conf", 1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker", "shard1")
+        .setNode(cluster.getJettySolrRunner(0).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker2", "conf",1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker2", "shard1")
+        .setNode(cluster.getJettySolrRunner(1).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> {
+      CollectionAdminRequest.createCollection("halfcollection", "conf", 1, 1)
+          .setCreateNodeSet(nn1 + "," + nn2)
+          .process(cluster.getSolrClient());
+    });
+  }
+
+  @Test
+  public void testNoConfigSetExist() throws Exception {
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.createCollection("noconfig", "conf123", 1, 1)
+          .process(cluster.getSolrClient());
+    });
+
+    TimeUnit.MILLISECONDS.sleep(1000);
+    // in both cases, the collection should have default to the core name
+    cluster.getSolrClient().getZkStateReader().forceUpdateCollection("noconfig");
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains("noconfig"));
+  }
+
+  @Test
+  public void testCoresAreDistributedAcrossNodes() throws Exception {
+    CollectionAdminRequest.createCollection("nodes_used_collection", "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    Set<String> liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes();
+
+    List<String> createNodeList = new ArrayList<>(liveNodes);
+
+    DocCollection collection = getCollectionState("nodes_used_collection");
+    for (Slice slice : collection.getSlices()) {
+      for (Replica replica : slice.getReplicas()) {
+        createNodeList.remove(replica.getNodeName());
+      }
+    }
+
+    assertEquals(createNodeList.toString(), 0, createNodeList.size());
+  }
+
+  @Test
+  public void testDeleteNonExistentCollection() throws Exception {
+
+    expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.deleteCollection("unknown_collection").process(cluster.getSolrClient());
+    });
+
+    // create another collection should still work
+    CollectionAdminRequest.createCollection("acollectionafterbaddelete", "conf", 1, 2)
+        .process(cluster.getSolrClient());
+    waitForState("Collection creation after a bad delete failed", "acollectionafterbaddelete",
+        (n, c) -> DocCollection.isFullyActive(n, c, 1, 2));
+  }
+
+  @Test
+  public void testSpecificConfigsets() throws Exception {
+    CollectionAdminRequest.createCollection("withconfigset2", "conf2", 1, 1).process(cluster.getSolrClient());
+    String configName  = cluster.getSolrClient().getClusterStateProvider().getCollection("withconfigset2").getConfigName();
+    assertEquals("conf2", configName);
+  }
+
+  @Test
+  public void testCreateNodeSet() throws Exception {
+    JettySolrRunner jetty1 = cluster.getRandomJetty(random());
+    JettySolrRunner jetty2 = cluster.getRandomJetty(random());
+
+    List<String> baseUrls = ImmutableList.of(jetty1.getBaseUrl().toString(), jetty2.getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection("nodeset_collection", "conf", 2, 1)
+        .setCreateNodeSet(baseUrls.get(0) + "," + baseUrls.get(1))
+        .process(cluster.getSolrClient());
+
+    DocCollection collectionState = getCollectionState("nodeset_collection");
+    for (Replica replica : collectionState.getReplicas()) {
+      String replicaUrl = replica.getCoreUrl();
+      boolean matchingJetty = false;
+      for (String jettyUrl : baseUrls) {
+        if (replicaUrl.startsWith(jettyUrl)) {
+          matchingJetty = true;
+        }
+      }
+      if (matchingJetty == false) {
+        fail("Expected replica to be on " + baseUrls + " but was on " + replicaUrl);
+      }
+    }
+  }
+
+  @Test
+  //28-June-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  // See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows machines occasionally
+  // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 09-Aug-2018 SOLR-12028
+  public void testCollectionsAPI() throws Exception {
+
+    // create new collections rapid fire
+    int cnt = random().nextInt(TEST_NIGHTLY ? 3 : 1) + 1;
+    CollectionAdminRequest.Create[] createRequests = new CollectionAdminRequest.Create[cnt];
+    
+    class Coll {
+      String name;
+      int numShards;
+      int replicationFactor;
+    }
+    
+    List<Coll> colls = new ArrayList<>();
+
+    for (int i = 0; i < cnt; i++) {
+
+      int numShards = TestUtil.nextInt(random(), 0, cluster.getJettySolrRunners().size()) + 1;
+      int replicationFactor = TestUtil.nextInt(random(), 0, 3) + 1;
+
+      createRequests[i]
+          = CollectionAdminRequest.createCollection("awhollynewcollection_" + i, "conf2", numShards, replicationFactor);
+      createRequests[i].processAsync(cluster.getSolrClient());
+      
+      Coll coll = new Coll();
+      coll.name = "awhollynewcollection_" + i;
+      coll.numShards = numShards;
+      coll.replicationFactor = replicationFactor;
+      colls.add(coll);
+    }
+
+    for (Coll coll : colls) {
+      cluster.waitForActiveCollection(coll.name, coll.numShards, coll.numShards * coll.replicationFactor);
+    }
+
+    waitForStable(cnt, createRequests);
+
+    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
+      checkInstanceDirs(cluster.getJettySolrRunner(i));
+    }
+    
+    String collectionName = createRequests[random().nextInt(createRequests.length)].getCollectionName();
+    
+    // TODO: we should not need this...beast test well when trying to fix
+    Thread.sleep(1000);
+    
+    cluster.getSolrClient().getZkStateReader().forciblyRefreshAllClusterStateSlow();
+
+    new UpdateRequest()
+        .add("id", "6")
+        .add("id", "7")
+        .add("id", "8")
+        .commit(cluster.getSolrClient(), collectionName);
+    long numFound = 0;
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+
+      numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
+      if (numFound == 3) {
+        break;
+      }
+
+      Thread.sleep(500);
+    }
+    
+    if (timeOut.hasTimedOut()) {
+      fail("Timeout waiting to see 3 found, instead saw " + numFound + " for collection " + collectionName);
+    }
+
+    checkNoTwoShardsUseTheSameIndexDir();
+  }
+
+  private void waitForStable(int cnt, CollectionAdminRequest.Create[] createRequests) throws InterruptedException {
+    for (int i = 0; i < cnt; i++) {
+      String collectionName = "awhollynewcollection_" + i;
+      final int j = i;
+      waitForState("Expected to see collection " + collectionName, collectionName,
+          (n, c) -> {
+            CollectionAdminRequest.Create req = createRequests[j];
+            return DocCollection.isFullyActive(n, c, req.getNumShards(), req.getReplicationFactor());
+          });
+      
+      ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+      // make sure we have leaders for each shard
+      for (int z = 1; z < createRequests[j].getNumShards(); z++) {
+        zkStateReader.getLeaderRetry(collectionName, "shard" + z, 10000);
+      }      // make sure we again have leaders for each shard
+    }
+  }
+
+  @Test
+  public void testCollectionReload() throws Exception {
+    final String collectionName = "reloaded_collection";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2).process(cluster.getSolrClient());
+
+    // get core open times
+    Map<String, Long> urlToTimeBefore = new HashMap<>();
+    collectStartTimes(collectionName, urlToTimeBefore);
+    assertTrue(urlToTimeBefore.size() > 0);
+
+    CollectionAdminRequest.reloadCollection(collectionName).processAsync(cluster.getSolrClient());
+
+    // reloads make take a short while
+    boolean allTimesAreCorrect = waitForReloads(collectionName, urlToTimeBefore);
+    assertTrue("some core start times did not change on reload", allTimesAreCorrect);
+  }
+
+  private void checkInstanceDirs(JettySolrRunner jetty) throws IOException {
+    CoreContainer cores = jetty.getCoreContainer();
+    Collection<SolrCore> theCores = cores.getCores();
+    for (SolrCore core : theCores) {
+      // look for core props file
+      Path instancedir = core.getInstancePath();
+      assertTrue("Could not find expected core.properties file", Files.exists(instancedir.resolve("core.properties")));
+
+      Path expected = Paths.get(jetty.getSolrHome()).toAbsolutePath().resolve(core.getName());
+
+      assertTrue("Expected: " + expected + "\nFrom core stats: " + instancedir, Files.isSameFile(expected, instancedir));
+    }
+  }
+
+  private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+
+    boolean allTimesAreCorrect = false;
+    while (! timeout.hasTimedOut()) {
+      Map<String,Long> urlToTimeAfter = new HashMap<>();
+      collectStartTimes(collectionName, urlToTimeAfter);
+      
+      boolean retry = false;
+      Set<Entry<String,Long>> entries = urlToTimeBefore.entrySet();
+      for (Entry<String,Long> entry : entries) {
+        Long beforeTime = entry.getValue();
+        Long afterTime = urlToTimeAfter.get(entry.getKey());
+        assertNotNull(afterTime);
+        if (afterTime <= beforeTime) {
+          retry = true;
+          break;
+        }
+
+      }
+      if (!retry) {
+        allTimesAreCorrect = true;
+        break;
+      }
+    }
+    return allTimesAreCorrect;
+  }
+
+  private void collectStartTimes(String collectionName, Map<String,Long> urlToTime)
+      throws SolrServerException, IOException {
+
+    DocCollection collectionState = getCollectionState(collectionName);
+    if (collectionState != null) {
+      for (Slice shard : collectionState) {
+        for (Replica replica : shard) {
+          ZkCoreNodeProps coreProps = new ZkCoreNodeProps(replica);
+          CoreStatus coreStatus;
+          try (HttpSolrClient server = getHttpSolrClient(coreProps.getBaseUrl())) {
+            coreStatus = CoreAdminRequest.getCoreStatus(coreProps.getCoreName(), false, server);
+          }
+          long before = coreStatus.getCoreStartTime().getTime();

Review comment:
       *JavaUtilDate:*  Date has a bad API that leads to bugs; prefer java.time.Instant or LocalDate. [(details)](https://errorprone.info/bugpattern/JavaUtilDate)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  abstract String getConfigSet();
+
+  @Before
+  public void setupCluster() throws Exception {
+    // we don't want this test to have zk timeouts
+    System.setProperty("zkClientTimeout", "60000");
+    System.setProperty("createCollectionWaitTimeTillActive", "5");
+    TestInjection.randomDelayInCoreCreation = "true:5";
+    System.setProperty("validateAfterInactivity", "200");
+    System.setProperty("solr.allowPaths", "*");
+
+    configureCluster(4)
+        .addConfig("conf", configset(getConfigSet()))
+        .addConfig("conf2", configset(getConfigSet()))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+  }
+  
+  @After
+  public void tearDownCluster() throws Exception {
+    try {
+      shutdownCluster();
+    } finally {
+      System.clearProperty("createCollectionWaitTimeTillActive");
+      System.clearProperty("solr.allowPaths");
+      super.tearDown();
+    }
+  }
+
+  @Test
+  public void testCreationAndDeletion() throws Exception {
+    String collectionName = "created_and_deleted";
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+        .contains(collectionName));
+
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deleteCollectionRemovesStaleZkCollectionsNode() throws Exception {
+    String collectionName = "out_of_sync_collection";
+
+    // manually create a collections zknode
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+    
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deletePartiallyCreatedCollection() throws Exception {
+    final String collectionName = "halfdeletedcollection";
+
+    assertEquals(0, CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    String dataDir = createTempDir().toFile().getAbsolutePath();
+    // create a core that simulates something left over from a partially-deleted collection
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard(collectionName, "shard1")
+        .setDataDir(dataDir)
+        .process(cluster.getSolrClient()).isSuccess());
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void deleteCollectionOnlyInZk() throws Exception {
+    final String collectionName = "onlyinzk";
+
+    // create the collections node, but nothing else
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    // delete via API - should remove collections node
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+    
+    // now creating that collection should work
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void testBadActionNames() {
+    // try a bad action
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", "BADACTION");
+    String collectionName = "badactioncollection";
+    params.set("name", collectionName);
+    params.set("numShards", 2);
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingRequiredParameters() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("numShards", 2);
+    // missing required collection parameter
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingNumShards() {
+    // No numShards should fail
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testZeroNumShards() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("numShards", 0);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testCreateShouldFailOnExistingCore() throws Exception {
+    String nn1 = cluster.getJettySolrRunner(0).getNodeName();
+    String nn2 = cluster.getJettySolrRunner(1).getNodeName();
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker", "conf", 1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker", "shard1")
+        .setNode(cluster.getJettySolrRunner(0).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker2", "conf",1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker2", "shard1")
+        .setNode(cluster.getJettySolrRunner(1).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> {
+      CollectionAdminRequest.createCollection("halfcollection", "conf", 1, 1)
+          .setCreateNodeSet(nn1 + "," + nn2)
+          .process(cluster.getSolrClient());
+    });
+  }
+
+  @Test
+  public void testNoConfigSetExist() throws Exception {
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.createCollection("noconfig", "conf123", 1, 1)
+          .process(cluster.getSolrClient());
+    });
+
+    TimeUnit.MILLISECONDS.sleep(1000);
+    // in both cases, the collection should have default to the core name
+    cluster.getSolrClient().getZkStateReader().forceUpdateCollection("noconfig");
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains("noconfig"));
+  }
+
+  @Test
+  public void testCoresAreDistributedAcrossNodes() throws Exception {
+    CollectionAdminRequest.createCollection("nodes_used_collection", "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    Set<String> liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes();
+
+    List<String> createNodeList = new ArrayList<>(liveNodes);
+
+    DocCollection collection = getCollectionState("nodes_used_collection");
+    for (Slice slice : collection.getSlices()) {
+      for (Replica replica : slice.getReplicas()) {
+        createNodeList.remove(replica.getNodeName());
+      }
+    }
+
+    assertEquals(createNodeList.toString(), 0, createNodeList.size());
+  }
+
+  @Test
+  public void testDeleteNonExistentCollection() throws Exception {
+
+    expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.deleteCollection("unknown_collection").process(cluster.getSolrClient());
+    });
+
+    // create another collection should still work
+    CollectionAdminRequest.createCollection("acollectionafterbaddelete", "conf", 1, 2)
+        .process(cluster.getSolrClient());
+    waitForState("Collection creation after a bad delete failed", "acollectionafterbaddelete",
+        (n, c) -> DocCollection.isFullyActive(n, c, 1, 2));
+  }
+
+  @Test
+  public void testSpecificConfigsets() throws Exception {
+    CollectionAdminRequest.createCollection("withconfigset2", "conf2", 1, 1).process(cluster.getSolrClient());
+    String configName  = cluster.getSolrClient().getClusterStateProvider().getCollection("withconfigset2").getConfigName();
+    assertEquals("conf2", configName);
+  }
+
+  @Test
+  public void testCreateNodeSet() throws Exception {
+    JettySolrRunner jetty1 = cluster.getRandomJetty(random());
+    JettySolrRunner jetty2 = cluster.getRandomJetty(random());
+
+    List<String> baseUrls = ImmutableList.of(jetty1.getBaseUrl().toString(), jetty2.getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection("nodeset_collection", "conf", 2, 1)
+        .setCreateNodeSet(baseUrls.get(0) + "," + baseUrls.get(1))
+        .process(cluster.getSolrClient());
+
+    DocCollection collectionState = getCollectionState("nodeset_collection");
+    for (Replica replica : collectionState.getReplicas()) {
+      String replicaUrl = replica.getCoreUrl();
+      boolean matchingJetty = false;
+      for (String jettyUrl : baseUrls) {
+        if (replicaUrl.startsWith(jettyUrl)) {
+          matchingJetty = true;
+        }
+      }
+      if (matchingJetty == false) {
+        fail("Expected replica to be on " + baseUrls + " but was on " + replicaUrl);
+      }
+    }
+  }
+
+  @Test
+  //28-June-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  // See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows machines occasionally
+  // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 09-Aug-2018 SOLR-12028
+  public void testCollectionsAPI() throws Exception {
+
+    // create new collections rapid fire
+    int cnt = random().nextInt(TEST_NIGHTLY ? 3 : 1) + 1;
+    CollectionAdminRequest.Create[] createRequests = new CollectionAdminRequest.Create[cnt];
+    
+    class Coll {
+      String name;
+      int numShards;
+      int replicationFactor;
+    }
+    
+    List<Coll> colls = new ArrayList<>();
+
+    for (int i = 0; i < cnt; i++) {
+
+      int numShards = TestUtil.nextInt(random(), 0, cluster.getJettySolrRunners().size()) + 1;
+      int replicationFactor = TestUtil.nextInt(random(), 0, 3) + 1;
+
+      createRequests[i]
+          = CollectionAdminRequest.createCollection("awhollynewcollection_" + i, "conf2", numShards, replicationFactor);
+      createRequests[i].processAsync(cluster.getSolrClient());
+      
+      Coll coll = new Coll();
+      coll.name = "awhollynewcollection_" + i;
+      coll.numShards = numShards;
+      coll.replicationFactor = replicationFactor;
+      colls.add(coll);
+    }
+
+    for (Coll coll : colls) {
+      cluster.waitForActiveCollection(coll.name, coll.numShards, coll.numShards * coll.replicationFactor);
+    }
+
+    waitForStable(cnt, createRequests);
+
+    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
+      checkInstanceDirs(cluster.getJettySolrRunner(i));
+    }
+    
+    String collectionName = createRequests[random().nextInt(createRequests.length)].getCollectionName();
+    
+    // TODO: we should not need this...beast test well when trying to fix
+    Thread.sleep(1000);
+    
+    cluster.getSolrClient().getZkStateReader().forciblyRefreshAllClusterStateSlow();
+
+    new UpdateRequest()
+        .add("id", "6")
+        .add("id", "7")
+        .add("id", "8")
+        .commit(cluster.getSolrClient(), collectionName);
+    long numFound = 0;
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+
+      numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
+      if (numFound == 3) {
+        break;
+      }
+
+      Thread.sleep(500);
+    }
+    
+    if (timeOut.hasTimedOut()) {
+      fail("Timeout waiting to see 3 found, instead saw " + numFound + " for collection " + collectionName);
+    }
+
+    checkNoTwoShardsUseTheSameIndexDir();
+  }
+
+  private void waitForStable(int cnt, CollectionAdminRequest.Create[] createRequests) throws InterruptedException {
+    for (int i = 0; i < cnt; i++) {
+      String collectionName = "awhollynewcollection_" + i;
+      final int j = i;
+      waitForState("Expected to see collection " + collectionName, collectionName,
+          (n, c) -> {
+            CollectionAdminRequest.Create req = createRequests[j];
+            return DocCollection.isFullyActive(n, c, req.getNumShards(), req.getReplicationFactor());
+          });
+      
+      ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+      // make sure we have leaders for each shard
+      for (int z = 1; z < createRequests[j].getNumShards(); z++) {
+        zkStateReader.getLeaderRetry(collectionName, "shard" + z, 10000);
+      }      // make sure we again have leaders for each shard
+    }
+  }
+
+  @Test
+  public void testCollectionReload() throws Exception {
+    final String collectionName = "reloaded_collection";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2).process(cluster.getSolrClient());
+
+    // get core open times
+    Map<String, Long> urlToTimeBefore = new HashMap<>();
+    collectStartTimes(collectionName, urlToTimeBefore);
+    assertTrue(urlToTimeBefore.size() > 0);
+
+    CollectionAdminRequest.reloadCollection(collectionName).processAsync(cluster.getSolrClient());
+
+    // reloads make take a short while
+    boolean allTimesAreCorrect = waitForReloads(collectionName, urlToTimeBefore);
+    assertTrue("some core start times did not change on reload", allTimesAreCorrect);
+  }
+
+  private void checkInstanceDirs(JettySolrRunner jetty) throws IOException {
+    CoreContainer cores = jetty.getCoreContainer();
+    Collection<SolrCore> theCores = cores.getCores();
+    for (SolrCore core : theCores) {
+      // look for core props file
+      Path instancedir = core.getInstancePath();
+      assertTrue("Could not find expected core.properties file", Files.exists(instancedir.resolve("core.properties")));
+
+      Path expected = Paths.get(jetty.getSolrHome()).toAbsolutePath().resolve(core.getName());
+
+      assertTrue("Expected: " + expected + "\nFrom core stats: " + instancedir, Files.isSameFile(expected, instancedir));
+    }
+  }
+
+  private boolean waitForReloads(String collectionName, Map<String,Long> urlToTimeBefore) throws SolrServerException, IOException {
+    TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+
+    boolean allTimesAreCorrect = false;
+    while (! timeout.hasTimedOut()) {
+      Map<String,Long> urlToTimeAfter = new HashMap<>();
+      collectStartTimes(collectionName, urlToTimeAfter);
+      
+      boolean retry = false;
+      Set<Entry<String,Long>> entries = urlToTimeBefore.entrySet();
+      for (Entry<String,Long> entry : entries) {
+        Long beforeTime = entry.getValue();
+        Long afterTime = urlToTimeAfter.get(entry.getKey());
+        assertNotNull(afterTime);
+        if (afterTime <= beforeTime) {
+          retry = true;
+          break;
+        }
+
+      }
+      if (!retry) {
+        allTimesAreCorrect = true;
+        break;
+      }
+    }
+    return allTimesAreCorrect;
+  }
+
+  private void collectStartTimes(String collectionName, Map<String,Long> urlToTime)
+      throws SolrServerException, IOException {
+
+    DocCollection collectionState = getCollectionState(collectionName);
+    if (collectionState != null) {
+      for (Slice shard : collectionState) {
+        for (Replica replica : shard) {
+          ZkCoreNodeProps coreProps = new ZkCoreNodeProps(replica);
+          CoreStatus coreStatus;
+          try (HttpSolrClient server = getHttpSolrClient(coreProps.getBaseUrl())) {
+            coreStatus = CoreAdminRequest.getCoreStatus(coreProps.getCoreName(), false, server);
+          }
+          long before = coreStatus.getCoreStartTime().getTime();
+          urlToTime.put(coreProps.getCoreUrl(), before);
+        }
+      }
+    } else {
+      throw new IllegalArgumentException("Could not find collection " + collectionName);
+    }
+  }
+  
+  private void checkNoTwoShardsUseTheSameIndexDir() {
+    Map<String, Set<String>> indexDirToShardNamesMap = new HashMap<>();
+    
+    List<MBeanServer> servers = new LinkedList<>();

Review comment:
       *JdkObsolete:*  It is very rare for LinkedList to out-perform ArrayList or ArrayDeque. Avoid it unless you're willing to invest a lot of time into benchmarking. Caveat: LinkedList supports null elements, but ArrayDeque does not. [(details)](https://errorprone.info/bugpattern/JdkObsolete)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractMoveReplicaTestBase.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.common.SolrInputDocument;
+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.util.NamedList;
+import org.apache.solr.util.IdUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractMoveReplicaTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  // used by MoveReplicaHDFSTest
+  protected boolean inPlaceMove = true;
+  protected boolean isCollectionApiDistributed = false;
+
+  protected String getConfigSet() {
+    return "cloud-dynamic";
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    inPlaceMove = true;
+
+    configureCluster(4)
+        .addConfig("conf1", configset(getConfigSet()))
+        .addConfig("conf2", configset(getConfigSet()))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+
+    // If Collection API is distributed let's not wait for Overseer.
+    if (isCollectionApiDistributed = new CollectionAdminRequest.RequestApiDistributedProcessing().process(cluster.getSolrClient()).getIsCollectionApiDistributed()) {

Review comment:
       *LogicalAssignment:*  Assignment where a boolean expression was expected; use == if this assignment wasn't expected or add parentheses for clarity. [(details)](https://errorprone.info/bugpattern/LogicalAssignment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *MissingSummary:*  A summary line is required on public/protected Javadocs. [(details)](https://google.github.io/styleguide/javaguide.html#s7.2-summary-fragment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       *MissingSummary:*  A summary line is required on public/protected Javadocs. [(details)](https://google.github.io/styleguide/javaguide.html#s7.2-summary-fragment)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.cloud;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Slow
+public abstract class AbstractRecoveryZkTestBase extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  private final List<StoppableIndexingThread> threads = new ArrayList<>();
+
+  @After
+  public void stopThreads() throws InterruptedException {
+    for (StoppableIndexingThread t : threads) {
+      t.safeStop();
+    }
+    for (StoppableIndexingThread t : threads) {
+      t.join();
+    }
+    threads.clear();
+  }
+
+  @Test
+  //commented 2-Aug-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 28-June-2018
+  public void test() throws Exception {
+
+    final String collection = "recoverytest";
+
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 2)
+        .process(cluster.getSolrClient());
+    waitForState("Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
+    cluster.getSolrClient().setDefaultCollection(collection);
+
+    // start a couple indexing threads
+
+    int[] maxDocList = new int[] {300, 700, 1200, 1350, 3000};
+    int[] maxDocNightlyList = new int[] {3000, 7000, 12000, 30000, 45000, 60000};
+
+    int maxDoc;
+    if (!TEST_NIGHTLY) {
+      maxDoc = maxDocList[random().nextInt(maxDocList.length - 1)];
+    } else {
+      maxDoc = maxDocNightlyList[random().nextInt(maxDocList.length - 1)];
+    }
+    log.info("Indexing {} documents", maxDoc);
+
+    final StoppableIndexingThread indexThread
+        = new StoppableIndexingThread(null, cluster.getSolrClient(), "1", true, maxDoc, 1, true);
+    threads.add(indexThread);
+    indexThread.start();
+
+    final StoppableIndexingThread indexThread2
+        = new StoppableIndexingThread(null, cluster.getSolrClient(), "2", true, maxDoc, 1, true);
+    threads.add(indexThread2);
+    indexThread2.start();
+
+    // give some time to index...
+    int[] waitTimes = new int[] {200, 2000, 3000};
+    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
+
+    // bring shard replica down
+    DocCollection state = getCollectionState(collection);
+    Replica leader = state.getLeader("shard1");
+    Replica replica = getRandomReplica(state.getSlice("shard1"), (r) -> leader != r);

Review comment:
       *ReferenceEquality:*  Comparison using reference equality instead of value equality [(details)](https://errorprone.info/bugpattern/ReferenceEquality)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractSyncSliceTestBase.java
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.cloud;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Test sync phase that occurs when Leader goes down and a new Leader is
+ * elected.
+ */
+public abstract class AbstractSyncSliceTestBase extends AbstractFullDistribZkTestBase {
+  private boolean success = false;
+
+  @Override
+  public void distribTearDown() throws Exception {
+    if (!success) {
+      printLayoutOnTearDown = true;
+    }
+    super.distribTearDown();
+  }
+
+  public AbstractSyncSliceTestBase() {
+    super();
+    sliceCount = 1;
+    fixShardCount(TEST_NIGHTLY ? 7 : 4);
+  }
+
+  @Test
+  public void test() throws Exception {
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+    
+    waitForThingsToLevelOut(30, TimeUnit.SECONDS);
+
+    del("*:*");
+    List<CloudJettyRunner> skipServers = new ArrayList<>();
+    int docId = 0;
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "to come to the aid of their country.");
+    
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "old haven was blue.");
+    
+    skipServers.add(shardToJetty.get("shard1").get(1));
+    
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "but the song was fancy.");
+    
+    skipServers.add(shardToJetty.get("shard1").get(2));
+    
+    indexDoc(skipServers, id,docId++, i1, 50, tlong, 50, t1,
+        "under the moon and over the lake");
+    
+    commit();
+    
+    waitForRecoveriesToFinish(false);
+
+    // shard should be inconsistent
+    String shardFailMessage = checkShardConsistency("shard1", true, false);
+    assertNotNull(shardFailMessage);
+    
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.SYNCSHARD.toString());
+    params.set("collection", "collection1");
+    params.set("shard", "shard1");
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    
+    String baseUrl = ((HttpSolrClient) shardToJetty.get("shard1").get(2).client.solrClient)
+        .getBaseURL();
+    baseUrl = baseUrl.substring(0, baseUrl.length() - "collection1".length());
+    
+    // we only set the connect timeout, not so timeout
+    try (HttpSolrClient baseClient = getHttpSolrClient(baseUrl, 30000)) {
+      baseClient.request(request);
+    }
+
+    waitForThingsToLevelOut(15, TimeUnit.SECONDS);
+    
+    checkShardConsistency(false, true);
+    
+    long cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    assertEquals(4, cloudClientDocs);
+    
+    
+    // kill the leader - new leader could have all the docs or be missing one
+    CloudJettyRunner leaderJetty = shardToLeaderJetty.get("shard1");
+    
+    skipServers = getRandomOtherJetty(leaderJetty, null); // but not the leader
+    
+    // this doc won't be on one node
+    indexDoc(skipServers, id, docId++, i1, 50, tlong, 50, t1,
+        "to come to the aid of their country.");
+    commit();
+    
+    
+    Set<CloudJettyRunner> jetties = new HashSet<>();
+    jetties.addAll(shardToJetty.get("shard1"));
+    jetties.remove(leaderJetty);
+    assertEquals(getShardCount() - 1, jetties.size());
+    
+    leaderJetty.jetty.stop();
+    
+    Thread.sleep(3000);
+    
+    waitForNoShardInconsistency();
+    
+    Thread.sleep(1000);
+    
+    checkShardConsistency(false, true);
+    
+    cloudClientDocs = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    assertEquals(5, cloudClientDocs);
+    
+    CloudJettyRunner deadJetty = leaderJetty;
+    
+    // let's get the latest leader
+    while (deadJetty == leaderJetty) {

Review comment:
       *ReferenceEquality:*  Comparison using reference equality instead of value equality [(details)](https://errorprone.info/bugpattern/ReferenceEquality)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractRestartWhileUpdatingTestBase.java
##########
@@ -0,0 +1,197 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.common.SolrInputDocument;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractRestartWhileUpdatingTestBase extends AbstractFullDistribZkTestBase {
+
+  //private static final String DISTRIB_UPDATE_CHAIN = "distrib-update-chain";
+  private List<StoppableIndexingThread> threads;
+
+  private volatile boolean stopExpire = false;
+
+  public AbstractRestartWhileUpdatingTestBase() throws Exception {
+    super();
+    sliceCount = 1;
+    fixShardCount(3);
+    schemaString = "schema15.xml";      // we need a string id

Review comment:
       *StaticAssignmentInConstructor:*  This assignment is to a static field. Mutating static state from a constructor is highly error-prone. [(details)](https://errorprone.info/bugpattern/StaticAssignmentInConstructor)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractTlogReplayBufferedWhileIndexingTestBase.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.TestInjection;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractTlogReplayBufferedWhileIndexingTestBase extends AbstractFullDistribZkTestBase {
+
+  private List<StoppableIndexingThread> threads;
+
+  public AbstractTlogReplayBufferedWhileIndexingTestBase() throws Exception {
+    super();
+    sliceCount = 1;
+    fixShardCount(2);
+    schemaString = "schema15.xml";      // we need a string id

Review comment:
       *StaticAssignmentInConstructor:*  This assignment is to a static field. Mutating static state from a constructor is highly error-prone. [(details)](https://errorprone.info/bugpattern/StaticAssignmentInConstructor)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");

Review comment:
       *StringSplitter:*  String.split(String) has surprising behavior [(details)](https://errorprone.info/bugpattern/StringSplitter)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+    assertEquals(3, allDocs);
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void createCollection(String collection,

Review comment:
       *UnusedMethod:*  Private method 'createCollection' is never used. [(details)](https://errorprone.info/bugpattern/UnusedMethod)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)

Review comment:
       *UnusedMethod:*  Private method 'waitForDocCount' is never used. [(details)](https://errorprone.info/bugpattern/UnusedMethod)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       *UnusedVariable:*  The field 'log' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.cloud;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.mockfile.FilterPath;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.handler.BackupStatusChecker;
+import org.apache.solr.handler.ReplicationHandler;
+import org.junit.Test;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZk2TestBase extends AbstractFullDistribZkTestBase {
+  private static final String SHARD2 = "shard2";
+  private static final String SHARD1 = "shard1";
+  private static final String ONE_NODE_COLLECTION = "onenodecollection";
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+
+  public AbstractBasicDistributedZk2TestBase() {
+    super();
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+
+    sliceCount = 2;
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  public void test() throws Exception {
+    boolean testFinished = false;
+    try {
+      handle.clear();
+      handle.put("timestamp", SKIPVAL);
+
+      testNodeWithoutCollectionForwarding();
+
+      indexr(id, 1, i1, 100, tlong, 100, t1,
+          "now is the time for all good men", "foo_f", 1.414f, "foo_b", "true",
+          "foo_d", 1.414d);
+
+      commit();
+
+      // make sure we are in a steady state...
+      waitForRecoveriesToFinish(false);
+
+      assertDocCounts(false);
+
+      indexAbunchOfDocs();
+
+      // check again
+      waitForRecoveriesToFinish(false);
+
+      commit();
+
+      assertDocCounts(VERBOSE);
+      checkQueries();
+
+      assertDocCounts(VERBOSE);
+
+      query("q", "*:*", "sort", "n_tl1 desc");
+
+      bringDownShardIndexSomeDocsAndRecover();
+
+      query("q", "*:*", "sort", "n_tl1 desc");
+
+      // test adding another replica to a shard - it should do a
+      // recovery/replication to pick up the index from the leader
+      addNewReplica();
+
+      long docId = testUpdateAndDelete();
+
+      // index a bad doc...
+      expectThrows(SolrException.class, () -> indexr(t1, "a doc with no id"));
+
+      // TODO: bring this to its own method?
+      // try indexing to a leader that has no replicas up
+      ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+      ZkNodeProps leaderProps = zkStateReader.getLeaderRetry(
+          DEFAULT_COLLECTION, SHARD2);
+
+      String nodeName = leaderProps.getStr(ZkStateReader.NODE_NAME_PROP);
+      chaosMonkey.stopShardExcept(SHARD2, nodeName);
+
+      SolrClient client = getClient(nodeName);
+
+      index_specific(client, "id", docId + 1, t1, "what happens here?");
+
+      // expire a session...
+      CloudJettyRunner cloudJetty = shardToJetty.get(SHARD1).get(0);
+      chaosMonkey.expireSession(cloudJetty.jetty);
+
+      indexr("id", docId + 1, t1, "slip this doc in");
+
+      waitForRecoveriesToFinish(false);
+
+      checkShardConsistency(SHARD1);
+      checkShardConsistency(SHARD2);
+
+      testFinished = true;
+    } finally {
+      if (!testFinished) {
+        printLayoutOnTearDown = true;
+      }
+    }
+
+  }
+
+  private void testNodeWithoutCollectionForwarding() throws Exception {
+    assertEquals(0, CollectionAdminRequest
+        .createCollection(ONE_NODE_COLLECTION, "conf1", 1, 1)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard(ONE_NODE_COLLECTION, "shard1")
+        .setCoreName(ONE_NODE_COLLECTION + "core")
+        .process(cloudClient).isSuccess());
+
+    waitForCollection(cloudClient.getZkStateReader(), ONE_NODE_COLLECTION, 1);
+    waitForRecoveriesToFinish(ONE_NODE_COLLECTION, cloudClient.getZkStateReader(), false);
+
+    cloudClient.getZkStateReader().getLeaderRetry(ONE_NODE_COLLECTION, SHARD1, 30000);
+
+    int docs = 2;
+    for (SolrClient client : clients) {
+      final String clientUrl = getBaseUrl((HttpSolrClient) client);
+      addAndQueryDocs(clientUrl, docs);
+      docs += 2;
+    }
+  }
+
+  // 2 docs added every call
+  private void addAndQueryDocs(final String baseUrl, int docs)
+      throws Exception {
+
+    SolrQuery query = new SolrQuery("*:*");
+
+    try (HttpSolrClient qclient = getHttpSolrClient(baseUrl + "/onenodecollection" + "core")) {
+
+      // it might take a moment for the proxy node to see us in their cloud state
+      waitForNon403or404or503(qclient);
+
+      // add a doc
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", docs);
+      qclient.add(doc);
+      qclient.commit();
+
+
+      QueryResponse results = qclient.query(query);
+      assertEquals(docs - 1, results.getResults().getNumFound());
+    }
+
+    try (HttpSolrClient qclient = getHttpSolrClient(baseUrl + "/onenodecollection")) {
+      QueryResponse results = qclient.query(query);
+      assertEquals(docs - 1, results.getResults().getNumFound());
+
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", docs + 1);
+      qclient.add(doc);
+      qclient.commit();
+
+      query = new SolrQuery("*:*");
+      query.set("rows", 0);
+      results = qclient.query(query);
+      assertEquals(docs, results.getResults().getNumFound());
+    }
+  }
+
+  private long testUpdateAndDelete() throws Exception {
+    long docId = 99999999L;
+    indexr("id", docId, t1, "originalcontent");
+
+    commit();
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", t1 + ":originalcontent");
+    QueryResponse results = clients.get(0).query(params);
+    assertEquals(1, results.getResults().getNumFound());
+
+    // update doc
+    indexr("id", docId, t1, "updatedcontent");
+
+    commit();
+
+    results = clients.get(0).query(params);
+    assertEquals(0, results.getResults().getNumFound());
+
+    params.set("q", t1 + ":updatedcontent");
+
+    results = clients.get(0).query(params);
+    assertEquals(1, results.getResults().getNumFound());
+
+    UpdateRequest uReq = new UpdateRequest();
+    // uReq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
+    uReq.deleteById(Long.toString(docId)).process(clients.get(0));
+
+    commit();
+
+    results = clients.get(0).query(params);
+    assertEquals(0, results.getResults().getNumFound());
+    return docId;
+  }
+
+  private void bringDownShardIndexSomeDocsAndRecover() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+
+    commit();
+
+    long deadShardCount = shardToJetty.get(SHARD2).get(0).client.solrClient

Review comment:
       *UnusedVariable:*  The local variable 'deadShardCount' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.cloud;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.mockfile.FilterPath;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.handler.BackupStatusChecker;
+import org.apache.solr.handler.ReplicationHandler;
+import org.junit.Test;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZk2TestBase extends AbstractFullDistribZkTestBase {
+  private static final String SHARD2 = "shard2";
+  private static final String SHARD1 = "shard1";
+  private static final String ONE_NODE_COLLECTION = "onenodecollection";
+  private final boolean onlyLeaderIndexes = random().nextBoolean();

Review comment:
       *UnusedVariable:*  The field 'onlyLeaderIndexes' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(

Review comment:
       *UnusedVariable:*  The local variable 'result' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();

Review comment:
       *UnusedVariable:*  The local variable 'fourDocs' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();
+    long fourDocs = client4.query(query).getResults().getNumFound();
+
+    query.set("collection", oneInstanceCollection2);
+    query.set("distrib", true);
+    long allDocs = getCommonCloudSolrClient().query(query).getResults().getNumFound();
+
+//    System.out.println("1:" + oneDocs);
+//    System.out.println("2:" + twoDocs);
+//    System.out.println("3:" + threeDocs);
+//    System.out.println("4:" + fourDocs);
+//    System.out.println("All Docs:" + allDocs);
+
+//    assertEquals(oneDocs, threeDocs);
+//    assertEquals(twoDocs, fourDocs);
+//    assertNotSame(oneDocs, twoDocs);
+    assertEquals(3, allDocs);
+
+    // we added a role of none on these creates - check for it
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+    zkStateReader.forceUpdateCollection(oneInstanceCollection2);
+    Map<String,Slice> slices = zkStateReader.getClusterState().getCollection(oneInstanceCollection2).getSlicesMap();
+    assertNotNull(slices);
+
+    ZkCoreNodeProps props = new ZkCoreNodeProps(getCommonCloudSolrClient().getZkStateReader().getClusterState()
+        .getCollection(oneInstanceCollection2).getLeader("shard1"));
+
+    // now test that unloading a core gets us a new leader
+    try (HttpSolrClient unloadClient = getHttpSolrClient(jettys.get(0).getBaseUrl().toString(), 15000, 60000)) {
+      Unload unloadCmd = new Unload(true);
+      unloadCmd.setCoreName(props.getCoreName());
+
+      String leader = props.getCoreUrl();
+
+      testExecutor.execute(new Runnable() {
+
+        @Override
+        public void run() {
+          try {
+            unloadClient.request(unloadCmd);
+          } catch (SolrServerException e) {
+            throw new RuntimeException(e);
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        }
+      });
+
+      try {
+        getCommonCloudSolrClient().getZkStateReader().waitForState(oneInstanceCollection2, 20000, TimeUnit.MILLISECONDS, (n, c) -> {
+
+
+          try {
+            if (leader.equals(zkStateReader.getLeaderUrl(oneInstanceCollection2, "shard1", 10000))) {
+              return false;
+            }
+          } catch (InterruptedException e) {
+            throw new RuntimeException(e);
+          }
+          return true;
+        });
+      } catch (TimeoutException | InterruptedException e) {
+        fail("Leader never changed");
+      }
+    }
+
+    IOUtils.close(collectionClients);
+
+  }
+
+  private void testSearchByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testSearchByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      SolrQuery query = new SolrQuery("*:*");
+      long oneDocs = client1.query(query).getResults().getNumFound();
+      assertEquals(3, oneDocs);
+    }
+  }
+
+  private void testUpdateByCollectionName() throws SolrServerException, IOException {
+    log.info("### STARTING testUpdateByCollectionName");
+    SolrClient client = clients.get(0);
+    final String baseUrl = ((HttpSolrClient) client).getBaseURL().substring(
+        0,
+        ((HttpSolrClient) client).getBaseURL().length()
+            - DEFAULT_COLLECTION.length() - 1);
+
+    // the cores each have different names, but if we add the collection name to the url
+    // we should get mapped to the right core
+    // test hitting an update url
+    try (SolrClient client1 = createNewSolrClient(oneInstanceCollection, baseUrl)) {
+      client1.commit();
+    }
+  }
+
+  private void testANewCollectionInOneInstance() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstance");
+    CollectionAdminResponse response = CollectionAdminRequest.createCollection(oneInstanceCollection, "conf1", 2, 2)
+        .setCreateNodeSet(jettys.get(0).getNodeName())
+        .process(cloudClient);
+    assertEquals(0, response.getStatus());
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (String coreName : response.getCollectionCoresStatus().keySet()) {
+      collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+    waitForRecoveriesToFinish(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader(), false);
+    assertAllActive(oneInstanceCollection, getCommonCloudSolrClient().getZkStateReader());
+
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();

Review comment:
       *UnusedVariable:*  The local variable 'oneDocs' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();
+    long threeDocs = client3.query(query).getResults().getNumFound();

Review comment:
       *UnusedVariable:*  The local variable 'threeDocs' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+  String t1="a_t";
+  String i1="a_i1";
+  String tlong = "other_tl1";
+  String tsort="t_sortable";
+
+  String oddField="oddField_s";
+  String missingField="ignore_exception__missing_but_valid_field_t";
+
+  private Map<String,List<SolrClient>> otherCollectionClients = new HashMap<>();
+
+  private String oneInstanceCollection = "oneInstanceCollection";
+  private String oneInstanceCollection2 = "oneInstanceCollection2";
+  
+  private AtomicInteger nodeCounter = new AtomicInteger();
+  
+  CompletionService<Object> completionService;
+  Set<Future<Object>> pending;
+  
+  private static Hook newSearcherHook = new Hook() {
+    volatile CountDownLatch latch;
+    AtomicReference<String> collection = new AtomicReference<>();
+
+    @Override
+    public void newSearcher(String collectionName) {
+      String c = collection.get();
+      if (c  != null && c.equals(collectionName)) {
+        log.info("Hook detected newSearcher");
+        try {
+          latch.countDown();
+        } catch (NullPointerException e) {
+
+        }
+      }
+    }
+  
+    public void waitForSearcher(String collection, int cnt, int timeoutms, boolean failOnTimeout) throws InterruptedException {
+      latch = new CountDownLatch(cnt);
+      this.collection.set(collection);
+      boolean timeout = !latch.await(timeoutms, TimeUnit.MILLISECONDS);
+      if (timeout && failOnTimeout) {
+        fail("timed out waiting for new searcher event " + latch.getCount());
+      }
+    }
+  
+  };
+
+  public AbstractBasicDistributedZkTestBase() {
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+    
+    sliceCount = 2;
+    completionService = new ExecutorCompletionService<>(executor);
+    pending = new HashSet<>();
+    
+  }
+  
+  @BeforeClass
+  public static void beforeBDZKTClass() {
+    TestInjection.newSearcherHook(newSearcherHook);
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Override
+  protected void setDistributedParams(ModifiableSolrParams params) {
+
+    if (r.nextBoolean()) {
+      // don't set shards, let that be figured out from the cloud state
+    } else {
+      // use shard ids rather than physical locations
+      StringBuilder sb = new StringBuilder();
+      for (int i = 0; i < getShardCount(); i++) {
+        if (i > 0)
+          sb.append(',');
+        sb.append("shard" + (i + 3));
+      }
+      params.set("shards", sb.toString());
+    }
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  protected void test() throws Exception {
+    // setLoggingLevel(null);
+
+    ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+    // make sure we have leaders for each shard
+    for (int j = 1; j < sliceCount; j++) {
+      zkStateReader.getLeaderRetry(DEFAULT_COLLECTION, "shard" + j, 10000);
+    }      // make sure we again have leaders for each shard
+    
+    waitForRecoveriesToFinish(false);
+    
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    del("*:*");
+    queryAndCompareShards(params("q", "*:*", "distrib", "false", "sanity_check", "is_empty"));
+
+    // ask every individual replica of every shard to update+commit the same doc id
+    // with an incrementing counter on each update+commit
+    int foo_i_counter = 0;
+    for (SolrClient client : clients) {
+      foo_i_counter++;
+      indexDoc(client, params("commit", "true"), // SOLR-4923
+               sdoc(id,1, i1,100, tlong,100, "foo_i", foo_i_counter));
+      // after every update+commit, check all the shards consistency
+      queryAndCompareShards(params("q", "id:1", "distrib", "false", 
+                                   "sanity_check", "non_distrib_id_1_lookup"));
+      queryAndCompareShards(params("q", "id:1", 
+                                   "sanity_check", "distrib_id_1_lookup"));
+    }
+
+    indexr(id,1, i1, 100, tlong, 100,t1,"now is the time for all good men"
+            ,"foo_f", 1.414f, "foo_b", "true", "foo_d", 1.414d, tsort, "now is the time for all good men");
+    indexr(id, 2, i1, 50, tlong, 50, t1, "to come to the aid of their country."
+        , tsort, "to come to the aid of their country.");
+    indexr(id, 3, i1, 2, tlong, 2, t1, "how now brown cow", tsort, "how now brown cow");
+    indexr(id, 4, i1, -100, tlong, 101, t1, "the quick fox jumped over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 5, i1, 500, tlong, 500, t1, "the quick fox jumped way over the lazy dog"
+        , tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 6, i1, -600, tlong, 600, t1, "humpty dumpy sat on a wall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id, 7, i1, 123, tlong, 123, t1, "humpty dumpy had a great fall", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,8, i1, 876, tlong, 876,t1,"all the kings horses and all the kings men",tsort,"all the kings horses and all the kings men");
+    indexr(id, 9, i1, 7, tlong, 7, t1, "couldn't put humpty together again", tsort, "the quick fox jumped over the lazy dog");
+    indexr(id,10, i1, 4321, tlong, 4321,t1,"this too shall pass",tsort,"this too shall pass");
+    indexr(id,11, i1, -987, tlong, 987,t1,"An eye for eye only ends up making the whole world blind."
+        ,tsort,"An eye for eye only ends up making the whole world blind.");
+    indexr(id,12, i1, 379, tlong, 379,t1,"Great works are performed, not by strength, but by perseverance.",
+        tsort,"Great works are performed, not by strength, but by perseverance.");
+    indexr(id,13, i1, 232, tlong, 232,t1,"no eggs on wall, lesson learned", oddField, "odd man out",
+        tsort,"no eggs on wall, lesson learned");
+
+    indexr(id, 14, "SubjectTerms_mfacet", new String[]  {"mathematical models", "mathematical analysis"});
+    indexr(id, 15, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    indexr(id, 16, "SubjectTerms_mfacet", new String[]  {"test 1", "test 2", "test3"});
+    String[] vals = new String[100];
+    for (int i=0; i<100; i++) {
+      vals[i] = "test " + i;
+    }
+    indexr(id, 17, "SubjectTerms_mfacet", vals);
+
+    for (int i=100; i<150; i++) {
+      indexr(id, i);      
+    }
+
+    commit();
+
+    testTokenizedGrouping();
+    testSortableTextFaceting();
+    testSortableTextSorting();
+    testSortableTextGrouping();
+
+    queryAndCompareShards(params("q", "*:*", 
+                                 "sort", "id desc",
+                                 "distrib", "false", 
+                                 "sanity_check", "is_empty"));
+
+    // random value sort
+    for (String f : fieldNames) {
+      query(false, new String[] {"q","*:*", "sort",f+" desc"});
+      query(false, new String[] {"q","*:*", "sort",f+" asc"});
+    }
+
+    // these queries should be exactly ordered and scores should exactly match
+    query(false, new String[] {"q","*:*", "sort",i1+" desc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" asc"});
+    query(false, new String[] {"q","*:*", "sort",i1+" desc", "fl","*,score"});
+    query(false, new String[] {"q","*:*", "sort","n_tl1 asc", "fl","*,score"}); 
+    query(false, new String[] {"q","*:*", "sort","n_tl1 desc"});
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","{!func}"+i1});// does not expect maxScore. So if it comes ,ignore it. JavaBinCodec.writeSolrDocumentList()
+    //is agnostic of request params.
+    handle.remove("maxScore");
+    query(false, new String[] {"q","{!func}"+i1, "fl","*,score"});  // even scores should match exactly here
+
+    handle.put("highlighting", UNORDERED);
+    handle.put("response", UNORDERED);
+
+    handle.put("maxScore", SKIPVAL);
+    query(false, new String[] {"q","quick"});
+    query(false, new String[] {"q","all","fl","id","start","0"});
+    query(false, new String[] {"q","all","fl","foofoofoo","start","0"});  // no fields in returned docs
+    query(false, new String[] {"q","all","fl","id","start","100"});
+
+    handle.put("score", SKIPVAL);
+    query(false, new String[] {"q","quick","fl","*,score"});
+    query(false, new String[] {"q","all","fl","*,score","start","1"});
+    query(false, new String[] {"q","all","fl","*,score","start","100"});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","*,score",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","now their fox sat had put","fl","foofoofoo",
+            "hl","true","hl.fl",t1});
+
+    query(false, new String[] {"q","matchesnothing","fl","*,score"});  
+
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","count", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.limit",-1, "facet.sort","index", "facet.mincount",2});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1,"facet.limit",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.offset",1});
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",t1, "facet.mincount",2});
+
+    // test faceting multiple things at once
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","quick", "facet.query","all", "facet.query","*:*"
+    ,"facet.field",t1});
+
+    // test filter tagging, facet exclusion, and naming (multi-select facet support)
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.query","{!key=myquick}quick", "facet.query","{!key=myall ex=a}all", "facet.query","*:*"
+    ,"facet.field","{!key=mykey ex=a}"+t1
+    ,"facet.field","{!key=other ex=b}"+t1
+    ,"facet.field","{!key=again ex=a,b}"+t1
+    ,"facet.field",t1
+    ,"fq","{!tag=a}id_i1:[1 TO 7]", "fq","{!tag=b}id_i1:[3 TO 9]"}
+    );
+    query(false, new Object[] {"q", "*:*", "facet", "true", "facet.field", "{!ex=t1}SubjectTerms_mfacet", "fq", "{!tag=t1}SubjectTerms_mfacet:(test 1)", "facet.limit", "10", "facet.mincount", "1"});
+
+    // test field that is valid in schema but missing in all shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",missingField, "facet.mincount",2});
+    // test field that is valid in schema and missing in some shards
+    query(false, new Object[] {"q","*:*", "rows",100, "facet","true", "facet.field",oddField, "facet.mincount",2});
+
+    query(false, new Object[] {"q","*:*", "sort",i1+" desc", "stats", "true", "stats.field", i1});
+
+    /*** TODO: the failure may come back in "exception"
+    try {
+      // test error produced for field that is invalid for schema
+      query("q","*:*", "rows",100, "facet","true", "facet.field",invalidField, "facet.mincount",2);
+      TestCase.fail("SolrServerException expected for invalid field that is not in schema");
+    } catch (SolrServerException ex) {
+      // expected
+    }
+    ***/
+
+    // Try to get better coverage for refinement queries by turning off over requesting.
+    // This makes it much more likely that we may not get the top facet values and hence
+    // we turn of that checking.
+    handle.put("facet_fields", SKIPVAL);    
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field",t1,"facet.limit",5, "facet.shard.limit",5});
+    // check a complex key name
+    query(false, new Object[] {"q","*:*", "rows",0, "facet","true", "facet.field","{!key='a b/c \\' \\} foo'}"+t1,"facet.limit",5, "facet.shard.limit",5});
+    handle.remove("facet_fields");
+
+
+    // index the same document to two servers and make sure things
+    // don't blow up.
+    if (clients.size()>=2) {
+      index(id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      for (int i=0; i<clients.size(); i++) {
+        index_specific(i, id,100, i1, 107 ,t1,"oh no, a duplicate!");
+      }
+      commit();
+      query(false, new Object[] {"q","duplicate", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","fox duplicate horses", "hl","true", "hl.fl", t1});
+      query(false, new Object[] {"q","*:*", "rows",100});
+    }
+
+    // test debugging
+    handle.put("explain", SKIPVAL);
+    handle.put("debug", UNORDERED);
+    handle.put("time", SKIPVAL);
+    handle.put("track", SKIP);
+    query(false, new Object[] {"q","now their fox sat had put","fl","*,score",CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG_QUERY, "true"});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.TIMING});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.RESULTS});
+    query(false, new Object[] {"q", "id_i1:[1 TO 5]", CommonParams.DEBUG, CommonParams.QUERY});
+
+    // try add commitWithin
+    long before = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+    for (SolrClient client : clients) {
+      assertEquals("unexpected pre-commitWithin document count on node: " + ((HttpSolrClient)client).getBaseURL(), before, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("commitWithin", 10);
+    add(cloudClient, params , getDoc("id", 300), getDoc("id", 301));
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    DocCollection dColl = clusterState.getCollection(DEFAULT_COLLECTION);
+
+    assertSliceCounts("should have found 2 docs, 300 and 301", before + 2, dColl);
+
+    // try deleteById commitWithin
+    UpdateRequest deleteByIdReq = new UpdateRequest();
+    deleteByIdReq.deleteById("300");
+    deleteByIdReq.setCommitWithin(10);
+    deleteByIdReq.process(cloudClient);
+    
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+
+    assertSliceCounts("deleteById commitWithin did not work", before + 1, dColl);
+    
+    // try deleteByQuery commitWithin
+    UpdateRequest deleteByQueryReq = new UpdateRequest();
+    deleteByQueryReq.deleteByQuery("id:301");
+    deleteByQueryReq.setCommitWithin(10);
+    deleteByQueryReq.process(cloudClient);
+
+    newSearcherHook.waitForSearcher(DEFAULT_COLLECTION, 2, 20000, false);
+    
+    assertSliceCounts("deleteByQuery commitWithin did not work", before, dColl);
+    
+
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // would be better if these where all separate tests - but much, much
+    // slower
+    doOptimisticLockingAndUpdating();
+    testShardParamVariations();
+    testMultipleCollections();
+    testANewCollectionInOneInstance();
+    testSearchByCollectionName();
+    testUpdateByCollectionName();
+    testANewCollectionInOneInstanceWithManualShardAssignement();
+    testNumberOfCommitsWithCommitAfterAdd();
+
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-explicit");
+    testUpdateProcessorsRunOnlyOnce("distrib-dup-test-chain-implicit");
+
+    testStopAndStartCoresInOneInstance();
+  }
+
+  private void testSortableTextFaceting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addFacetField(tsort);
+    query.setFacetMissing(false);
+    QueryResponse resp = queryServer(query);
+    List<FacetField> ffs = resp.getFacetFields();
+    for (FacetField ff : ffs) {
+      if (ff.getName().equals(tsort) == false) continue;
+      for (FacetField.Count count : ff.getValues()) {
+        long num = count.getCount();
+        switch (count.getName()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + ff.getName(), 1, num);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, num);
+            break;
+          default:
+            fail("No case for facet '" + ff.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testSortableTextSorting() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.addSort(tsort, SolrQuery.ORDER.desc);
+    query.addField("*");
+    query.addField("eoe_sortable");
+    query.addField(tsort);
+    QueryResponse resp = queryServer(query);
+
+    SolrDocumentList docs = resp.getResults();
+
+    String title = docs.get(0).getFieldValue(tsort).toString();
+    for (SolrDocument doc : docs) {
+      assertTrue("Docs should be back in sorted order, descending", title.compareTo(doc.getFieldValue(tsort).toString()) >= 0);
+      title = doc.getFieldValue(tsort).toString();
+    }
+  }
+
+  private void testSortableTextGrouping() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.add("group", "true");
+    query.add("group.field", tsort);
+    QueryResponse resp = queryServer(query);
+    GroupResponse groupResp = resp.getGroupResponse();
+    List<GroupCommand> grpCmds = groupResp.getValues();
+    for (GroupCommand grpCmd : grpCmds) {
+      if (grpCmd.getName().equals(tsort) == false) continue;
+      for (Group grp : grpCmd.getValues()) {
+        long count = grp.getResult().getNumFound();
+        if (grp.getGroupValue() == null) continue; // Don't count the groups without an entry as the numnber is variable
+        switch (grp.getGroupValue()) {
+          case "all the kings horses and all the kings men":
+          case "An eye for eye only ends up making the whole world blind.":
+          case "Great works are performed, not by strength, but by perseverance.":
+          case "how now brown cow":
+          case "no eggs on wall, lesson learned":
+          case "now is the time for all good men":
+          case "this too shall pass":
+          case "to come to the aid of their country.":
+            assertEquals("Should have exactly one facet count for field " + grpCmd.getName(), 1, count);
+            break;
+          case "the quick fox jumped over the lazy dog":
+            assertEquals("Should have 5 docs for the lazy dog", 5, count);
+            break;
+          default:
+            fail("No case for facet '" + grpCmd.getName() + "'");
+
+        }
+      }
+    }
+  }
+
+  private void testTokenizedGrouping() throws Exception {
+    SolrException ex = expectThrows(SolrException.class, () -> {
+      query(false, new String[]{"q", "*:*", "group", "true", "group.field", t1});
+    });
+    assertTrue("Expected error from server that SortableTextFields are required", ex.getMessage().contains("Sorting on a tokenized field that is not a SortableTextField is not supported in cloud mode"));
+  }
+
+  private void assertSliceCounts(String msg, long expected, DocCollection dColl) throws Exception {
+    long found = checkSlicesSameCounts(dColl);
+    
+    if (found != expected) {
+      // we get one do over in a bad race
+      Thread.sleep(1000);
+      found = checkSlicesSameCounts(dColl);
+    }
+    
+    assertEquals(msg, expected, checkSlicesSameCounts(dColl));
+  }
+
+  // Ensure that total docs found is the expected number.
+  private void waitForDocCount(long expectedNumFound, long waitMillis, String failureMessage)
+      throws Exception {
+    AtomicLong total = new AtomicLong(-1);
+    try {
+      getCommonCloudSolrClient().getZkStateReader().waitForState(DEFAULT_COLLECTION, waitMillis, TimeUnit.MILLISECONDS, (n, c) -> {
+        long docTotal;
+        try {
+          docTotal = checkSlicesSameCounts(c);
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+        total.set(docTotal);
+        if (docTotal == expectedNumFound) {
+          return true;
+        }
+        return false;
+      });
+    } catch (TimeoutException | InterruptedException e) {
+     
+    }
+    // We could fail here if we broke out of the above because we exceeded the time allowed.
+    assertEquals(failureMessage, expectedNumFound, total.get());
+
+    // This should be redundant, but it caught a test error after all.
+    for (SolrClient client : clients) {
+      assertEquals(failureMessage, expectedNumFound, client.query(new SolrQuery("*:*")).getResults().getNumFound());
+    }
+  }
+
+  // Insure that counts are the same for all replicas in each shard
+  // Return the total doc count for the query.
+  private long checkSlicesSameCounts(DocCollection dColl) throws SolrServerException, IOException {
+    long docTotal = 0; // total number of documents found counting only one replica per slice.
+    for (Slice slice : dColl.getActiveSlices()) {
+      long sliceDocCount = -1;
+      for (Replica rep : slice.getReplicas()) {
+        try (HttpSolrClient one = getHttpSolrClient(rep.getCoreUrl())) {
+          SolrQuery query = new SolrQuery("*:*");
+          query.setDistrib(false);
+          QueryResponse resp = one.query(query);
+          long hits = resp.getResults().getNumFound();
+          if (sliceDocCount == -1) {
+            sliceDocCount = hits;
+            docTotal += hits;
+          } else {
+            if (hits != sliceDocCount) {
+              return -1;
+            }
+          }
+        }
+      }
+    }
+    return docTotal;
+  }
+
+  private void testShardParamVariations() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    Map<String,Long> shardCounts = new HashMap<>();
+
+    for (String shard : shardToJetty.keySet()) {
+      // every client should give the same numDocs for this shard
+      // shffle the clients in a diff order for each shard
+      List<SolrClient> solrclients = new ArrayList<>(this.clients);
+      Collections.shuffle(solrclients, random());
+      for (SolrClient client : solrclients) {
+        query.set("shards", shard);
+        long numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for shard "+shard+" via "+client,
+                   0 <= numDocs);
+        if (!shardCounts.containsKey(shard)) {
+          shardCounts.put(shard, numDocs);
+        }
+        assertEquals("inconsitent numDocs for shard "+shard+" via "+client,
+                     shardCounts.get(shard).longValue(), numDocs);
+        
+        List<CloudJettyRunner> replicaJetties 
+          = new ArrayList<>(shardToJetty.get(shard));
+        Collections.shuffle(replicaJetties, random());
+
+        // each replica should also give the same numDocs
+        ArrayList<String> replicaAlts = new ArrayList<>(replicaJetties.size() * 2);
+        for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+          String replica = replicaJetty.url;
+          query.set("shards", replica);
+
+          // replicas already shuffled, use this in the alternative check below
+          if (0 == random().nextInt(3) || replicaAlts.size() < 2) {
+            replicaAlts.add(replica);
+          }
+
+          numDocs = client.query(query).getResults().getNumFound();
+          assertTrue("numDocs < 0 for replica "+replica+" via "+client,
+                     0 <= numDocs);
+          assertEquals("inconsitent numDocs for shard "+shard+
+                       " in replica "+replica+" via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+        }
+
+        // any combination of replica alternatives should give same numDocs
+        String replicas = String.join("|", replicaAlts);
+        query.set("shards", replicas);
+        numDocs = client.query(query).getResults().getNumFound();
+        assertTrue("numDocs < 0 for replicas "+replicas+" via "+client,
+                   0 <= numDocs);
+          assertEquals("inconsitent numDocs for replicas "+replicas+
+                       " via "+client,
+                       shardCounts.get(shard).longValue(), numDocs);
+      }
+    }
+
+    // sums of multiple shards should add up regardless of how we 
+    // query those shards or which client we use
+    long randomShardCountsExpected = 0;
+    ArrayList<String> randomShards = new ArrayList<>(shardCounts.size());
+    for (Map.Entry<String,Long> shardData : shardCounts.entrySet()) {
+      if (random().nextBoolean() || randomShards.size() < 2) {
+        String shard = shardData.getKey();
+        randomShardCountsExpected += shardData.getValue();
+        if (random().nextBoolean()) {
+          // use shard id
+          randomShards.add(shard);
+        } else {
+          // use some set explicit replicas
+          ArrayList<String> replicas = new ArrayList<>(7);
+          for (CloudJettyRunner replicaJetty : shardToJetty.get(shard)) {
+            if (0 == random().nextInt(3) || 0 == replicas.size()) {
+              replicas.add(replicaJetty.url);
+            }
+          }
+          Collections.shuffle(replicas, random());
+          randomShards.add(String.join("|", replicas));
+        }
+      }
+    }
+    String randShards = String.join(",", randomShards);
+    query.set("shards", randShards);
+    for (SolrClient client : this.clients) {
+      assertEquals("numDocs for "+randShards+" via "+client,
+                   randomShardCountsExpected, 
+                   client.query(query).getResults().getNumFound());
+    }
+
+    // total num docs must match sum of every shard's numDocs
+    query = new SolrQuery("*:*");
+    long totalShardNumDocs = 0;
+    for (Long c : shardCounts.values()) {
+      totalShardNumDocs += c;
+    }
+    for (SolrClient client : clients) {
+      assertEquals("sum of shard numDocs on client: " + client, 
+                   totalShardNumDocs,
+                   client.query(query).getResults().getNumFound());
+    }
+    assertTrue("total numDocs <= 0, WTF? Test is useless",
+        0 < totalShardNumDocs);
+
+  }
+
+  private void testStopAndStartCoresInOneInstance() throws Exception {
+    JettySolrRunner jetty = jettys.get(0);
+    try (final HttpSolrClient httpSolrClient = (HttpSolrClient) jetty.newClient(15000, 60000)) {
+      ThreadPoolExecutor executor = null;
+      try {
+        executor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+            5, TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new SolrNamedThreadFactory("testExecutor"));
+        int cnt = 3;
+
+        // create the cores
+        createCollectionInOneInstance(httpSolrClient, jetty.getNodeName(), executor, "multiunload2", 1, cnt);
+      } finally {
+        if (executor != null) {
+          ExecutorUtil.shutdownAndAwaitTermination(executor);
+        }
+      }
+    }
+    
+    cloudJettys.get(0).jetty.stop();
+    printLayout();
+
+    cloudJettys.get(0).jetty.start();
+    cloudClient.getZkStateReader().forceUpdateCollection("multiunload2");
+    try {
+      cloudClient.getZkStateReader().getLeaderRetry("multiunload2", "shard1", 30000);
+    } catch (SolrException e) {
+      printLayout();
+      throw e;
+    }
+    
+    printLayout();
+
+  }
+
+  /**
+   * Create a collection in single node
+   */
+  protected void createCollectionInOneInstance(final SolrClient client, String nodeName,
+                                               ThreadPoolExecutor executor, final String collection,
+                                               final int numShards, int numReplicas) {
+    assertNotNull(nodeName);
+    try {
+      assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+          .setCreateNodeSet("")
+          .process(client).getStatus());
+    } catch (SolrServerException | IOException e) {
+      throw new RuntimeException(e);
+    }
+    for (int i = 0; i < numReplicas; i++) {
+      final int freezeI = i;
+      executor.execute(() -> {
+        try {
+          assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard"+((freezeI%numShards)+1))
+              .setCoreName(collection + freezeI)
+              .setNode(nodeName).process(client).isSuccess());
+        } catch (SolrServerException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      });
+    }
+  }
+
+  protected String getBaseUrl(SolrClient client) {
+    String url2 = ((HttpSolrClient) client).getBaseURL()
+        .substring(
+            0,
+            ((HttpSolrClient) client).getBaseURL().length()
+                - DEFAULT_COLLECTION.length() -1);
+    return url2;
+  }
+
+  @Override
+  protected CollectionAdminResponse createCollection(Map<String, List<Integer>> collectionInfos,
+                                                     String collectionName, String configSetName, int numShards, int numReplicas, SolrClient client, String createNodeSetStr) throws SolrServerException, IOException {
+    // TODO: Use CollectionAdminRequest for this test
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+
+    params.set(CollectionHandlingUtils.NUM_SLICES, numShards);
+    params.set(ZkStateReader.REPLICATION_FACTOR, numReplicas);
+    if (createNodeSetStr != null) params.set(CollectionHandlingUtils.CREATE_NODE_SET, createNodeSetStr);
+
+    int clientIndex = clients.size() > 1 ? random().nextInt(2) : 0;
+    List<Integer> list = new ArrayList<>();
+    list.add(numShards);
+    list.add(numReplicas);
+    if (collectionInfos != null) {
+      collectionInfos.put(collectionName, list);
+    }
+    params.set("name", collectionName);
+    params.set("collection.configName", configSetName);
+    QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    CollectionAdminResponse res = new CollectionAdminResponse();
+    if (client == null) {
+      final String baseUrl = ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().substring(
+          0,
+          ((HttpSolrClient) clients.get(clientIndex)).getBaseURL().length()
+              - DEFAULT_COLLECTION.length() - 1);
+
+      try (SolrClient aClient = createNewSolrClient("", baseUrl)) {
+        res.setResponse(aClient.request(request));
+      }
+    } else {
+      res.setResponse(client.request(request));
+    }
+    return res;
+  }
+
+  protected ZkCoreNodeProps getLeaderUrlFromZk(String collection, String slice) {
+    ClusterState clusterState = getCommonCloudSolrClient().getZkStateReader().getClusterState();
+    ZkNodeProps leader = clusterState.getCollection(collection).getLeader(slice);
+    if (leader == null) {
+      throw new RuntimeException("Could not find leader:" + collection + " " + slice);
+    }
+    return new ZkCoreNodeProps(leader);
+  }
+
+  /**
+   * Expects a RegexReplaceProcessorFactories in the chain which will
+   * "double up" the values in two (stored) string fields.
+   * <p>
+   * If the values are "double-doubled" or "not-doubled" then we know
+   * the processor was not run the appropriate number of times
+   * </p>
+   */
+  private void testUpdateProcessorsRunOnlyOnce(final String chain) throws Exception {
+
+    final String fieldA = "regex_dup_A_s";
+    final String fieldB = "regex_dup_B_s";
+    final String val = "x";
+    final String expected = "x_x";
+    final ModifiableSolrParams updateParams = new ModifiableSolrParams();
+    updateParams.add(UpdateParams.UPDATE_CHAIN, chain);
+
+    final int numLoops = atLeast(50);
+
+    for (int i = 1; i < numLoops; i++) {
+      // add doc to random client
+      SolrClient updateClient = clients.get(random().nextInt(clients.size()));
+      SolrInputDocument doc = new SolrInputDocument();
+      addFields(doc, id, i, fieldA, val, fieldB, val);
+      UpdateResponse ures = add(updateClient, updateParams, doc);
+      assertEquals(chain + ": update failed", 0, ures.getStatus());
+      ures = updateClient.commit();
+      assertEquals(chain + ": commit failed", 0, ures.getStatus());
+    }
+
+    // query for each doc, and check both fields to ensure the value is correct
+    for (int i = 1; i < numLoops; i++) {
+      final String query = id + ":" + i;
+      QueryResponse qres = queryServer(new SolrQuery(query));
+      assertEquals(chain + ": query failed: " + query,
+          0, qres.getStatus());
+      assertEquals(chain + ": didn't find correct # docs with query: " + query,
+          1, qres.getResults().getNumFound());
+      SolrDocument doc = qres.getResults().get(0);
+
+      for (String field : new String[] {fieldA, fieldB}) {
+        assertEquals(chain + ": doc#" + i+ " has wrong value for " + field,
+            expected, doc.getFirstValue(field));
+      }
+    }
+
+  }
+
+  // cloud level test mainly needed just to make sure that versions and errors are propagated correctly
+  private void doOptimisticLockingAndUpdating() throws Exception {
+    log.info("### STARTING doOptimisticLockingAndUpdating");
+    printLayout();
+
+    final SolrInputDocument sd =  sdoc("id", 1000, "_version_", -1);
+    indexDoc(sd);
+
+    ignoreException("version conflict");
+    for (SolrClient client : clients) {
+      SolrException e = expectThrows(SolrException.class, () -> client.add(sd));
+      assertEquals(409, e.code());
+    }
+    unIgnoreException("version conflict");
+
+    // TODO: test deletes.  SolrJ needs a good way to pass version for delete...
+
+    final SolrInputDocument sd2 =  sdoc("id", 1000, "foo_i",5);
+    clients.get(0).add(sd2);
+
+    List<Integer> expected = new ArrayList<>();
+    int val = 0;
+    for (SolrClient client : clients) {
+      val += 10;
+      client.add(sdoc("id", 1000, "val_i", map("add",val), "foo_i",val));
+      expected.add(val);
+    }
+
+    QueryRequest qr = new QueryRequest(params("qt", "/get", "id","1000"));
+    for (SolrClient client : clients) {
+      val += 10;
+      NamedList<?> rsp = client.request(qr);
+      String match = JSONTestUtil.matchObj("/val_i", rsp.get("doc"), expected);
+      if (match != null) throw new RuntimeException(match);
+    }
+  }
+
+  private void testNumberOfCommitsWithCommitAfterAdd()
+      throws SolrServerException, IOException {
+    log.info("### STARTING testNumberOfCommitsWithCommitAfterAdd");
+    long startCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+
+    NamedList<Object> result = clients.get(0).request(
+        new StreamingUpdateRequest("/update",
+            getFile("books_numeric_ids.csv"), "application/csv")
+            .setCommitWithin(900000)
+            .setAction(AbstractUpdateRequest.ACTION.COMMIT, true, true));
+
+    long endCommits = getNumCommits((HttpSolrClient) clients.get(0));
+
+    assertEquals(startCommits + 1L, endCommits);
+  }
+
+  private Long getNumCommits(HttpSolrClient sourceClient) throws
+      SolrServerException, IOException {
+    // construct the /admin/metrics URL
+    URL url = new URL(sourceClient.getBaseURL());
+    String path = url.getPath().substring(1);
+    String[] elements = path.split("/");
+    String collection = elements[elements.length - 1];
+    String urlString = url.toString();
+    urlString = urlString.substring(0, urlString.length() - collection.length() - 1);
+    try (HttpSolrClient client = getHttpSolrClient(urlString, 15000, 60000)) {
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      //params.set("qt", "/admin/metrics?prefix=UPDATE.updateHandler&registry=solr.core." + collection);
+      params.set("qt", "/admin/metrics");
+      params.set("prefix", "UPDATE.updateHandler");
+      params.set("registry", "solr.core." + collection);
+      // use generic request to avoid extra processing of queries
+      QueryRequest req = new QueryRequest(params);
+      NamedList<Object> resp = client.request(req);
+      NamedList<?> metrics = (NamedList<?>) resp.get("metrics");
+      NamedList<?> uhandlerCat = (NamedList<?>) metrics.getVal(0);
+      @SuppressWarnings({"unchecked"})
+      Map<String,Object> commits = (Map<String,Object>) uhandlerCat.get("UPDATE.updateHandler.commits");
+      return (Long) commits.get("count");
+    }
+  }
+
+  private void testANewCollectionInOneInstanceWithManualShardAssignement() throws Exception {
+    log.info("### STARTING testANewCollectionInOneInstanceWithManualShardAssignement");
+    assertEquals(0, CollectionAdminRequest.createCollection(oneInstanceCollection2, "conf1", 2, 2)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+
+    List<SolrClient> collectionClients = new ArrayList<>();
+    for (int i = 0; i < 4; i++) {
+      CollectionAdminResponse resp = CollectionAdminRequest
+          .addReplicaToShard(oneInstanceCollection2, "shard" + ((i%2)+1))
+          .setNode(jettys.get(0).getNodeName())
+          .process(cloudClient);
+      for (String coreName : resp.getCollectionCoresStatus().keySet()) {
+        collectionClients.add(createNewSolrClient(coreName, jettys.get(0).getBaseUrl().toString()));
+      }
+
+
+    }
+
+    SolrClient client1 = collectionClients.get(0);
+    SolrClient client2 = collectionClients.get(1);
+    SolrClient client3 = collectionClients.get(2);
+    SolrClient client4 = collectionClients.get(3);
+
+
+    // no one should be recovering
+    waitForRecoveriesToFinish(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader(), false, true);
+
+    assertAllActive(oneInstanceCollection2, getCommonCloudSolrClient().getZkStateReader());
+
+    //printLayout();
+
+    // TODO: enable when we don't falsely get slice1...
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice1", 30000);
+    // solrj.getZkStateReader().getLeaderUrl(oneInstanceCollection2, "slice2", 30000);
+    client2.add(getDoc(id, "1"));
+    client3.add(getDoc(id, "2"));
+    client4.add(getDoc(id, "3"));
+
+    client1.commit();
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+    long oneDocs = client1.query(query).getResults().getNumFound();
+    long twoDocs = client2.query(query).getResults().getNumFound();

Review comment:
       *UnusedVariable:*  The local variable 'twoDocs' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZkTestBase.java
##########
@@ -0,0 +1,1350 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.lucene.util.IOUtils;
+import org.apache.solr.JSONTestUtil;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Create;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.StreamingUpdateRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.FacetField;
+import org.apache.solr.client.solrj.response.Group;
+import org.apache.solr.client.solrj.response.GroupCommand;
+import org.apache.solr.client.solrj.response.GroupResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.client.solrj.response.UpdateResponse;
+import org.apache.solr.cloud.api.collections.CollectionHandlingUtils;
+import org.apache.solr.common.SolrDocument;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TestInjection.Hook;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZkTestBase extends AbstractFullDistribZkTestBase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final String DEFAULT_COLLECTION = "collection1";
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();

Review comment:
       *UnusedVariable:*  The field 'onlyLeaderIndexes' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractChaosMonkeyNothingIsSafeTestBase.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.cloud;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractChaosMonkeyNothingIsSafeTestBase extends AbstractFullDistribZkTestBase {
+  private static final int FAIL_TOLERANCE = 100;
+
+  private static final Integer RUN_LENGTH = Integer.parseInt(System.getProperty("solr.tests.cloud.cm.runlength", "-1"));
+
+  private final boolean onlyLeaderIndexes = random().nextBoolean();

Review comment:
       *UnusedVariable:*  The field 'onlyLeaderIndexes' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractMoveReplicaTestBase.java
##########
@@ -0,0 +1,377 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.common.SolrInputDocument;
+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.util.NamedList;
+import org.apache.solr.util.IdUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class AbstractMoveReplicaTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  // used by MoveReplicaHDFSTest
+  protected boolean inPlaceMove = true;
+  protected boolean isCollectionApiDistributed = false;
+
+  protected String getConfigSet() {
+    return "cloud-dynamic";
+  }
+
+  @Before
+  public void beforeTest() throws Exception {
+    inPlaceMove = true;
+
+    configureCluster(4)
+        .addConfig("conf1", configset(getConfigSet()))
+        .addConfig("conf2", configset(getConfigSet()))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+
+    // If Collection API is distributed let's not wait for Overseer.
+    if (isCollectionApiDistributed = new CollectionAdminRequest.RequestApiDistributedProcessing().process(cluster.getSolrClient()).getIsCollectionApiDistributed()) {
+      return;
+    }
+
+    NamedList<Object> overSeerStatus = cluster.getSolrClient().request(CollectionAdminRequest.getOverseerStatus());
+    JettySolrRunner overseerJetty = null;
+    String overseerLeader = (String) overSeerStatus.get("leader");
+    for (JettySolrRunner jetty : cluster.getJettySolrRunners()) {
+      if (jetty.getNodeName().equals(overseerLeader)) {
+        overseerJetty = jetty;
+        break;
+      }
+    }
+    if (overseerJetty == null) {
+      fail("no overseer leader!");
+    }
+  }
+
+  @After
+  public void afterTest() throws Exception {
+    try {
+      shutdownCluster();
+    } finally {
+      super.tearDown();
+    }
+  }
+
+  @Test
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  public void test() throws Exception {
+    String coll = getTestClass().getSimpleName() + "_coll_" + inPlaceMove;
+    if (log.isInfoEnabled()) {
+      log.info("total_jettys: {}", cluster.getJettySolrRunners().size());
+    }
+    int REPLICATION = 2;
+
+    CloudSolrClient cloudClient = cluster.getSolrClient();
+
+    // random create tlog or pull type replicas with nrt
+    boolean isTlog = random().nextBoolean();
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 1, isTlog ? 1 : 0, !isTlog ? 1 : 0);
+    cloudClient.request(create);
+
+    addDocs(coll, 100);
+
+    Replica replica = getRandomReplica(coll, cloudClient);
+    Set<String> liveNodes = cloudClient.getZkStateReader().getClusterState().getLiveNodes();
+    ArrayList<String> l = new ArrayList<>(liveNodes);
+    Collections.shuffle(l, random());
+    String targetNode = null;
+    for (String node : liveNodes) {
+      if (!replica.getNodeName().equals(node)) {
+        targetNode = node;
+        break;
+      }
+    }
+    assertNotNull(targetNode);
+    String shardId = null;
+    for (Slice slice : cloudClient.getZkStateReader().getClusterState().getCollection(coll).getSlices()) {
+      if (slice.getReplicas().contains(replica)) {
+        shardId = slice.getName();
+      }
+    }
+
+    int sourceNumCores = getNumOfCores(cloudClient, replica.getNodeName(), coll, replica.getType().name());
+    int targetNumCores = getNumOfCores(cloudClient, targetNode, coll, replica.getType().name());
+
+    CollectionAdminRequest.MoveReplica moveReplica = createMoveReplicaRequest(coll, replica, targetNode);
+    moveReplica.setInPlaceMove(inPlaceMove);
+    String asyncId = IdUtils.randomId();
+    moveReplica.processAsync(asyncId, cloudClient);
+    CollectionAdminRequest.RequestStatus requestStatus = CollectionAdminRequest.requestStatus(asyncId);
+    // wait for async request success
+    boolean success = false;
+    for (int i = 0; i < 200; i++) {
+      CollectionAdminRequest.RequestStatusResponse rsp = requestStatus.process(cloudClient);
+      if (rsp.getRequestStatus() == RequestStatusState.COMPLETED) {
+        success = true;
+        break;
+      }
+      assertNotSame(rsp.getRequestStatus(), RequestStatusState.FAILED);
+      Thread.sleep(500);
+    }
+    assertTrue(success);
+    assertEquals("should be one less core on the source node!", sourceNumCores - 1, getNumOfCores(cloudClient, replica.getNodeName(), coll, replica.getType().name()));
+    assertEquals("should be one more core on target node!", targetNumCores + 1, getNumOfCores(cloudClient, targetNode, coll, replica.getType().name()));
+    // wait for recovery
+    boolean recovered = false;
+    for (int i = 0; i < 300; i++) {
+      DocCollection collState = getCollectionState(coll);
+      log.debug("###### {}", collState);
+      Collection<Replica> replicas = collState.getSlice(shardId).getReplicas();
+      boolean allActive = true;
+      boolean hasLeaders = true;
+      if (replicas != null && !replicas.isEmpty()) {
+        for (Replica r : replicas) {
+          if (!r.getNodeName().equals(targetNode)) {
+            continue;
+          }
+          if (!r.isActive(Collections.singleton(targetNode))) {
+            log.info("Not active: {}", r);
+            allActive = false;
+          }
+        }
+      } else {
+        allActive = false;
+      }
+      for (Slice slice : collState.getSlices()) {
+        if (slice.getLeader() == null) {
+          hasLeaders = false;
+        }
+      }
+      if (allActive && hasLeaders) {
+        // check the number of active replicas
+        assertEquals("total number of replicas", REPLICATION, replicas.size());
+        recovered = true;
+        break;
+      } else {
+        log.info("--- waiting, allActive={}, hasLeaders={}", allActive, hasLeaders);
+        Thread.sleep(1000);
+      }
+    }
+    assertTrue("replica never fully recovered", recovered);
+
+    assertEquals(100, cluster.getSolrClient().query(coll, new SolrQuery("*:*")).getResults().getNumFound());
+
+    moveReplica = createMoveReplicaRequest(coll, replica, targetNode, shardId);
+    moveReplica.setInPlaceMove(inPlaceMove);
+    moveReplica.process(cloudClient);
+    checkNumOfCores(cloudClient, replica.getNodeName(), coll, sourceNumCores);
+    // wait for recovery
+    recovered = false;
+    for (int i = 0; i < 300; i++) {
+      DocCollection collState = getCollectionState(coll);
+      log.debug("###### {}", collState);
+      Collection<Replica> replicas = collState.getSlice(shardId).getReplicas();
+      boolean allActive = true;
+      boolean hasLeaders = true;
+      if (replicas != null && !replicas.isEmpty()) {
+        for (Replica r : replicas) {
+          if (!r.getNodeName().equals(replica.getNodeName())) {
+            continue;
+          }
+          if (!r.isActive(Collections.singleton(replica.getNodeName()))) {
+            log.info("Not active yet: {}", r);
+            allActive = false;
+          }
+        }
+      } else {
+        allActive = false;
+      }
+      for (Slice slice : collState.getSlices()) {
+        if (slice.getLeader() == null) {
+          hasLeaders = false;
+        }
+      }
+      if (allActive && hasLeaders) {
+        assertEquals("total number of replicas", REPLICATION, replicas.size());
+        recovered = true;
+        break;
+      } else {
+        Thread.sleep(1000);
+      }
+    }
+    assertTrue("replica never fully recovered", recovered);
+
+    assertEquals(100, cluster.getSolrClient().query(coll, new SolrQuery("*:*")).getResults().getNumFound());
+  }
+
+  //Commented out 5-Dec-2017
+  // @AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-11458")
+  @Test
+  // 12-Jun-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 17-Mar-2018 This JIRA is fixed, but this test still fails
+  //17-Aug-2018 commented  @LuceneTestCase.BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 2-Aug-2018
+  // commented out on: 17-Feb-2019   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // annotated on: 24-Dec-2018
+  public void testFailedMove() throws Exception {
+    String coll = getTestClass().getSimpleName() + "_failed_coll_" + inPlaceMove;
+    int REPLICATION = 2;

Review comment:
       *UnusedVariable:*  The local variable 'REPLICATION' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractUnloadDistributedZkTestBase.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.cloud;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.common.SolrInputDocument;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrPaths;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.Test;
+
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractUnloadDistributedZkTestBase extends AbstractBasicDistributedZkTestBase {
+  public AbstractUnloadDistributedZkTestBase() {
+    super();
+  }
+
+  protected String getSolrXml() {
+    return "solr.xml";
+  }
+
+  @Test
+  public void test() throws Exception {
+    jettys.forEach(j -> {
+      Set<Path> allowPath = j.getCoreContainer().getAllowPaths();
+      allowPath.clear();
+      allowPath.add(SolrPaths.ALL_PATH); // Allow non-standard core instance path
+    });
+    testCoreUnloadAndLeaders(); // long
+    testUnloadLotsOfCores(); // long
+
+    testUnloadShardAndCollection();
+  }
+
+  private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
+                                                  boolean shouldBePresent, int expectedSliceCount) throws Exception {
+    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    Boolean isPresent = null; // null meaning "don't know"
+    while (null == isPresent || shouldBePresent != isPresent) {
+      final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);
+      final Collection<Slice> slices = (docCollection != null) ? docCollection.getSlices() : Collections.emptyList();
+      if (timeout.hasTimedOut()) {
+        printLayout();
+        fail("checkCoreNamePresenceAndSliceCount failed:"
+            +" collection="+collectionName+" CoreName="+coreName
+            +" shouldBePresent="+shouldBePresent+" isPresent="+isPresent
+            +" expectedSliceCount="+expectedSliceCount+" actualSliceCount="+slices.size());
+      }
+      if (expectedSliceCount == slices.size()) {
+        isPresent = false;
+        for (Slice slice : slices) {
+          for (Replica replica : slice.getReplicas()) {
+            if (coreName.equals(replica.get("core"))) {
+              isPresent = true;
+            }
+          }
+        }
+      }
+      Thread.sleep(1000);
+    }
+  }
+
+  private void testUnloadShardAndCollection() throws Exception{
+    final int numShards = 2;
+
+    final String collection = "test_unload_shard_and_collection";
+
+    final String coreName1 = collection+"_1";
+    final String coreName2 = collection+"_2";
+
+    assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard1")
+        .setCoreName(coreName1)
+        .setNode(jettys.get(0).getNodeName())
+        .process(cloudClient).isSuccess());
+
+    assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard2")
+        .setCoreName(coreName2)
+        .setNode(jettys.get(0).getNodeName())
+        .process(cloudClient).isSuccess());
+
+
+    // does not mean they are active and up yet :*
+    waitForRecoveriesToFinish(collection, false);
+
+    final boolean unloadInOrder = random().nextBoolean();
+    final String unloadCmdCoreName1 = (unloadInOrder ? coreName1 : coreName2);
+    final String unloadCmdCoreName2 = (unloadInOrder ? coreName2 : coreName1);
+
+    try (HttpSolrClient adminClient = getHttpSolrClient(buildUrl(jettys.get(0).getLocalPort()))) {
+      // now unload one of the two
+      Unload unloadCmd = new Unload(false);
+      unloadCmd.setCoreName(unloadCmdCoreName1);
+      adminClient.request(unloadCmd);
+
+      // there should still be two shards (as of SOLR-5209)
+      checkCoreNamePresenceAndSliceCount(collection, unloadCmdCoreName1, false /* shouldBePresent */, numShards /* expectedSliceCount */);
+
+      // now unload one of the other
+      unloadCmd = new Unload(false);
+      unloadCmd.setCoreName(unloadCmdCoreName2);
+      adminClient.request(unloadCmd);
+      checkCoreNamePresenceAndSliceCount(collection, unloadCmdCoreName2, false /* shouldBePresent */, numShards /* expectedSliceCount */);
+    }
+
+    //printLayout();
+    // the collection should still be present (as of SOLR-5209 replica removal does not cascade to remove the slice and collection)
+    assertTrue("No longer found collection "+collection, getCommonCloudSolrClient().getZkStateReader().getClusterState().hasCollection(collection));
+  }
+
+  protected SolrCore getFirstCore(String collection, JettySolrRunner jetty) {
+    SolrCore solrCore = null;
+    for (SolrCore core : jetty.getCoreContainer().getCores()) {
+      if (core.getName().startsWith(collection)) {
+        solrCore = core;
+      }
+    }
+    return solrCore;
+  }
+
+  /**
+   * @throws Exception on any problem
+   */
+  private void testCoreUnloadAndLeaders() throws Exception {
+    JettySolrRunner jetty1 = jettys.get(0);
+
+    assertEquals(0, CollectionAdminRequest
+        .createCollection("unloadcollection", "conf1", 1,1)
+        .setCreateNodeSet(jetty1.getNodeName())
+        .process(cloudClient).getStatus());
+    ZkStateReader zkStateReader = getCommonCloudSolrClient().getZkStateReader();
+
+    zkStateReader.forceUpdateCollection("unloadcollection");
+
+    int slices = zkStateReader.getClusterState().getCollection("unloadcollection").getSlices().size();
+    assertEquals(1, slices);
+    SolrCore solrCore = getFirstCore("unloadcollection", jetty1);
+    String core1DataDir = solrCore.getDataDir();
+
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard("unloadcollection", "shard1")
+        .setCoreName("unloadcollection_shard1_replica2")
+        .setNode(jettys.get(1).getNodeName())
+        .process(cloudClient).isSuccess());
+    zkStateReader.forceUpdateCollection("unloadcollection");
+    slices = zkStateReader.getClusterState().getCollection("unloadcollection").getSlices().size();
+    assertEquals(1, slices);
+
+    waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);
+
+    ZkCoreNodeProps leaderProps = getLeaderUrlFromZk("unloadcollection", "shard1");
+
+    Random random = random();
+    if (random.nextBoolean()) {
+      try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getCoreUrl())) {
+        // lets try and use the solrj client to index and retrieve a couple
+        // documents
+        SolrInputDocument doc1 = getDoc(id, 6, i1, -600, tlong, 600, t1,
+            "humpty dumpy sat on a wall");
+        SolrInputDocument doc2 = getDoc(id, 7, i1, -600, tlong, 600, t1,
+            "humpty dumpy3 sat on a walls");
+        SolrInputDocument doc3 = getDoc(id, 8, i1, -600, tlong, 600, t1,
+            "humpty dumpy2 sat on a walled");
+        collectionClient.add(doc1);
+        collectionClient.add(doc2);
+        collectionClient.add(doc3);
+        collectionClient.commit();
+      }
+    }
+
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard("unloadcollection", "shard1")
+        .setCoreName("unloadcollection_shard1_replica3")
+        .setNode(jettys.get(2).getNodeName())
+        .process(cloudClient).isSuccess());
+
+    waitForRecoveriesToFinish("unloadcollection", zkStateReader, false);
+
+    // so that we start with some versions when we reload...
+    TestInjection.skipIndexWriterCommitOnClose = true;
+
+    try (HttpSolrClient addClient = getHttpSolrClient(jettys.get(2).getBaseUrl() + "/unloadcollection_shard1_replica3", 30000)) {
+
+      // add a few docs
+      for (int x = 20; x < 100; x++) {
+        SolrInputDocument doc1 = getDoc(id, x, i1, -600, tlong, 600, t1,
+            "humpty dumpy sat on a wall");
+        addClient.add(doc1);
+      }
+    }
+    // don't commit so they remain in the tran log
+    //collectionClient.commit();
+
+    // unload the leader
+    try (HttpSolrClient collectionClient = getHttpSolrClient(leaderProps.getBaseUrl(), 15000, 30000)) {
+
+      Unload unloadCmd = new Unload(false);
+      unloadCmd.setCoreName(leaderProps.getCoreName());
+      ModifiableSolrParams p = (ModifiableSolrParams) unloadCmd.getParams();

Review comment:
       *UnusedVariable:*  The local variable 'p' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       *UnusedVariable:*  The field 'log' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/hadoop/HadoopTestUtil.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.cloud.hadoop;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.time.FastDateFormat;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HadoopTestUtil {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       *UnusedVariable:*  The field 'log' is never read. [(details)](https://errorprone.info/bugpattern/UnusedVariable)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractTlogReplayBufferedWhileIndexingTestBase.java
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.cloud;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.util.TestInjection;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public abstract class AbstractTlogReplayBufferedWhileIndexingTestBase extends AbstractFullDistribZkTestBase {
+
+  private List<StoppableIndexingThread> threads;
+
+  public AbstractTlogReplayBufferedWhileIndexingTestBase() throws Exception {
+    super();
+    sliceCount = 1;
+    fixShardCount(2);
+    schemaString = "schema15.xml";      // we need a string id
+  }
+
+  @BeforeClass
+  public static void beforeRestartWhileUpdatingTest() throws Exception {
+    System.setProperty("leaderVoteWait", "300000");
+    System.setProperty("solr.autoCommit.maxTime", "10000");
+    System.setProperty("solr.autoSoftCommit.maxTime", "3000");
+    TestInjection.updateLogReplayRandomPause = "true:10";
+    TestInjection.updateRandomPause = "true:10";
+    if (System.getProperty("solr.hdfs.home") != null) useFactory("solr.StandardDirectoryFactory");
+  }
+
+  @AfterClass
+  public static void afterRestartWhileUpdatingTest() {
+    System.clearProperty("leaderVoteWait");
+    System.clearProperty("solr.autoCommit.maxTime");
+    System.clearProperty("solr.autoSoftCommit.maxTime");
+  }
+
+  @Test
+  public void test() throws Exception {
+    handle.clear();
+    handle.put("timestamp", SKIPVAL);
+
+    waitForRecoveriesToFinish(false);
+
+    int numThreads = 3;
+
+    threads = new ArrayList<>(numThreads);
+
+    ArrayList<JettySolrRunner> allJetty = new ArrayList<>();
+    allJetty.addAll(jettys);
+    allJetty.remove(shardToLeaderJetty.get("shard1").jetty);
+    assert allJetty.size() == 1 : allJetty.size();

Review comment:
       *UseCorrectAssertInTests:*  Java assert is used in test. For testing purposes Assert.* matchers should be used. [(details)](https://errorprone.info/bugpattern/UseCorrectAssertInTests)
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792130743



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore

##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792131123



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792131051



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794991015



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,12 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      // if the update log class is not defined in the plugin info / solrconfig.xml
+      // (like <updateLog class="${solr.ulog:solr.UpdateLog}"> )
+      // we fall back use the one which is the default for the given directory factory
+      ulog = ulogPluginInfo.className == null ? dirFactory.newDefaultUpdateLog() :

Review comment:
       Ooh; I missed line 132.  Thanks.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794727164



##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  implementation ('com.github.ben-manes.caffeine:caffeine')
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:solrj')
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.slf4j:jcl-over-slf4j'
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:modules:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // used by the hadoop-specific test framework classes
+  testImplementation ('org.apache.commons:commons-compress') { transitive = false }
+  testImplementation ('org.apache.commons:commons-collections4') { transitive = false }
+  testImplementation ('org.apache.commons:commons-lang3') { transitive = false }
+  testImplementation ('com.sun.jersey:jersey-servlet') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')
+}
+
+
+// Copy all the test resource files from core to the build/resources/test directory
+// of the HDFS module so we can avoid duplication of the test resource files like
+// schemas and SolrConfigs. This can be improved later by making the test classes
+// load the resources from core directories directly.
+task copySolrCoreTestResources(type: Copy) {
+  from('../../core/src/test-files') {

Review comment:
       Can we do `from project(':solr:core').sourceSets.test.resources.srcDirs` here? Not sure if dependency resolution actually allows it or not.

##########
File path: solr/core/build.gradle
##########
@@ -146,11 +146,6 @@ dependencies {
   testImplementation 'org.apache.commons:commons-compress'
   testImplementation ('com.sun.jersey:jersey-servlet') { transitive = false }
 
-  // HDFS

Review comment:
       Can we remove all of the Hadoop stuff above this paragraph as well?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794920667



##########
File path: solr/core/build.gradle
##########
@@ -146,11 +146,6 @@ dependencies {
   testImplementation 'org.apache.commons:commons-compress'
   testImplementation ('com.sun.jersey:jersey-servlet') { transitive = false }
 
-  // HDFS

Review comment:
       So sadly no we can't straight up remove them. I did however reorganize them under `hadoop-auth` to be clear and added to the HDFS module the necessary runtimeOnly dependencies for when `hadoop-auth` does get removed from core - we don't break the HDFS module.
   
   Addressed in ae67b144357113a9ee80ab981ee1279242c53242




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r793724999



##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:modules:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')

Review comment:
       @sonatype-lift ignore

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794903379



##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  implementation ('com.github.ben-manes.caffeine:caffeine')
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:solrj')
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.slf4j:jcl-over-slf4j'
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:modules:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // used by the hadoop-specific test framework classes
+  testImplementation ('org.apache.commons:commons-compress') { transitive = false }
+  testImplementation ('org.apache.commons:commons-collections4') { transitive = false }
+  testImplementation ('org.apache.commons:commons-lang3') { transitive = false }
+  testImplementation ('com.sun.jersey:jersey-servlet') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')
+}
+
+
+// Copy all the test resource files from core to the build/resources/test directory
+// of the HDFS module so we can avoid duplication of the test resource files like
+// schemas and SolrConfigs. This can be improved later by making the test classes
+// load the resources from core directories directly.
+task copySolrCoreTestResources(type: Copy) {
+  from('../../core/src/test-files') {

Review comment:
       Done in https://github.com/apache/solr/pull/324/commits/843fc717695608b2f41dc2c8e4f8417e5a183344




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r789009008



##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }

Review comment:
       Agreed we should remove the `transitive = false` since it causes some pain, but it could be done separately.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1017779969


   BTW I went through all the files and there aren't THAT many files that need to be reviewed for real changes.
   * ~344/447 are just test-files - a lot could be removed?
   * ~57/447 - are primarily just only moves
   * 46 are real changes


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1026197123


   Merged in https://github.com/apache/solr/commit/cb2e58f6569704c45387a8d094625ba8e014776b


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792962631



##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }

Review comment:
       I've been looking at this as part of PR #553 and some other related dependency improvements. It doesn't make sense to "fix" this as part of this move. This should be done post moving to a module.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792076791



##########
File path: solr/contrib/hdfs/.gitignore
##########
@@ -0,0 +1,2 @@
+# IDEA project file
+hdfs.iml

Review comment:
       Done (https://github.com/apache/solr/pull/324/commits/e0a4b5bcd355554e2a59fc240123338e05a97716)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792126971



##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/hadoop/HadoopTestUtil.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.cloud.hadoop;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.time.FastDateFormat;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HadoopTestUtil {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r730319810



##########
File path: solr/contrib/hdfs/.gitignore
##########
@@ -0,0 +1,2 @@
+# IDEA project file
+hdfs.iml

Review comment:
       Remove please; this is not specific to this module.

##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module
+===============================
+
+Introduction
+------------
+This contrib module implements the support for Hadoop Distributed File System in Apache Solr. 
+
+
+Building
+--------
+The HDFS contrib module uses the same Gradle build as the core Solr components. 
+
+To build the module, you can use
+
+./gradlew :solr:contrib:hdfs:assemble
+
+The resulting module will be placed to the libs directory, for example:
+solr/contrib/hdfs/build/libs/solr-hdfs-9.0.0-SNAPSHOT.jar
+
+
+To execute the module tests:
+
+./gradlew :solr:contrib:hdfs:test
+
+
+
+Usage
+-----
+To use the module, it needs to be placed to the Solr web application classpath (for example by symlinking it to the WEB-INF/lib directory.)

Review comment:
       I suggest we don't recommend people putting libs into WEB-INF/lib unless it has to go there. <solr-home>/lib is the idea spot for libs that plug-in.

##########
File path: gradle/validation/rat-sources.gradle
##########
@@ -102,6 +102,12 @@ allprojects {
                     exclude "src/test-files/META-INF/services/*"
                     break
 
+                case ":solr:contrib:hdfs":

Review comment:
       Hopefully we can remove this part if we can remove the duplicated test stuff

##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes

Review comment:
       Please add a comment and URL to acknowledge this situation isn't good; it's a work-around.  Point to the JIRA comment where you explained why we have these dependencies.  This comment could also be placed on the abstract test base classes you moved to the test-framework module.  I could imagine someone looking at this and wonder what’s going on; the comment will help.  At some future time perhaps Gradle will fix the cross-test subclassing limitation and someone might fix this.

##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,9 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      ulog = ulogPluginInfo.className == null ? new UpdateLog():

Review comment:
       given that UpdateHandler no longer auto-picks the HdfsUpdateLog, do users need to do something different for it to be chosen?  (Should be documented if so)

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:contrib:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')

Review comment:
       I’m unclear what the embedded use-case is that keeps this on the runtime class path instead of only for tests.  I changed it to testRuntimeOnly and tests pass (to be expected I suppose).  Can you clarify?

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }

Review comment:
       You are probably copy-pasting some of the dependency lines from solr-core.  Historically Solr has specified each dependency (even transitive) explicitly but Gradle allows us to use transitive dependencies while still allowing us to notice we're adding them via our versions.lock mechanism.  For example, perhaps Hadoop-common is a transitive of hadoop-hdfs (I dunno; just a hypothetical) and so you wouldn't need to even mention Hadoop-common).  The point isn't to reduce the # of lines here but to help with dependency maintenance as they evolve and change their dependencies.  For example maybe Hadoop 3.3.x adds some new dependency or removes a dependency.  If we use transitives then it's easier for us to maintain this and not keep what we don't actually need.  If you're not up for it; I could give it a shot.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r793130704



##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-common@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-common@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }

Review comment:
       *Moderate OSS Vulnerability:*
   ### pkg:maven/com.google.guava/guava@25.1-jre
   0 Critical, 0 Severe, 1 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/com.google.guava/guava@25.1-jre</b></summary>
           <ul>
     <details>
       <summary><b>MODERATE Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-8908] A temp directory creation vulnerability exists in all versions of Guava, allowin...
   > A temp directory creation vulnerability exists in all versions of Guava, allowing an attacker with access to the machine to potentially access data in a temporary directory created by the Guava API com.google.common.io.Files.createTempDir(). By default, on unix-like systems, the created directory is world-readable (readable by an attacker with access to the system). The method in question has been marked @Deprecated in versions 30.0 and later and should not be used. For Android developers, we recommend choosing a temporary directory API provided by Android, such as context.getCacheDir(). For other Java developers, we recommend migrating to the Java 7 API java.nio.file.Files.createTempDirectory() which explicitly configures permissions of 700, or configuring the Java runtime&#39;s java.io.tmpdir system property to point to a location whose permissions are appropriately configured.
   >
   > **CVSS Score:** 3.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:L/AC:L/PR:L/UI:N/S:U/C:L/I:N/A:N
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:modules:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')

Review comment:
       *Severe OSS Vulnerability:*
   ### pkg:maven/org.apache.zookeeper/zookeeper@3.7.0
   0 Critical, 2 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.zookeeper/zookeeper@3.7.0</b></summary>
           <ul>
     <details>
       <summary><b>SEVERE Vulnerabilities (2)</b></summary><br/>
   <ul>
   <details>
               <summary>CVE-2021-28164</summary>
   
   > #### [CVE-2021-28164] In Eclipse Jetty 9.4.37.v20210219 to 9.4.38.v20210224, the default compliance mo...
   > In Eclipse Jetty 9.4.37.v20210219 to 9.4.38.v20210224, the default compliance mode allows requests with URIs that contain %2e or %2e%2e segments to access protected resources within the WEB-INF directory. For example a request to /context/%2e/WEB-INF/web.xml can retrieve the web.xml file. This can reveal sensitive information regarding the implementation of a web application.
   >
   > **CVSS Score:** 5.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N
   
   </details>
   <details>
               <summary>CVE-2021-34429</summary>
   
   > #### [CVE-2021-34429] For Eclipse Jetty versions 9.4.37-9.4.42, 10.0.1-10.0.5 & 11.0.1-11.0.5, URIs ca...
   > For Eclipse Jetty versions 9.4.37-9.4.42, 10.0.1-10.0.5 &amp; 11.0.1-11.0.5, URIs can be crafted using some encoded characters to access the content of the WEB-INF directory and/or bypass some security constraints. This is a variation of the vulnerability reported in CVE-2021-28164/GHSA-v7ff-8wcx-gmc5.
   >
   > **CVSS Score:** 5.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N
   
   </details>
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r790960401



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,9 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      ulog = ulogPluginInfo.className == null ? new UpdateLog():

Review comment:
       Having the DirectoryFactory help pick the UpdateLog makes total sense to me.  Though I don't understand why it simply can't instantiate it.  You said "don't have access to the update log type here" -- what do you mean?  It should be public.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk closed pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk closed pull request #324:
URL: https://github.com/apache/solr/pull/324


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1024628290


   I ran `gradlew assemble` and took a look at the output in `solr/packaging/build`... There were only 3 jars in `modules/hdfs/lib/` - I'm not sure what I expected but for some reason I imagined there would be more?
   
   Hadoop-auth, -annotations, and -common are all still in WEB-INF/lib, do we need all of those? I guess auth depends on common and we haven't moved that out yet?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794633440



##########
File path: gradle/validation/validate-source-patterns.gradle
##########
@@ -108,6 +108,11 @@ allprojects {
 configure(project(':solr:core')) {
   project.tasks.withType(ValidateSourcePatternsTask) {
     sourceFiles.exclude 'src/**/CheckLoggingConfiguration.java'
+  }
+}
+
+configure(project(':solr:modules:hdfs')) {
+  project.tasks.withType(ValidateSourcePatternsTask) {

Review comment:
       Actually this can be fixed. There were previously `//nowarn` but needs to be `// nowarn_valid_logger` apparently now. Will push the fix shortly.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794644433



##########
File path: solr/solr-ref-guide/src/solr-on-hdfs.adoc
##########
@@ -16,11 +16,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-WARNING: Storing indexes in HDFS is deprecated and may be be removed in 9.0.
-This functionality may be moved to a 3rd-party plugin in the future.
 
 Solr has support for writing and reading its index and transaction log files to the HDFS distributed filesystem.
 
+Starting from 9.0, this functionality has been moved from Solr Core to the HDFS contrib module. 

Review comment:
       Fixed in 9675773a7e4ebdd2a794a021db187576571d2db8




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794661450



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,12 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      // if the update log class is not defined in the plugin info / solrconfig.xml
+      // (like <updateLog class="${solr.ulog:solr.UpdateLog}"> )
+      // we fall back use the one which is the default for the given directory factory
+      ulog = ulogPluginInfo.className == null ? dirFactory.newDefaultUpdateLog() :

Review comment:
       @dsmiley  not sure I understand your comment here. The original logic:
   
   ```
   DirectoryFactory dirFactory = core.getDirectoryFactory();
         if (dirFactory instanceof HdfsDirectoryFactory) {
           ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
         } else {
           ulog = ulogPluginInfo.className == null ? new UpdateLog():
                   core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
         }
   ```
   
   If `ulogPluginInfo.className == null` then `new UpdateLog();`. So I don't think there is any change here to the logic for `UpdateLog()`. Line 132 has `ulog.init(ulogPluginInfo);` which I think makes sure the ulog implementation takes into account the `uLogPluginInfo`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r789003197



##########
File path: solr/core/src/test/org/apache/solr/core/TestBackupRepositoryFactory.java
##########
@@ -132,13 +131,7 @@ public void testRepositoryConfig() {
       attrs.put("location", "/tmp");
       plugins[0] = new PluginInfo("repository", attrs);
     }
-    {
-      Map<String, Object> attrs = new HashMap<>();
-      attrs.put(CoreAdminParams.NAME, "boom");
-      attrs.put(FieldType.CLASS_NAME, HdfsBackupRepository.class.getName());
-      attrs.put("location", "/tmp");
-      plugins[1] = new PluginInfo("repository", attrs);
-    }

Review comment:
       Not sure I understand this removal? This changes the test for no reason?

##########
File path: gradle/documentation/render-javadoc.gradle
##########
@@ -151,8 +151,25 @@ configure(project(":solr:test-framework")) {
         "org.apache.solr.analysis",
         "org.apache.solr.cloud",
         "org.apache.solr.core",
+        "org.apache.solr.handler",

Review comment:
       Agree with @madrob here

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }

Review comment:
       Agreed we should remove the `transitive = false` since it causes some pain.

##########
File path: gradle/documentation/render-javadoc.gradle
##########
@@ -151,8 +151,25 @@ configure(project(":solr:test-framework")) {
         "org.apache.solr.analysis",
         "org.apache.solr.cloud",
         "org.apache.solr.core",
+        "org.apache.solr.handler",
         "org.apache.solr.handler.component",
         "org.apache.solr.update.processor",
+        "org.apache.solr.util",
+        "org.apache.solr.search.similarities",
+        "org.apache.solr.search.function",
+        "org.apache.solr.search.facet",
+        "org.apache.solr.schema"

Review comment:
       Is this needed?

##########
File path: gradle/validation/rat-sources.gradle
##########
@@ -102,6 +102,12 @@ allprojects {
                     exclude "src/test-files/META-INF/services/*"
                     break
 
+                case ":solr:contrib:hdfs":

Review comment:
       Agreed this should be cleaned up. I'd be surprised if any of these test files are needed for HDFS.

##########
File path: solr/contrib/hdfs/.gitignore
##########
@@ -0,0 +1,2 @@
+# IDEA project file
+hdfs.iml

Review comment:
       Agreed this should be removed.

##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module

Review comment:
       Agree w/ @madrob - there is already a `solr/contrib/hdfs/README.md` as well? Should be consolidated.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r790769421



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,9 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      ulog = ulogPluginInfo.className == null ? new UpdateLog():

Review comment:
       I had a discussion with @risdenk on this one. Indeed the code currently picks the standard UpdateLog.class unless the sorlconfig.xml explicitely specifies the class of the updateLog definition. This can lead to the update logs being written to the local FS while the index of the same collection is being written to HDFS. While this is technically possible (it worked during my basic tests), this is likely not what we want. 
   The Solr tests use the solr.tests.ulog variable for this, I took care of the correct setting for Hdfs tests.
   <updateLog class="${solr.tests.ulog:solr.UpdateLog}">
   This should be documented. 
   With Kevin we also discussed that we could look into the Hdfs directory factory intialization and verify if we can check if an inconsistent update log class is used and error out in this case. I checked the directory factory code and to me it seems that we have no access to the update log type there, likely because they can also be defined per update handler and initialized later.
   I think we could also consider the following: add a new method like getDefaultUpdateLogClassName to the DirectoryFactory class. This method which would return "solr.UpdateLog" in the base class, and the HdfsDirectoryFactory could override this by returning "solr.HdfsUpdateLog". In the UpdateHandler part, if we detect that the solrconfig did not specify any class, we could fall back to the one returned by this method. This would ensure compatibility with the existing solrconfigs for HDFS and non-HDFS configs (less pain during an upgrade), and at the same time it would allow the user to override the value in SolrConfig. @risdenk , @dsmiley what do you think? 
   
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792127392



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r793853588



##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:modules:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.

##########
File path: solr/modules/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794672231



##########
File path: gradle/documentation/render-javadoc.gradle
##########
@@ -151,8 +151,25 @@ configure(project(":solr:test-framework")) {
         "org.apache.solr.analysis",
         "org.apache.solr.cloud",
         "org.apache.solr.core",
+        "org.apache.solr.handler",
         "org.apache.solr.handler.component",
         "org.apache.solr.update.processor",
+        "org.apache.solr.util",
+        "org.apache.solr.search.similarities",
+        "org.apache.solr.search.function",
+        "org.apache.solr.search.facet",
+        "org.apache.solr.schema"

Review comment:
       Ok so I looked into this. A handful of classes were moved from solr core into solr test-framework due to limitations in gradle not being able to share test classes. The move ended up in a few more split packages - which causes issues in the render-javadoc. So these exclusions are due to the new split packages. I didn't see a simple way to address this related to this change. 
   
   I think test-framework should probably look at how it does packages (maybe need to at `.test.` or something to the package name) to avoid split packages.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794671450



##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module
+===============================
+
+Introduction
+------------
+This contrib module implements the support for Hadoop Distributed File System in Apache Solr. 
+
+
+Building
+--------
+The HDFS contrib module uses the same Gradle build as the core Solr components. 
+
+To build the module, you can use
+
+./gradlew :solr:contrib:hdfs:assemble
+
+The resulting module will be placed to the libs directory, for example:
+solr/contrib/hdfs/build/libs/solr-hdfs-9.0.0-SNAPSHOT.jar
+
+
+To execute the module tests:
+
+./gradlew :solr:contrib:hdfs:test
+
+
+
+Usage
+-----
+To use the module, it needs to be placed to the Solr web application classpath (for example by symlinking it to the WEB-INF/lib directory.)

Review comment:
       Yea this needs to be addressed. I'm marking this as resolved since you have another comment on the correct file location. So its not going to get lost.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-950059934


   > (me) I'm most concerned with the apparent duplication of tons of Solr test configs. How can we avoid that?
   
   I'd be a hack but I wonder if this contrib could declare a test resource "input" pointing to a sub-directory of solr-core test configs?  I'd take almost any hack over duplicating all that test config.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794787601



##########
File path: solr/modules/hdfs/src/test/org/apache/solr/store/blockcache/BlockCacheTest.java
##########
@@ -257,7 +259,7 @@ public void testCacheConcurrent() throws Exception {
           return;
         }
       }
-      assertEquals("cache key differs from value's key", k, (Long) v.key);
+      Assert.assertEquals("cache key differs from value's key", k, (Long) v.key);

Review comment:
       Fixed in e75d6ac1bc6c6ee7777f23ea0b2b8877928b86e6




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1024642024


   I've been looking through and I _think_ that we can exclude Hadoop-annotations and leave it in only as a test dependency? I'm not sure.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1024691338


   Thanks @madrob and @dsmiley. I think your comments have been addressed by @warperwolf and I. Any other questions/comments/concerns/thoughts?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1024662117


   @madrob I'm with you I expected more jars to be there in `hdfs/lib`, but `hadoop-auth`, `hadoop-annotations`, and `hadoop-common` are all needed by the existing Hadoop authentication logic. 
   
   FWIW removing `hadoop-annotations` fails with:
   
   ```
   > Task :solr:core:compileJava
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-common/3.2.0/e47a88c42c450e6e4b23bf951356c203cae2db24/hadoop-common-3.2.0.jar(/org/apache/hadoop/fs/FileSystem.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate': class file for org.apache.hadoop.classification.InterfaceAudience not found
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-common/3.2.0/e47a88c42c450e6e4b23bf951356c203cae2db24/hadoop-common-3.2.0.jar(/org/apache/hadoop/fs/FileSystem.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate'
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-common/3.2.0/e47a88c42c450e6e4b23bf951356c203cae2db24/hadoop-common-3.2.0.jar(/org/apache/hadoop/fs/Path.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate'
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-common/3.2.0/e47a88c42c450e6e4b23bf951356c203cae2db24/hadoop-common-3.2.0.jar(/org/apache/hadoop/security/UserGroupInformation.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate'
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-common/3.2.0/e47a88c42c450e6e4b23bf951356c203cae2db24/hadoop-common-3.2.0.jar(/org/apache/hadoop/security/UserGroupInformation.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate'
   /Users/risdenk/.gradle/caches/modules-2/files-2.1/org.apache.hadoop/hadoop-auth/3.2.0/b1b95aed9aa956ffb7d21e30a0415ca14d91c4ad/hadoop-auth-3.2.0.jar(/org/apache/hadoop/security/authentication/util/KerberosName.class): warning: Cannot find annotation method 'value()' in type 'LimitedPrivate'
   error: warnings found and -Werror specified
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r790893626



##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractBasicDistributedZk2TestBase.java
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.cloud;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.mockfile.FilterPath;
+import org.apache.solr.client.solrj.SolrClient;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.handler.BackupStatusChecker;
+import org.apache.solr.handler.ReplicationHandler;
+import org.junit.Test;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractBasicDistributedZk2TestBase extends AbstractFullDistribZkTestBase {
+  private static final String SHARD2 = "shard2";
+  private static final String SHARD1 = "shard1";
+  private static final String ONE_NODE_COLLECTION = "onenodecollection";
+  private final boolean onlyLeaderIndexes = random().nextBoolean();
+
+
+  public AbstractBasicDistributedZk2TestBase() {
+    super();
+    // we need DVs on point fields to compute stats & facets
+    if (Boolean.getBoolean(NUMERIC_POINTS_SYSPROP)) System.setProperty(NUMERIC_DOCVALUES_SYSPROP,"true");
+
+    sliceCount = 2;
+  }
+
+  @Override
+  protected boolean useTlogReplicas() {
+    return false; // TODO: tlog replicas makes commits take way to long due to what is likely a bug and it's TestInjection use
+  }
+
+  @Test
+  @ShardsFixed(num = 4)
+  public void test() throws Exception {
+    boolean testFinished = false;
+    try {
+      handle.clear();
+      handle.put("timestamp", SKIPVAL);
+
+      testNodeWithoutCollectionForwarding();
+
+      indexr(id, 1, i1, 100, tlong, 100, t1,
+          "now is the time for all good men", "foo_f", 1.414f, "foo_b", "true",
+          "foo_d", 1.414d);
+
+      commit();
+
+      // make sure we are in a steady state...
+      waitForRecoveriesToFinish(false);
+
+      assertDocCounts(false);
+
+      indexAbunchOfDocs();
+
+      // check again
+      waitForRecoveriesToFinish(false);
+
+      commit();
+
+      assertDocCounts(VERBOSE);
+      checkQueries();
+
+      assertDocCounts(VERBOSE);
+
+      query("q", "*:*", "sort", "n_tl1 desc");
+
+      bringDownShardIndexSomeDocsAndRecover();
+
+      query("q", "*:*", "sort", "n_tl1 desc");
+
+      // test adding another replica to a shard - it should do a
+      // recovery/replication to pick up the index from the leader
+      addNewReplica();
+
+      long docId = testUpdateAndDelete();
+
+      // index a bad doc...
+      expectThrows(SolrException.class, () -> indexr(t1, "a doc with no id"));
+
+      // TODO: bring this to its own method?
+      // try indexing to a leader that has no replicas up
+      ZkStateReader zkStateReader = cloudClient.getZkStateReader();
+      ZkNodeProps leaderProps = zkStateReader.getLeaderRetry(
+          DEFAULT_COLLECTION, SHARD2);
+
+      String nodeName = leaderProps.getStr(ZkStateReader.NODE_NAME_PROP);
+      chaosMonkey.stopShardExcept(SHARD2, nodeName);
+
+      SolrClient client = getClient(nodeName);
+
+      index_specific(client, "id", docId + 1, t1, "what happens here?");
+
+      // expire a session...
+      CloudJettyRunner cloudJetty = shardToJetty.get(SHARD1).get(0);
+      chaosMonkey.expireSession(cloudJetty.jetty);
+
+      indexr("id", docId + 1, t1, "slip this doc in");
+
+      waitForRecoveriesToFinish(false);
+
+      checkShardConsistency(SHARD1);
+      checkShardConsistency(SHARD2);
+
+      testFinished = true;
+    } finally {
+      if (!testFinished) {
+        printLayoutOnTearDown = true;
+      }
+    }
+
+  }
+
+  private void testNodeWithoutCollectionForwarding() throws Exception {
+    assertEquals(0, CollectionAdminRequest
+        .createCollection(ONE_NODE_COLLECTION, "conf1", 1, 1)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard(ONE_NODE_COLLECTION, "shard1")
+        .setCoreName(ONE_NODE_COLLECTION + "core")
+        .process(cloudClient).isSuccess());
+
+    waitForCollection(cloudClient.getZkStateReader(), ONE_NODE_COLLECTION, 1);
+    waitForRecoveriesToFinish(ONE_NODE_COLLECTION, cloudClient.getZkStateReader(), false);
+
+    cloudClient.getZkStateReader().getLeaderRetry(ONE_NODE_COLLECTION, SHARD1, 30000);
+
+    int docs = 2;
+    for (SolrClient client : clients) {
+      final String clientUrl = getBaseUrl((HttpSolrClient) client);
+      addAndQueryDocs(clientUrl, docs);
+      docs += 2;
+    }
+  }
+
+  // 2 docs added every call
+  private void addAndQueryDocs(final String baseUrl, int docs)
+      throws Exception {
+
+    SolrQuery query = new SolrQuery("*:*");
+
+    try (HttpSolrClient qclient = getHttpSolrClient(baseUrl + "/onenodecollection" + "core")) {
+
+      // it might take a moment for the proxy node to see us in their cloud state
+      waitForNon403or404or503(qclient);
+
+      // add a doc
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", docs);
+      qclient.add(doc);
+      qclient.commit();
+
+
+      QueryResponse results = qclient.query(query);
+      assertEquals(docs - 1, results.getResults().getNumFound());
+    }
+
+    try (HttpSolrClient qclient = getHttpSolrClient(baseUrl + "/onenodecollection")) {
+      QueryResponse results = qclient.query(query);
+      assertEquals(docs - 1, results.getResults().getNumFound());
+
+      SolrInputDocument doc = new SolrInputDocument();
+      doc.addField("id", docs + 1);
+      qclient.add(doc);
+      qclient.commit();
+
+      query = new SolrQuery("*:*");
+      query.set("rows", 0);
+      results = qclient.query(query);
+      assertEquals(docs, results.getResults().getNumFound());
+    }
+  }
+
+  private long testUpdateAndDelete() throws Exception {
+    long docId = 99999999L;
+    indexr("id", docId, t1, "originalcontent");
+
+    commit();
+
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.add("q", t1 + ":originalcontent");
+    QueryResponse results = clients.get(0).query(params);
+    assertEquals(1, results.getResults().getNumFound());
+
+    // update doc
+    indexr("id", docId, t1, "updatedcontent");
+
+    commit();
+
+    results = clients.get(0).query(params);
+    assertEquals(0, results.getResults().getNumFound());
+
+    params.set("q", t1 + ":updatedcontent");
+
+    results = clients.get(0).query(params);
+    assertEquals(1, results.getResults().getNumFound());
+
+    UpdateRequest uReq = new UpdateRequest();
+    // uReq.setParam(UpdateParams.UPDATE_CHAIN, DISTRIB_UPDATE_CHAIN);
+    uReq.deleteById(Long.toString(docId)).process(clients.get(0));
+
+    commit();
+
+    results = clients.get(0).query(params);
+    assertEquals(0, results.getResults().getNumFound());
+    return docId;
+  }
+
+  private void bringDownShardIndexSomeDocsAndRecover() throws Exception {
+    SolrQuery query = new SolrQuery("*:*");
+    query.set("distrib", false);
+
+    commit();
+
+    long deadShardCount = shardToJetty.get(SHARD2).get(0).client.solrClient
+        .query(query).getResults().getNumFound();
+
+    query("q", "*:*", "sort", "n_tl1 desc");
+
+    int oldLiveNodes = cloudClient.getZkStateReader().getZkClient().getChildren(ZkStateReader.LIVE_NODES_ZKNODE, null, true).size();
+
+    assertEquals(5, oldLiveNodes);
+
+    // kill a shard
+    CloudJettyRunner deadShard = chaosMonkey.stopShard(SHARD1, 0);
+
+    // ensure shard is dead
+    expectThrows(SolrServerException.class,
+        "This server should be down and this update should have failed",
+        () -> index_specific(deadShard.client.solrClient, id, 999, i1, 107, t1, "specific doc!")
+    );
+
+    commit();
+
+    query("q", "*:*", "sort", "n_tl1 desc");
+
+    // long cloudClientDocs = cloudClient.query(new
+    // SolrQuery("*:*")).getResults().getNumFound();
+    // System.out.println("clouddocs:" + cloudClientDocs);
+
+    // try to index to a living shard at shard2
+
+
+    long numFound1 = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+
+    cloudClient.getZkStateReader().getLeaderRetry(DEFAULT_COLLECTION, SHARD1, 60000);
+
+    try {
+      index_specific(shardToJetty.get(SHARD1).get(1).client.solrClient, id, 1000, i1, 108, t1,
+          "specific doc!");
+    } catch (Exception e) {
+      // wait and try again
+      Thread.sleep(4000);
+      index_specific(shardToJetty.get(SHARD1).get(1).client.solrClient, id, 1000, i1, 108, t1,
+          "specific doc!");
+    }
+
+    commit();
+
+    checkShardConsistency(true, false);
+
+    query("q", "*:*", "sort", "n_tl1 desc");
+
+
+    cloudClient.setDefaultCollection(DEFAULT_COLLECTION);
+
+    long numFound2 = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+
+    assertEquals(numFound1 + 1, numFound2);
+
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.addField("id", 1001);
+
+    controlClient.add(doc);
+
+    // try adding a doc with CloudSolrServer
+    UpdateRequest ureq = new UpdateRequest();
+    ureq.add(doc);
+    // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
+
+    try {
+      ureq.process(cloudClient);
+    } catch(SolrServerException e){
+      // try again
+      Thread.sleep(3500);
+      ureq.process(cloudClient);
+    }
+
+    commit();
+
+    query("q", "*:*", "sort", "n_tl1 desc");
+
+    long numFound3 = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
+
+    // lets just check that the one doc since last commit made it in...
+    assertEquals(numFound2 + 1, numFound3);
+
+    // test debugging
+    testDebugQueries();
+
+    if (VERBOSE) {
+      System.err.println(controlClient.query(new SolrQuery("*:*")).getResults()
+          .getNumFound());
+
+      for (SolrClient client : clients) {
+        try {
+          SolrQuery q = new SolrQuery("*:*");
+          q.set("distrib", false);
+          System.err.println(client.query(q).getResults()
+              .getNumFound());
+        } catch (Exception e) {
+
+        }
+      }
+    }
+    // TODO: This test currently fails because debug info is obtained only
+    // on shards with matches.
+    // query("q","matchesnothing","fl","*,score", "debugQuery", "true");
+
+    // this should trigger a recovery phase on deadShard
+    deadShard.jetty.start();
+
+    // make sure we have published we are recovering
+    Thread.sleep(1500);
+
+    waitForRecoveriesToFinish(false);
+
+    deadShardCount = shardToJetty.get(SHARD1).get(0).client.solrClient
+        .query(query).getResults().getNumFound();
+    // if we properly recovered, we should now have the couple missing docs that
+    // came in while shard was down
+    checkShardConsistency(true, false);
+
+
+    // recover over 100 docs so we do more than just peer sync (replicate recovery)
+    chaosMonkey.stopJetty(deadShard);
+
+    for (int i = 0; i < 226; i++) {
+      doc = new SolrInputDocument();
+      doc.addField("id", 2000 + i);
+      controlClient.add(doc);
+      ureq = new UpdateRequest();
+      ureq.add(doc);
+      // ureq.setParam("update.chain", DISTRIB_UPDATE_CHAIN);
+      ureq.process(cloudClient);
+    }
+    commit();
+
+    Thread.sleep(1500);
+
+    deadShard.jetty.start();
+
+    // make sure we have published we are recovering
+    Thread.sleep(1500);
+
+    waitForThingsToLevelOut(1, TimeUnit.MINUTES);
+
+    Thread.sleep(500);
+
+    waitForRecoveriesToFinish(false);
+
+    checkShardConsistency(true, false);
+
+    // try a backup command
+    try(final HttpSolrClient client = getHttpSolrClient((String) shardToJetty.get(SHARD2).get(0).info.get("base_url"))) {
+      final String backupName = "the_backup";
+      ModifiableSolrParams params = new ModifiableSolrParams();
+      params.set("qt", ReplicationHandler.PATH);
+      params.set("command", "backup");
+      params.set("name", backupName);
+      final Path location = FilterPath.unwrap(createTempDir()).toRealPath();
+      // Allow non-standard location outside SOLR_HOME
+      jettys.forEach(j -> j.getCoreContainer().getAllowPaths().add(location));

Review comment:
       *NULL_DEREFERENCE:*  object returned by `j.getCoreContainer()` could be null and is dereferenced at line 407.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractRecoveryZkTestBase.java
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.cloud;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Slow
+public abstract class AbstractRecoveryZkTestBase extends SolrCloudTestCase {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @BeforeClass
+  public static void setupCluster() throws Exception {
+    configureCluster(2)
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  private final List<StoppableIndexingThread> threads = new ArrayList<>();
+
+  @After
+  public void stopThreads() throws InterruptedException {
+    for (StoppableIndexingThread t : threads) {
+      t.safeStop();
+    }
+    for (StoppableIndexingThread t : threads) {
+      t.join();
+    }
+    threads.clear();
+  }
+
+  @Test
+  //commented 2-Aug-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // 28-June-2018
+  public void test() throws Exception {
+
+    final String collection = "recoverytest";
+
+    CollectionAdminRequest.createCollection(collection, "conf", 1, 2)
+        .process(cluster.getSolrClient());
+    waitForState("Expected a collection with one shard and two replicas", collection, clusterShape(1, 2));
+    cluster.getSolrClient().setDefaultCollection(collection);
+
+    // start a couple indexing threads
+
+    int[] maxDocList = new int[] {300, 700, 1200, 1350, 3000};
+    int[] maxDocNightlyList = new int[] {3000, 7000, 12000, 30000, 45000, 60000};
+
+    int maxDoc;
+    if (!TEST_NIGHTLY) {
+      maxDoc = maxDocList[random().nextInt(maxDocList.length - 1)];
+    } else {
+      maxDoc = maxDocNightlyList[random().nextInt(maxDocList.length - 1)];
+    }
+    log.info("Indexing {} documents", maxDoc);
+
+    final StoppableIndexingThread indexThread
+        = new StoppableIndexingThread(null, cluster.getSolrClient(), "1", true, maxDoc, 1, true);
+    threads.add(indexThread);
+    indexThread.start();
+
+    final StoppableIndexingThread indexThread2
+        = new StoppableIndexingThread(null, cluster.getSolrClient(), "2", true, maxDoc, 1, true);
+    threads.add(indexThread2);
+    indexThread2.start();
+
+    // give some time to index...
+    int[] waitTimes = new int[] {200, 2000, 3000};
+    Thread.sleep(waitTimes[random().nextInt(waitTimes.length - 1)]);
+
+    // bring shard replica down
+    DocCollection state = getCollectionState(collection);
+    Replica leader = state.getLeader("shard1");
+    Replica replica = getRandomReplica(state.getSlice("shard1"), (r) -> leader != r);
+
+    JettySolrRunner jetty = cluster.getReplicaJetty(replica);

Review comment:
       *NULL_DEREFERENCE:*  object `replica` last assigned on line 105 could be null and is dereferenced by call to `getReplicaJetty(...)` at line 107.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractUnloadDistributedZkTestBase.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.cloud;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.common.SolrInputDocument;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrPaths;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.Test;
+
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractUnloadDistributedZkTestBase extends AbstractBasicDistributedZkTestBase {
+  public AbstractUnloadDistributedZkTestBase() {
+    super();
+  }
+
+  protected String getSolrXml() {
+    return "solr.xml";
+  }
+
+  @Test
+  public void test() throws Exception {
+    jettys.forEach(j -> {
+      Set<Path> allowPath = j.getCoreContainer().getAllowPaths();
+      allowPath.clear();
+      allowPath.add(SolrPaths.ALL_PATH); // Allow non-standard core instance path
+    });
+    testCoreUnloadAndLeaders(); // long
+    testUnloadLotsOfCores(); // long
+
+    testUnloadShardAndCollection();
+  }
+
+  private void checkCoreNamePresenceAndSliceCount(String collectionName, String coreName,
+                                                  boolean shouldBePresent, int expectedSliceCount) throws Exception {
+    final TimeOut timeout = new TimeOut(45, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    Boolean isPresent = null; // null meaning "don't know"
+    while (null == isPresent || shouldBePresent != isPresent) {
+      final DocCollection docCollection = getCommonCloudSolrClient().getZkStateReader().getClusterState().getCollectionOrNull(collectionName);
+      final Collection<Slice> slices = (docCollection != null) ? docCollection.getSlices() : Collections.emptyList();
+      if (timeout.hasTimedOut()) {
+        printLayout();
+        fail("checkCoreNamePresenceAndSliceCount failed:"
+            +" collection="+collectionName+" CoreName="+coreName
+            +" shouldBePresent="+shouldBePresent+" isPresent="+isPresent
+            +" expectedSliceCount="+expectedSliceCount+" actualSliceCount="+slices.size());
+      }
+      if (expectedSliceCount == slices.size()) {
+        isPresent = false;
+        for (Slice slice : slices) {
+          for (Replica replica : slice.getReplicas()) {
+            if (coreName.equals(replica.get("core"))) {
+              isPresent = true;
+            }
+          }
+        }
+      }
+      Thread.sleep(1000);
+    }
+  }
+
+  private void testUnloadShardAndCollection() throws Exception{
+    final int numShards = 2;
+
+    final String collection = "test_unload_shard_and_collection";
+
+    final String coreName1 = collection+"_1";
+    final String coreName2 = collection+"_2";
+
+    assertEquals(0, CollectionAdminRequest.createCollection(collection, "conf1", numShards, 1)
+        .setCreateNodeSet("")
+        .process(cloudClient).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard1")
+        .setCoreName(coreName1)
+        .setNode(jettys.get(0).getNodeName())
+        .process(cloudClient).isSuccess());
+
+    assertTrue(CollectionAdminRequest.addReplicaToShard(collection, "shard2")
+        .setCoreName(coreName2)
+        .setNode(jettys.get(0).getNodeName())
+        .process(cloudClient).isSuccess());
+
+
+    // does not mean they are active and up yet :*
+    waitForRecoveriesToFinish(collection, false);
+
+    final boolean unloadInOrder = random().nextBoolean();
+    final String unloadCmdCoreName1 = (unloadInOrder ? coreName1 : coreName2);
+    final String unloadCmdCoreName2 = (unloadInOrder ? coreName2 : coreName1);
+
+    try (HttpSolrClient adminClient = getHttpSolrClient(buildUrl(jettys.get(0).getLocalPort()))) {
+      // now unload one of the two
+      Unload unloadCmd = new Unload(false);
+      unloadCmd.setCoreName(unloadCmdCoreName1);
+      adminClient.request(unloadCmd);
+
+      // there should still be two shards (as of SOLR-5209)
+      checkCoreNamePresenceAndSliceCount(collection, unloadCmdCoreName1, false /* shouldBePresent */, numShards /* expectedSliceCount */);
+
+      // now unload one of the other
+      unloadCmd = new Unload(false);
+      unloadCmd.setCoreName(unloadCmdCoreName2);
+      adminClient.request(unloadCmd);
+      checkCoreNamePresenceAndSliceCount(collection, unloadCmdCoreName2, false /* shouldBePresent */, numShards /* expectedSliceCount */);
+    }
+
+    //printLayout();
+    // the collection should still be present (as of SOLR-5209 replica removal does not cascade to remove the slice and collection)
+    assertTrue("No longer found collection "+collection, getCommonCloudSolrClient().getZkStateReader().getClusterState().hasCollection(collection));
+  }
+
+  protected SolrCore getFirstCore(String collection, JettySolrRunner jetty) {
+    SolrCore solrCore = null;
+    for (SolrCore core : jetty.getCoreContainer().getCores()) {

Review comment:
       *NULL_DEREFERENCE:*  object returned by `jetty.getCoreContainer()` could be null and is dereferenced at line 158.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/AbstractUnloadDistributedZkTestBase.java
##########
@@ -0,0 +1,372 @@
+/*
+ * 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.cloud;
+
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest.Unload;
+import org.apache.solr.common.SolrInputDocument;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.common.util.SolrNamedThreadFactory;
+import org.apache.solr.core.SolrPaths;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.Test;
+
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * This test simply does a bunch of basic things in solrcloud mode and asserts things
+ * work as expected.
+ */
+public abstract class AbstractUnloadDistributedZkTestBase extends AbstractBasicDistributedZkTestBase {
+  public AbstractUnloadDistributedZkTestBase() {
+    super();
+  }
+
+  protected String getSolrXml() {
+    return "solr.xml";
+  }
+
+  @Test
+  public void test() throws Exception {
+    jettys.forEach(j -> {
+      Set<Path> allowPath = j.getCoreContainer().getAllowPaths();

Review comment:
       *NULL_DEREFERENCE:*  object returned by `j.getCoreContainer()` could be null and is dereferenced at line 68.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  abstract String getConfigSet();
+
+  @Before
+  public void setupCluster() throws Exception {
+    // we don't want this test to have zk timeouts
+    System.setProperty("zkClientTimeout", "60000");
+    System.setProperty("createCollectionWaitTimeTillActive", "5");
+    TestInjection.randomDelayInCoreCreation = "true:5";
+    System.setProperty("validateAfterInactivity", "200");
+    System.setProperty("solr.allowPaths", "*");
+
+    configureCluster(4)
+        .addConfig("conf", configset(getConfigSet()))
+        .addConfig("conf2", configset(getConfigSet()))
+        .withSolrXml(TEST_PATH().resolve("solr.xml"))
+        .configure();
+  }
+  
+  @After
+  public void tearDownCluster() throws Exception {
+    try {
+      shutdownCluster();
+    } finally {
+      System.clearProperty("createCollectionWaitTimeTillActive");
+      System.clearProperty("solr.allowPaths");
+      super.tearDown();
+    }
+  }
+
+  @Test
+  public void testCreationAndDeletion() throws Exception {
+    String collectionName = "created_and_deleted";
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 1, 1).process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+        .contains(collectionName));
+
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deleteCollectionRemovesStaleZkCollectionsNode() throws Exception {
+    String collectionName = "out_of_sync_collection";
+
+    // manually create a collections zknode
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient())
+                  .contains(collectionName));
+    
+    assertFalse(cluster.getZkClient().exists(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true));
+  }
+
+  @Test
+  public void deletePartiallyCreatedCollection() throws Exception {
+    final String collectionName = "halfdeletedcollection";
+
+    assertEquals(0, CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    String dataDir = createTempDir().toFile().getAbsolutePath();
+    // create a core that simulates something left over from a partially-deleted collection
+    assertTrue(CollectionAdminRequest
+        .addReplicaToShard(collectionName, "shard1")
+        .setDataDir(dataDir)
+        .process(cluster.getSolrClient()).isSuccess());
+
+    CollectionAdminRequest.deleteCollection(collectionName)
+        .process(cluster.getSolrClient());
+
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void deleteCollectionOnlyInZk() throws Exception {
+    final String collectionName = "onlyinzk";
+
+    // create the collections node, but nothing else
+    cluster.getZkClient().makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/" + collectionName, true);
+
+    // delete via API - should remove collections node
+    CollectionAdminRequest.deleteCollection(collectionName).process(cluster.getSolrClient());
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+    
+    // now creating that collection should work
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 1)
+        .process(cluster.getSolrClient());
+    assertTrue(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains(collectionName));
+  }
+
+  @Test
+  public void testBadActionNames() {
+    // try a bad action
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", "BADACTION");
+    String collectionName = "badactioncollection";
+    params.set("name", collectionName);
+    params.set("numShards", 2);
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingRequiredParameters() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("numShards", 2);
+    // missing required collection parameter
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testMissingNumShards() {
+    // No numShards should fail
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testZeroNumShards() {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set("action", CollectionAction.CREATE.toString());
+    params.set("name", "acollection");
+    params.set(REPLICATION_FACTOR, 10);
+    params.set("numShards", 0);
+    params.set("collection.configName", "conf");
+
+    final QueryRequest request = new QueryRequest(params);
+    request.setPath("/admin/collections");
+    expectThrows(Exception.class, () -> {
+      cluster.getSolrClient().request(request);
+    });
+  }
+
+  @Test
+  public void testCreateShouldFailOnExistingCore() throws Exception {
+    String nn1 = cluster.getJettySolrRunner(0).getNodeName();
+    String nn2 = cluster.getJettySolrRunner(1).getNodeName();
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker", "conf", 1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker", "shard1")
+        .setNode(cluster.getJettySolrRunner(0).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    assertEquals(0, CollectionAdminRequest.createCollection("halfcollectionblocker2", "conf",1, 1)
+        .setCreateNodeSet("")
+        .process(cluster.getSolrClient()).getStatus());
+    assertTrue(CollectionAdminRequest.addReplicaToShard("halfcollectionblocker2", "shard1")
+        .setNode(cluster.getJettySolrRunner(1).getNodeName())
+        .setCoreName("halfcollection_shard1_replica_n1")
+        .process(cluster.getSolrClient()).isSuccess());
+
+    expectThrows(BaseHttpSolrClient.RemoteSolrException.class, () -> {
+      CollectionAdminRequest.createCollection("halfcollection", "conf", 1, 1)
+          .setCreateNodeSet(nn1 + "," + nn2)
+          .process(cluster.getSolrClient());
+    });
+  }
+
+  @Test
+  public void testNoConfigSetExist() throws Exception {
+    expectThrows(Exception.class, () -> {
+      CollectionAdminRequest.createCollection("noconfig", "conf123", 1, 1)
+          .process(cluster.getSolrClient());
+    });
+
+    TimeUnit.MILLISECONDS.sleep(1000);
+    // in both cases, the collection should have default to the core name
+    cluster.getSolrClient().getZkStateReader().forceUpdateCollection("noconfig");
+    assertFalse(CollectionAdminRequest.listCollections(cluster.getSolrClient()).contains("noconfig"));
+  }
+
+  @Test
+  public void testCoresAreDistributedAcrossNodes() throws Exception {
+    CollectionAdminRequest.createCollection("nodes_used_collection", "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    Set<String> liveNodes = cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes();
+
+    List<String> createNodeList = new ArrayList<>(liveNodes);
+
+    DocCollection collection = getCollectionState("nodes_used_collection");
+    for (Slice slice : collection.getSlices()) {
+      for (Replica replica : slice.getReplicas()) {
+        createNodeList.remove(replica.getNodeName());
+      }
+    }
+
+    assertEquals(createNodeList.toString(), 0, createNodeList.size());
+  }
+
+  @Test
+  public void testDeleteNonExistentCollection() throws Exception {
+
+    expectThrows(SolrException.class, () -> {
+      CollectionAdminRequest.deleteCollection("unknown_collection").process(cluster.getSolrClient());
+    });
+
+    // create another collection should still work
+    CollectionAdminRequest.createCollection("acollectionafterbaddelete", "conf", 1, 2)
+        .process(cluster.getSolrClient());
+    waitForState("Collection creation after a bad delete failed", "acollectionafterbaddelete",
+        (n, c) -> DocCollection.isFullyActive(n, c, 1, 2));
+  }
+
+  @Test
+  public void testSpecificConfigsets() throws Exception {
+    CollectionAdminRequest.createCollection("withconfigset2", "conf2", 1, 1).process(cluster.getSolrClient());
+    String configName  = cluster.getSolrClient().getClusterStateProvider().getCollection("withconfigset2").getConfigName();
+    assertEquals("conf2", configName);
+  }
+
+  @Test
+  public void testCreateNodeSet() throws Exception {
+    JettySolrRunner jetty1 = cluster.getRandomJetty(random());
+    JettySolrRunner jetty2 = cluster.getRandomJetty(random());
+
+    List<String> baseUrls = ImmutableList.of(jetty1.getBaseUrl().toString(), jetty2.getBaseUrl().toString());
+
+    CollectionAdminRequest.createCollection("nodeset_collection", "conf", 2, 1)
+        .setCreateNodeSet(baseUrls.get(0) + "," + baseUrls.get(1))
+        .process(cluster.getSolrClient());
+
+    DocCollection collectionState = getCollectionState("nodeset_collection");
+    for (Replica replica : collectionState.getReplicas()) {
+      String replicaUrl = replica.getCoreUrl();
+      boolean matchingJetty = false;
+      for (String jettyUrl : baseUrls) {
+        if (replicaUrl.startsWith(jettyUrl)) {
+          matchingJetty = true;
+        }
+      }
+      if (matchingJetty == false) {
+        fail("Expected replica to be on " + baseUrls + " but was on " + replicaUrl);
+      }
+    }
+  }
+
+  @Test
+  //28-June-2018 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
+  // See: https://issues.apache.org/jira/browse/SOLR-12028 Tests cannot remove files on Windows machines occasionally
+  // commented out on: 24-Dec-2018   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 09-Aug-2018 SOLR-12028
+  public void testCollectionsAPI() throws Exception {
+
+    // create new collections rapid fire
+    int cnt = random().nextInt(TEST_NIGHTLY ? 3 : 1) + 1;
+    CollectionAdminRequest.Create[] createRequests = new CollectionAdminRequest.Create[cnt];
+    
+    class Coll {
+      String name;
+      int numShards;
+      int replicationFactor;
+    }
+    
+    List<Coll> colls = new ArrayList<>();
+
+    for (int i = 0; i < cnt; i++) {
+
+      int numShards = TestUtil.nextInt(random(), 0, cluster.getJettySolrRunners().size()) + 1;
+      int replicationFactor = TestUtil.nextInt(random(), 0, 3) + 1;
+
+      createRequests[i]
+          = CollectionAdminRequest.createCollection("awhollynewcollection_" + i, "conf2", numShards, replicationFactor);
+      createRequests[i].processAsync(cluster.getSolrClient());
+      
+      Coll coll = new Coll();
+      coll.name = "awhollynewcollection_" + i;
+      coll.numShards = numShards;
+      coll.replicationFactor = replicationFactor;
+      colls.add(coll);
+    }
+
+    for (Coll coll : colls) {
+      cluster.waitForActiveCollection(coll.name, coll.numShards, coll.numShards * coll.replicationFactor);
+    }
+
+    waitForStable(cnt, createRequests);
+
+    for (int i = 0; i < cluster.getJettySolrRunners().size(); i++) {
+      checkInstanceDirs(cluster.getJettySolrRunner(i));
+    }
+    
+    String collectionName = createRequests[random().nextInt(createRequests.length)].getCollectionName();
+    
+    // TODO: we should not need this...beast test well when trying to fix
+    Thread.sleep(1000);
+    
+    cluster.getSolrClient().getZkStateReader().forciblyRefreshAllClusterStateSlow();
+
+    new UpdateRequest()
+        .add("id", "6")
+        .add("id", "7")
+        .add("id", "8")
+        .commit(cluster.getSolrClient(), collectionName);
+    long numFound = 0;
+    TimeOut timeOut = new TimeOut(10, TimeUnit.SECONDS, TimeSource.NANO_TIME);
+    while (!timeOut.hasTimedOut()) {
+
+      numFound = cluster.getSolrClient().query(collectionName, new SolrQuery("*:*")).getResults().getNumFound();
+      if (numFound == 3) {
+        break;
+      }
+
+      Thread.sleep(500);
+    }
+    
+    if (timeOut.hasTimedOut()) {
+      fail("Timeout waiting to see 3 found, instead saw " + numFound + " for collection " + collectionName);
+    }
+
+    checkNoTwoShardsUseTheSameIndexDir();
+  }
+
+  private void waitForStable(int cnt, CollectionAdminRequest.Create[] createRequests) throws InterruptedException {
+    for (int i = 0; i < cnt; i++) {
+      String collectionName = "awhollynewcollection_" + i;
+      final int j = i;
+      waitForState("Expected to see collection " + collectionName, collectionName,
+          (n, c) -> {
+            CollectionAdminRequest.Create req = createRequests[j];
+            return DocCollection.isFullyActive(n, c, req.getNumShards(), req.getReplicationFactor());
+          });
+      
+      ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+      // make sure we have leaders for each shard
+      for (int z = 1; z < createRequests[j].getNumShards(); z++) {
+        zkStateReader.getLeaderRetry(collectionName, "shard" + z, 10000);
+      }      // make sure we again have leaders for each shard
+    }
+  }
+
+  @Test
+  public void testCollectionReload() throws Exception {
+    final String collectionName = "reloaded_collection";
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2).process(cluster.getSolrClient());
+
+    // get core open times
+    Map<String, Long> urlToTimeBefore = new HashMap<>();
+    collectStartTimes(collectionName, urlToTimeBefore);
+    assertTrue(urlToTimeBefore.size() > 0);
+
+    CollectionAdminRequest.reloadCollection(collectionName).processAsync(cluster.getSolrClient());
+
+    // reloads make take a short while
+    boolean allTimesAreCorrect = waitForReloads(collectionName, urlToTimeBefore);
+    assertTrue("some core start times did not change on reload", allTimesAreCorrect);
+  }
+
+  private void checkInstanceDirs(JettySolrRunner jetty) throws IOException {
+    CoreContainer cores = jetty.getCoreContainer();
+    Collection<SolrCore> theCores = cores.getCores();

Review comment:
       *NULL_DEREFERENCE:*  object `cores` last assigned on line 472 could be null and is dereferenced at line 473.
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }

Review comment:
       *Moderate OSS Vulnerability:*
   ### pkg:maven/com.google.guava/guava@25.1-jre
   0 Critical, 0 Severe, 1 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/com.google.guava/guava@25.1-jre</b></summary>
           <ul>
     <details>
       <summary><b>MODERATE Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-8908] A temp directory creation vulnerability exists in all versions of Guava, allowin...
   > A temp directory creation vulnerability exists in all versions of Guava, allowing an attacker with access to the machine to potentially access data in a temporary directory created by the Guava API com.google.common.io.Files.createTempDir(). By default, on unix-like systems, the created directory is world-readable (readable by an attacker with access to the system). The method in question has been marked @Deprecated in versions 30.0 and later and should not be used. For Android developers, we recommend choosing a temporary directory API provided by Android, such as context.getCacheDir(). For other Java developers, we recommend migrating to the Java 7 API java.nio.file.Files.createTempDirectory() which explicitly configures permissions of 700, or configuring the Java runtime&#39;s java.io.tmpdir system property to point to a location whose permissions are appropriately configured.
   >
   > **CVSS Score:** 3.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:L/AC:L/PR:L/UI:N/S:U/C:L/I:N/A:N
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Contrib Module'
+
+dependencies {
+  
+  configurations.all {
+    exclude group: 'log4j', module: 'log4j'
+    exclude group: 'commons-logging', module: 'commons-logging'
+    exclude group: 'org.slf4j', module: 'slf4j-log4j12'
+    exclude group: 'org.apache.yetus', module: 'audience-annotations'
+    exclude group: 'org.codehaus.mojo', module: 'animal-sniffer-annotations'
+   // be conservative on what's added here.  Affects *all* configs, including internal ones.
+  }
+
+
+  implementation project(':solr:core')
+
+  // Hadoop dependencies
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+  // Guava implements the VisibleForTesting annotations
+  implementation ('com.google.guava:guava') { transitive = false }
+
+  // Caffeine cache to implement HDFS block caching  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // HdfsCheckindexTool tests reference LuceneTestCase
+  testImplementation ('org.apache.lucene:lucene-test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:contrib:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')
+
+  // commons packages needed by test classes
+  testImplementation('commons-io:commons-io') { transitive = false }
+
+  // Zookeeper dependency - some tests like HdfsCloudBackupRestore need this
+  implementation ('org.apache.zookeeper:zookeeper')

Review comment:
       *Severe OSS Vulnerability:*
   ### pkg:maven/org.apache.zookeeper/zookeeper@3.7.0
   0 Critical, 2 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.zookeeper/zookeeper@3.7.0</b></summary>
           <ul>
     <details>
       <summary><b>SEVERE Vulnerabilities (2)</b></summary><br/>
   <ul>
   <details>
               <summary>CVE-2021-28164</summary>
   
   > #### [CVE-2021-28164] In Eclipse Jetty 9.4.37.v20210219 to 9.4.38.v20210224, the default compliance mo...
   > In Eclipse Jetty 9.4.37.v20210219 to 9.4.38.v20210224, the default compliance mode allows requests with URIs that contain %2e or %2e%2e segments to access protected resources within the WEB-INF directory. For example a request to /context/%2e/WEB-INF/web.xml can retrieve the web.xml file. This can reveal sensitive information regarding the implementation of a web application.
   >
   > **CVSS Score:** 5.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N
   
   </details>
   <details>
               <summary>CVE-2021-34429</summary>
   
   > #### [CVE-2021-34429] For Eclipse Jetty versions 9.4.37-9.4.42, 10.0.1-10.0.5 & 11.0.1-11.0.5, URIs ca...
   > For Eclipse Jetty versions 9.4.37-9.4.42, 10.0.1-10.0.5 &amp; 11.0.1-11.0.5, URIs can be crafted using some encoded characters to access the content of the WEB-INF directory and/or bypass some security constraints. This is a variation of the vulnerability reported in CVE-2021-28164/GHSA-v7ff-8wcx-gmc5.
   >
   > **CVSS Score:** 5.3
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:L/I:N/A:N
   
   </details>
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1023680842


   So current status as I see it:
   * render-javadoc needs a little more attention - basically shouldn't need so many changes
   * `Abstract*Base` test files moved to test-framework - not sure they are needed, but its a lot of the changes in this besides the move.
   
   I think all of the other comments from @dsmiley and @madrob have been addressed by @warperwolf 
   
   I'm planning to look into the above a bit more tomorrow. @warperwolf, @dsmiley, @madrob any other comments/thoughts/concerns? Any other blockers to getting this merged?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792077208



##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module

Review comment:
       Fixed (https://github.com/apache/solr/pull/324/commits/e0a4b5bcd355554e2a59fc240123338e05a97716)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792131065



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/store/hdfs/HdfsFileWriter.java
##########
@@ -30,9 +30,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r730319754



##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module
+===============================
+
+Introduction
+------------
+This contrib module implements the support for Hadoop Distributed File System in Apache Solr. 
+
+
+Building
+--------
+The HDFS contrib module uses the same Gradle build as the core Solr components. 
+
+To build the module, you can use
+
+./gradlew :solr:contrib:hdfs:assemble
+
+The resulting module will be placed to the libs directory, for example:
+solr/contrib/hdfs/build/libs/solr-hdfs-9.0.0-SNAPSHOT.jar
+
+
+To execute the module tests:
+
+./gradlew :solr:contrib:hdfs:test
+
+
+
+Usage
+-----
+To use the module, it needs to be placed to the Solr web application classpath (for example by symlinking it to the WEB-INF/lib directory.)

Review comment:
       I suggest we don't recommend people putting libs into WEB-INF/lib unless it has to go there. solr-home's /lib is the ideal spot for plugins.
   
   Any way, you can just refer to the Solr Ref Guide Page on installing plugins.
   For configuration, we can refer to the Solr Ref Guide again.  This way we're not repeating ourselves.
   I think it would be useful in this doc to list the handful of primary plugins offered here: HdfsDirectoryFactory, HdfsUpdateLog, and HdfsBackupRepository.  Everything else is in servitude to those.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
madrob commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r720538357



##########
File path: solr/contrib/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Contrib module

Review comment:
       Should this be under the hfs contrib directory instead of top level?

##########
File path: gradle/documentation/render-javadoc.gradle
##########
@@ -151,8 +151,25 @@ configure(project(":solr:test-framework")) {
         "org.apache.solr.analysis",
         "org.apache.solr.cloud",
         "org.apache.solr.core",
+        "org.apache.solr.handler",

Review comment:
       Not sure I understand what changes caused this to start failing?

##########
File path: solr/test-framework/src/java/org/apache/solr/handler/package.html
##########
@@ -0,0 +1,24 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->

Review comment:
       We probably want to look at renaming some of the packages for the moved classes, since split packages are going to cause problems down the line anyway. Maybe ok to do this in a follow on issue if it ends up being too much here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r734924006



##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }

Review comment:
       BTW I've been doing #349 after looking at your PR.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-933411418


   > The block cache feature was only used by HDFS, moved to the contrib module)
   
   Is it _possible_ to use it in some other/generic Directory impl?  I haven't looked at it closely but in my mind it was fundamentally intertwined with HDFS but maybe I'm wrong.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r720506575



##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-common@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-common@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.commons:commons-compress') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.commons/commons-compress@1.19
   4 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.commons/commons-compress@1.19</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (4)</b></summary><br/>
   <ul>
   <details>
               <summary>CVE-2021-36090</summary>
   
   > #### [CVE-2021-36090] When reading a specially crafted ZIP archive, Compress can be made to allocate l...
   > When reading a specially crafted ZIP archive, Compress can be made to allocate large amounts of memory that finally leads to an out of memory error even for very small inputs. This could be used to mount a denial of service attack against services that use Compress&#39; zip package.
   >
   > **CVSS Score:** 7.5
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:N/I:N/A:H
   
   </details>
   <details>
               <summary>CVE-2021-35517</summary>
   
   > #### [CVE-2021-35517] When reading a specially crafted TAR archive, Compress can be made to allocate l...
   > When reading a specially crafted TAR archive, Compress can be made to allocate large amounts of memory that finally leads to an out of memory error even for very small inputs. This could be used to mount a denial of service attack against services that use Compress&#39; tar package.
   >
   > **CVSS Score:** 7.5
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:N/I:N/A:H
   
   </details>
   <details>
               <summary>CVE-2021-35515</summary>
   
   > #### [CVE-2021-35515] When reading a specially crafted 7Z archive, the construction of the list of cod...
   > When reading a specially crafted 7Z archive, the construction of the list of codecs that decompress an entry can result in an infinite loop. This could be used to mount a denial of service attack against services that use Compress&#39; sevenz package.
   >
   > **CVSS Score:** 7.5
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:N/I:N/A:H
   
   </details>
   <details>
               <summary>CVE-2021-35516</summary>
   
   > #### [CVE-2021-35516] When reading a specially crafted 7Z archive, Compress can be made to allocate la...
   > When reading a specially crafted 7Z archive, Compress can be made to allocate large amounts of memory that finally leads to an out of memory error even for very small inputs. This could be used to mount a denial of service attack against services that use Compress&#39; sevenz package.
   >
   > **CVSS Score:** 7.5
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:N/UI:N/S:U/C:N/I:N/A:H
   
   </details>
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-auth@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-common@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-common@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)

##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }

Review comment:
       *Critical OSS Vulnerability:*
   ### pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0
   1 Critical, 0 Severe, 0 Moderate, 0 Unknown vulnerabilities have been found across 1 dependencies
   
   <details>
     <summary><b>Components</b></summary><br/>
     <ul>
         <details>
           <summary><b>pkg:maven/org.apache.hadoop/hadoop-hdfs-client@3.2.0</b></summary>
           <ul>
     <details>
       <summary><b>CRITICAL Vulnerabilities (1)</b></summary><br/>
   <ul>
   
   > #### [CVE-2020-9492] In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10....
   > In Apache Hadoop 3.2.0 to 3.2.1, 3.0.0-alpha1 to 3.1.3, and 2.0.0-alpha to 2.10.0, WebHDFS client might send SPNEGO authorization header to remote URL without proper verification.
   >
   > **CVSS Score:** 8.8
   >
   > **CVSS Vector:** CVSS:3.0/AV:N/AC:L/PR:L/UI:N/S:U/C:H/I:H/A:H
   
   </ul>
       </details>
           </ul>
         </details>
     </ul>
   </details>
   (at-me [in a reply](https://help.sonatype.com/lift/talking-to-lift) with `help` or `ignore`)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794669642



##########
File path: solr/test-framework/src/java/org/apache/solr/handler/package.html
##########
@@ -0,0 +1,24 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->

Review comment:
       I think this should be handled as a follow up. There are a bunch of existing split packages in `test-framework` which sadly this PR adds to, but follows the existing convention.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794687908



##########
File path: solr/modules/hdfs/README.md
##########
@@ -0,0 +1,31 @@
+Apache Solr HDFS Module
+===============================
+
+Introduction
+------------
+This module implements the support for Hadoop Distributed File System in Apache Solr. 
+
+Building
+--------
+The HDFS module uses the same Gradle build as the core Solr components. 
+
+To build the module, you can use
+
+```
+./gradlew :solr:modules:hdfs:assemble
+```
+
+The resulting module will be placed to the libs directory, for example:
+`solr/modules/hdfs/build/libs/solr-hdfs-9.0.0-SNAPSHOT.jar`
+
+To execute the module tests:
+
+```
+./gradlew :solr:modules:hdfs:test
+```
+
+Usage
+-----
+To use the module, it needs to be placed to the Solr web application classpath (for example by symlinking it to the WEB-INF/lib directory.)

Review comment:
       Addressed in bb216a74511e8a13787556ff94da35332ca71692




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1017774402


   @warperwolf do you have recent changes/updates on this? I can take a stab at updating and fixing some of the comments.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r735159627



##########
File path: solr/test-framework/src/java/org/apache/solr/handler/package.html
##########
@@ -0,0 +1,24 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->

Review comment:
       > We probably want to look at renaming some of the packages for the moved classes, since split packages are going to cause problems down the line anyway. Maybe ok to do this in a follow on issue if it ends up being too much here.
   
   Thanks for the review! 
   
   @madrob, @dsmiley: Do you think it would make sense to include something like testfixture in the package name of the test fixture classes we put to the test framework? For example org.apache.solr.testfixture.handler.  That way all the fixtures we put into the test framework would be centralized as they would all be subpackages of the org.apache.solr.testfixture and that would also make it easier to relocate them once we fix the problem with gradle vs. test fixtures ( https://issues.apache.org/jira/browse/SOLR-14660?focusedCommentId=17409690&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17409690 ). 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-950379069


   > > (me) I'm most concerned with the apparent duplication of tons of Solr test configs. How can we avoid that?
   > 
   > I'd be a hack but I wonder if this contrib could declare a test resource "input" pointing to a sub-directory of solr-core test configs? I'd take almost any hack over duplicating all that test config.
   
   Good idea @dsmiley. I'll give it a try along with implementing your other suggestions. Thanks for the review!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on pull request #324: SOLR-14660 - move HDFS to a module

Posted by GitBox <gi...@apache.org>.
risdenk commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1026198195


   Big shout out to @warperwolf who did 99% of the heavy lifting here. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] madrob commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
madrob commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1018662773


   I'm concerned about this missing changes to the Hadoop code from the endianness fix, the upgrade to 3.3.1, and whatever else we've had to patch up since this PR was originally opened. What is our strategy for keeping things in sync.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794644098



##########
File path: gradle/validation/validate-source-patterns.gradle
##########
@@ -108,6 +108,11 @@ allprojects {
 configure(project(':solr:core')) {
   project.tasks.withType(ValidateSourcePatternsTask) {
     sourceFiles.exclude 'src/**/CheckLoggingConfiguration.java'
+  }
+}
+
+configure(project(':solr:modules:hdfs')) {
+  project.tasks.withType(ValidateSourcePatternsTask) {

Review comment:
       Fixed in c34e6a9376c4b3bc681ffd45a7829cf710ba82e5




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1022685028


   > > (me) I'm most concerned with the apparent duplication of tons of Solr test configs. How can we avoid that?
   > 
   > I'd be a hack but I wonder if this contrib could declare a test resource "input" pointing to a sub-directory of solr-core test configs? I'd take almost any hack over duplicating all that test config.
   
   @dsmiley @risdenk @madrob in commit https://github.com/apache/solr/pull/324/commits/2ec7d59195238473a7e6e6e7c5fee0112cf20f40 I deleted all test resource files from the hdfs module and I added a gradle step to it which copies all the resource files from solr core's test-files to the build/resources/test directory of the hdfs module. A better solution could be to modify the test classes of the HDFS module to point to the resource files in the core's test-files directory, however the test base classes like SolrTestCaseJ4 load resources through the class loader (so that they end up being loaded from the build/resources/test directory of the module we are testing at the moment), and this resource loading seems to occur at various locations, so it would take some time to tweak these.  For now I'd suggest going with this resource copying solution and change the tests later in a separate ticket. (File copying occurs only on hdfs test execution, and this resolves the original issue of havi
 ng to maintain 2 set of resource files in git.)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794667891



##########
File path: gradle/documentation/render-javadoc.gradle
##########
@@ -151,8 +151,25 @@ configure(project(":solr:test-framework")) {
         "org.apache.solr.analysis",
         "org.apache.solr.cloud",
         "org.apache.solr.core",
+        "org.apache.solr.handler",

Review comment:
       Ok so I looked into this. A handful of classes were moved from solr core into solr test-framework due to limitations in gradle not being able to share test classes. The move ended up in a few more split packages - which causes issues in the render-javadoc. So these exclusions are due to the new split packages. I didn't see a simple way to address this related to this change. 
   
   I think test-framework should probably look at how it does packages (maybe need to at `.test.` or something to the package name) to avoid split packages.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794786304



##########
File path: solr/modules/hdfs/src/test/org/apache/solr/store/blockcache/BlockCacheTest.java
##########
@@ -257,7 +259,7 @@ public void testCacheConcurrent() throws Exception {
           return;
         }
       }
-      assertEquals("cache key differs from value's key", k, (Long) v.key);
+      Assert.assertEquals("cache key differs from value's key", k, (Long) v.key);

Review comment:
       This can be cleaned up. Will push a fix shortly. Looks like its only the `test/org/apache/solr/store` tests




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792126945



##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/api/collections/AbstractCollectionsAPIDistributedZkTestBase.java
##########
@@ -0,0 +1,645 @@
+/*
+ * 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.cloud.api.collections;
+
+import javax.management.MBeanServer;
+import javax.management.MBeanServerFactory;
+import javax.management.ObjectName;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.lang.management.ManagementFactory;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.lucene.util.LuceneTestCase.Slow;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.embedded.JettySolrRunner;
+import org.apache.solr.client.solrj.impl.BaseHttpSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.CoreAdminRequest;
+import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.UpdateRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrException;
+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.ZkCoreNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrInfoBean.Category;
+import org.apache.solr.util.TestInjection;
+import org.apache.solr.util.TimeOut;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.cloud.ZkStateReader.CORE_NAME_PROP;
+import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
+
+/**
+ * Tests the Cloud Collections API.
+ */
+@Slow
+public abstract class AbstractCollectionsAPIDistributedZkTestBase extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       @sonatype-lift ignore

##########
File path: solr/test-framework/src/java/org/apache/solr/cloud/hadoop/HadoopTestUtil.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.cloud.hadoop;
+
+import java.lang.invoke.MethodHandles;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.regex.Pattern;
+
+import org.apache.commons.lang3.time.FastDateFormat;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.HardLink;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.http.HttpServer2;
+import org.apache.hadoop.io.nativeio.NativeIO;
+
+import org.apache.hadoop.util.DiskChecker;
+import org.apache.lucene.util.Constants;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.common.util.SuppressForbidden;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class HadoopTestUtil {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792127370



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/core/backup/repository/HdfsBackupRepository.java
##########
@@ -46,10 +46,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * @deprecated since 8.6
- */
-@Deprecated
 public class HdfsBackupRepository implements BackupRepository {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());

Review comment:
       @sonatype-lift ignore




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] sonatype-lift[bot] commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
sonatype-lift[bot] commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792131153



##########
File path: solr/contrib/hdfs/src/java/org/apache/solr/update/HdfsUpdateLog.java
##########
@@ -44,9 +44,7 @@
 
 /**
  * @lucene.experimental

Review comment:
       I've recorded this as ignored for this pull request. If you change your mind, just comment `@sonatype-lift unignore`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r792281270



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,9 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      ulog = ulogPluginInfo.className == null ? new UpdateLog():

Review comment:
       We discussed this in Slack; either way is fine.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] warperwolf commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
warperwolf commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r793119490



##########
File path: solr/core/src/java/org/apache/solr/update/UpdateHandler.java
##########
@@ -115,12 +115,9 @@ public UpdateHandler(SolrCore core, UpdateLog updateLog)  {
     boolean skipUpdateLog = core.getCoreDescriptor().getCloudDescriptor() != null && !core.getCoreDescriptor().getCloudDescriptor().requiresTransactionLog();
     if (updateLog == null && ulogPluginInfo != null && ulogPluginInfo.isEnabled() && !skipUpdateLog) {
       DirectoryFactory dirFactory = core.getDirectoryFactory();
-      if (dirFactory instanceof HdfsDirectoryFactory) {
-        ulog = new HdfsUpdateLog(((HdfsDirectoryFactory)dirFactory).getConfDir());
-      } else {
-        ulog = ulogPluginInfo.className == null ? new UpdateLog():
-                core.getResourceLoader().newInstance(ulogPluginInfo, UpdateLog.class, true);
-      }
+
+      ulog = ulogPluginInfo.className == null ? new UpdateLog():

Review comment:
       Done in https://github.com/apache/solr/pull/324/commits/9ae46ef4588359f1f8d468120c2cc5b5d20f335b




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r794033698



##########
File path: solr/test-framework/build.gradle
##########
@@ -31,5 +31,16 @@ dependencies {
 
   implementation 'io.dropwizard.metrics:metrics-jetty9'
   implementation 'com.lmax:disruptor'
+
+  // used by the hadoop-specific test framework classes

Review comment:
       This has been cleaned up.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r736055291



##########
File path: solr/test-framework/src/java/org/apache/solr/handler/package.html
##########
@@ -0,0 +1,24 @@
+<!doctype html public "-//w3c//dtd html 4.0 transitional//en">
+<!--
+ 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.
+-->
+<!-- not a package-info.java, because we already defined this package in core/ -->

Review comment:
       I'm okay with a sub-package, though maybe it will raise some package-visibility issues, so I don't love it but don't mind particularly.  If the main point is find-ability then maybe a special comment could do that trick?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on a change in pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on a change in pull request #324:
URL: https://github.com/apache/solr/pull/324#discussion_r730321435



##########
File path: solr/contrib/hdfs/build.gradle
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.
+ */
+
+apply plugin: 'java-library'
+
+description = 'HDFS Package'
+
+dependencies {
+  
+  implementation project(':solr:core')
+
+
+  //implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-annotations') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-auth') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-common') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs-client') { transitive = false }
+  implementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+
+
+  
+  api('com.github.ben-manes.caffeine:caffeine', {
+    exclude group: "org.checkerframework", module: "checker-qual"
+  })
+
+  // Many HDFS tests are using/subclassing test framework classes
+  testImplementation project(':solr:test-framework')
+
+  // hadoop dependencies for tests
+  testImplementation ('org.apache.hadoop:hadoop-hdfs') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-common::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-hdfs::tests') { transitive = false }
+  testImplementation ('org.apache.hadoop:hadoop-minikdc') { transitive = false }
+
+  testImplementation 'org.apache.logging.log4j:log4j-1.2-api'
+
+  // classes like solr.ICUCollationField, used by NNFailoverTest for example.
+  testImplementation project(':solr:contrib:analysis-extras')
+
+  // required for instantiating a Zookeeper server in tests or embedded
+  runtimeOnly ('org.xerial.snappy:snappy-java')

Review comment:
       I’m unclear what the embedded use-case is that keeps this on the runtime class path instead of only for tests.  I changed it to testRuntimeOnly and tests pass (to be expected I suppose).  Can you clarify?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] risdenk edited a comment on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
risdenk edited a comment on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1017779969


   BTW I went through all the files and there aren't THAT many files that need to be reviewed for real changes.
   * ~344/447 are just test-files - a lot could be removed?
   * ~57/447 - are primarily just only moves
   * 46 are real changes (arguably most of these are moves from `***Test` -> `Abstract***TestBase`)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org


[GitHub] [solr] dsmiley commented on pull request #324: SOLR-14660 - move HDFS to a contrib module

Posted by GitBox <gi...@apache.org>.
dsmiley commented on pull request #324:
URL: https://github.com/apache/solr/pull/324#issuecomment-1017959651


   Note that the build.gradle of solr-core has since been improved by me to more clearly organize the dependencies and to use transitive=false much less.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@solr.apache.org
For additional commands, e-mail: issues-help@solr.apache.org